|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.connector.kafka.source.testutils; |
| 20 | + |
| 21 | +import org.apache.flink.api.connector.source.Boundedness; |
| 22 | +import org.apache.flink.api.connector.source.Source; |
| 23 | +import org.apache.flink.connector.kafka.source.KafkaSource; |
| 24 | +import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; |
| 25 | +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; |
| 26 | +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; |
| 27 | +import org.apache.flink.connectors.test.common.external.ExternalContext; |
| 28 | +import org.apache.flink.connectors.test.common.external.SourceSplitDataWriter; |
| 29 | + |
| 30 | +import org.apache.kafka.common.TopicPartition; |
| 31 | +import org.apache.kafka.common.serialization.StringDeserializer; |
| 32 | +import org.testcontainers.containers.KafkaContainer; |
| 33 | + |
| 34 | +import java.util.HashMap; |
| 35 | +import java.util.Map; |
| 36 | +import java.util.concurrent.ThreadLocalRandom; |
| 37 | +import java.util.regex.Pattern; |
| 38 | + |
| 39 | +/** |
| 40 | + * Kafka external context that will create multiple topics with only one partitions as source |
| 41 | + * splits. |
| 42 | + */ |
| 43 | +public class KafkaMultipleTopicExternalContext extends KafkaSingleTopicExternalContext { |
| 44 | + |
| 45 | + private int numTopics = 0; |
| 46 | + |
| 47 | + private final String topicPattern; |
| 48 | + |
| 49 | + private final Map<String, SourceSplitDataWriter<String>> topicNameToSplitWriters = |
| 50 | + new HashMap<>(); |
| 51 | + |
| 52 | + public KafkaMultipleTopicExternalContext(String bootstrapServers) { |
| 53 | + super(bootstrapServers); |
| 54 | + this.topicPattern = |
| 55 | + "kafka-multiple-topic-[0-9]+-" |
| 56 | + + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); |
| 57 | + } |
| 58 | + |
| 59 | + @Override |
| 60 | + public SourceSplitDataWriter<String> createSourceSplitDataWriter() { |
| 61 | + String topicName = getTopicName(); |
| 62 | + createTopic(topicName, 1, (short) 1); |
| 63 | + final KafkaPartitionDataWriter splitWriter = |
| 64 | + new KafkaPartitionDataWriter( |
| 65 | + getKafkaProducerProperties(numTopics), new TopicPartition(topicName, 0)); |
| 66 | + topicNameToSplitWriters.put(topicName, splitWriter); |
| 67 | + numTopics++; |
| 68 | + return splitWriter; |
| 69 | + } |
| 70 | + |
| 71 | + @Override |
| 72 | + public Source<String, ?, ?> createSource(Boundedness boundedness) { |
| 73 | + KafkaSourceBuilder<String> builder = KafkaSource.builder(); |
| 74 | + |
| 75 | + if (boundedness == Boundedness.BOUNDED) { |
| 76 | + builder = builder.setBounded(OffsetsInitializer.latest()); |
| 77 | + } |
| 78 | + |
| 79 | + return builder.setGroupId("flink-kafka-multiple-topic-test") |
| 80 | + .setBootstrapServers(bootstrapServers) |
| 81 | + .setTopicPattern(Pattern.compile(topicPattern)) |
| 82 | + .setDeserializer( |
| 83 | + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) |
| 84 | + .build(); |
| 85 | + } |
| 86 | + |
| 87 | + @Override |
| 88 | + public void close() { |
| 89 | + topicNameToSplitWriters.forEach( |
| 90 | + (topicName, splitWriter) -> { |
| 91 | + try { |
| 92 | + splitWriter.close(); |
| 93 | + deleteTopic(topicName); |
| 94 | + } catch (Exception e) { |
| 95 | + kafkaAdminClient.close(); |
| 96 | + throw new RuntimeException("Cannot close split writer", e); |
| 97 | + } |
| 98 | + }); |
| 99 | + topicNameToSplitWriters.clear(); |
| 100 | + kafkaAdminClient.close(); |
| 101 | + } |
| 102 | + |
| 103 | + private String getTopicName() { |
| 104 | + return topicPattern.replace("[0-9]+", String.valueOf(numTopics)); |
| 105 | + } |
| 106 | + |
| 107 | + @Override |
| 108 | + public String toString() { |
| 109 | + return "Multiple-topics Kafka"; |
| 110 | + } |
| 111 | + |
| 112 | + /** Factory of {@link KafkaSingleTopicExternalContext}. */ |
| 113 | + public static class Factory extends KafkaSingleTopicExternalContext.Factory { |
| 114 | + |
| 115 | + public Factory(KafkaContainer kafkaContainer) { |
| 116 | + super(kafkaContainer); |
| 117 | + } |
| 118 | + |
| 119 | + @Override |
| 120 | + public ExternalContext<String> createExternalContext() { |
| 121 | + return new KafkaMultipleTopicExternalContext(getBootstrapServer()); |
| 122 | + } |
| 123 | + } |
| 124 | +} |
0 commit comments