|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.kafka.clients.admin; |
| 19 | + |
| 20 | +import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaSourceSplit; |
| 21 | +import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaSourceSplitEnumerator; |
| 22 | + |
| 23 | +import org.apache.kafka.common.KafkaFuture; |
| 24 | +import org.apache.kafka.common.TopicPartition; |
| 25 | +import org.apache.kafka.common.TopicPartitionInfo; |
| 26 | + |
| 27 | +import org.junit.jupiter.api.Assertions; |
| 28 | +import org.junit.jupiter.api.BeforeEach; |
| 29 | +import org.junit.jupiter.api.Test; |
| 30 | +import org.mockito.Mockito; |
| 31 | + |
| 32 | +import java.util.Arrays; |
| 33 | +import java.util.Collections; |
| 34 | +import java.util.HashMap; |
| 35 | +import java.util.List; |
| 36 | +import java.util.Map; |
| 37 | +import java.util.Optional; |
| 38 | +import java.util.concurrent.ExecutionException; |
| 39 | + |
| 40 | +class KafkaSourceSplitEnumeratorTest { |
| 41 | + |
| 42 | + AdminClient adminClient = Mockito.mock(KafkaAdminClient.class); |
| 43 | + // prepare |
| 44 | + TopicPartition partition = new TopicPartition("test", 0); |
| 45 | + |
| 46 | + @BeforeEach |
| 47 | + void init() { |
| 48 | + |
| 49 | + Mockito.when(adminClient.listOffsets(Mockito.any(java.util.Map.class))) |
| 50 | + .thenReturn( |
| 51 | + new ListOffsetsResult( |
| 52 | + new HashMap< |
| 53 | + TopicPartition, |
| 54 | + KafkaFuture<ListOffsetsResult.ListOffsetsResultInfo>>() { |
| 55 | + { |
| 56 | + put( |
| 57 | + partition, |
| 58 | + KafkaFuture.completedFuture( |
| 59 | + new ListOffsetsResult.ListOffsetsResultInfo( |
| 60 | + 0, 0, Optional.of(0)))); |
| 61 | + } |
| 62 | + })); |
| 63 | + Mockito.when(adminClient.describeTopics(Mockito.any(java.util.Collection.class))) |
| 64 | + .thenReturn( |
| 65 | + DescribeTopicsResult.ofTopicNames( |
| 66 | + new HashMap<String, KafkaFuture<TopicDescription>>() { |
| 67 | + { |
| 68 | + put( |
| 69 | + partition.topic(), |
| 70 | + KafkaFuture.completedFuture( |
| 71 | + new TopicDescription( |
| 72 | + partition.topic(), |
| 73 | + false, |
| 74 | + Collections.singletonList( |
| 75 | + new TopicPartitionInfo( |
| 76 | + 0, |
| 77 | + null, |
| 78 | + Collections |
| 79 | + .emptyList(), |
| 80 | + Collections |
| 81 | + .emptyList()))))); |
| 82 | + } |
| 83 | + })); |
| 84 | + } |
| 85 | + |
| 86 | + @Test |
| 87 | + void addSplitsBack() { |
| 88 | + // test |
| 89 | + Map<TopicPartition, KafkaSourceSplit> assignedSplit = |
| 90 | + new HashMap<TopicPartition, KafkaSourceSplit>() { |
| 91 | + { |
| 92 | + put(partition, new KafkaSourceSplit(null, partition)); |
| 93 | + } |
| 94 | + }; |
| 95 | + Map<TopicPartition, KafkaSourceSplit> pendingSplit = new HashMap<>(); |
| 96 | + List<KafkaSourceSplit> splits = Arrays.asList(new KafkaSourceSplit(null, partition)); |
| 97 | + KafkaSourceSplitEnumerator enumerator = |
| 98 | + new KafkaSourceSplitEnumerator(adminClient, pendingSplit, assignedSplit); |
| 99 | + enumerator.addSplitsBack(splits, 1); |
| 100 | + Assertions.assertTrue(pendingSplit.size() == splits.size()); |
| 101 | + Assertions.assertNull(assignedSplit.get(partition)); |
| 102 | + Assertions.assertTrue(pendingSplit.get(partition).getEndOffset() == 0); |
| 103 | + } |
| 104 | + |
| 105 | + @Test |
| 106 | + void addStreamingSplitsBack() { |
| 107 | + // test |
| 108 | + Map<TopicPartition, KafkaSourceSplit> assignedSplit = |
| 109 | + new HashMap<TopicPartition, KafkaSourceSplit>() { |
| 110 | + { |
| 111 | + put(partition, new KafkaSourceSplit(null, partition)); |
| 112 | + } |
| 113 | + }; |
| 114 | + Map<TopicPartition, KafkaSourceSplit> pendingSplit = new HashMap<>(); |
| 115 | + List<KafkaSourceSplit> splits = |
| 116 | + Collections.singletonList(new KafkaSourceSplit(null, partition)); |
| 117 | + KafkaSourceSplitEnumerator enumerator = |
| 118 | + new KafkaSourceSplitEnumerator(adminClient, pendingSplit, assignedSplit, true); |
| 119 | + enumerator.addSplitsBack(splits, 1); |
| 120 | + Assertions.assertEquals(pendingSplit.size(), splits.size()); |
| 121 | + Assertions.assertNull(assignedSplit.get(partition)); |
| 122 | + Assertions.assertTrue(pendingSplit.get(partition).getEndOffset() == Long.MAX_VALUE); |
| 123 | + } |
| 124 | + |
| 125 | + @Test |
| 126 | + void addStreamingSplits() throws ExecutionException, InterruptedException { |
| 127 | + // test |
| 128 | + Map<TopicPartition, KafkaSourceSplit> assignedSplit = |
| 129 | + new HashMap<TopicPartition, KafkaSourceSplit>(); |
| 130 | + Map<TopicPartition, KafkaSourceSplit> pendingSplit = new HashMap<>(); |
| 131 | + List<KafkaSourceSplit> splits = |
| 132 | + Collections.singletonList(new KafkaSourceSplit(null, partition)); |
| 133 | + KafkaSourceSplitEnumerator enumerator = |
| 134 | + new KafkaSourceSplitEnumerator(adminClient, pendingSplit, assignedSplit, true); |
| 135 | + enumerator.fetchPendingPartitionSplit(); |
| 136 | + Assertions.assertEquals(pendingSplit.size(), splits.size()); |
| 137 | + Assertions.assertNotNull(pendingSplit.get(partition)); |
| 138 | + Assertions.assertTrue(pendingSplit.get(partition).getEndOffset() == Long.MAX_VALUE); |
| 139 | + } |
| 140 | + |
| 141 | + @Test |
| 142 | + void addplits() throws ExecutionException, InterruptedException { |
| 143 | + // test |
| 144 | + Map<TopicPartition, KafkaSourceSplit> assignedSplit = |
| 145 | + new HashMap<TopicPartition, KafkaSourceSplit>(); |
| 146 | + Map<TopicPartition, KafkaSourceSplit> pendingSplit = new HashMap<>(); |
| 147 | + List<KafkaSourceSplit> splits = |
| 148 | + Collections.singletonList(new KafkaSourceSplit(null, partition)); |
| 149 | + KafkaSourceSplitEnumerator enumerator = |
| 150 | + new KafkaSourceSplitEnumerator(adminClient, pendingSplit, assignedSplit, false); |
| 151 | + enumerator.fetchPendingPartitionSplit(); |
| 152 | + Assertions.assertEquals(pendingSplit.size(), splits.size()); |
| 153 | + Assertions.assertNotNull(pendingSplit.get(partition)); |
| 154 | + Assertions.assertTrue(pendingSplit.get(partition).getEndOffset() == 0); |
| 155 | + } |
| 156 | +} |
0 commit comments