AHeise commented on a change in pull request #16590: URL: https://github.com/apache/flink/pull/16590#discussion_r688017469
########## File path: flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java ########## @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import org.apache.flink.connector.kafka.source.testutils.KafkaMultipleTopicExternalContext; +import org.apache.flink.connector.kafka.source.testutils.KafkaSingleTopicExternalContext; +import org.apache.flink.connectors.test.common.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalContextFactory; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.TestEnv; +import org.apache.flink.connectors.test.common.testsuites.SourceTestSuiteBase; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.flink.FlinkContainerTestEnvironment; + +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Kafka E2E test based on connector testing framework. */ +public class KafkaSourceE2ECase extends SourceTestSuiteBase<String> { Review comment: This should be ``` public class KafkaSourceE2ECase extends SourceTestSuiteBase<String> { private static final String KAFKA_HOSTNAME = "kafka"; private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:5.5.2"; // Defines TestEnvironment @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment( 1, 6, TestUtils.getResource("kafka-connector.jar").toAbsolutePath().toString(), TestUtils.getResource("kafka-clients.jar").toAbsolutePath().toString()); // Defines ConnectorExternalSystem @ExternalSystem DefaultContainerizedExternalSystem<KafkaContainer> kafka = DefaultContainerizedExternalSystem.builder() .fromContainer(new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) .withNetworkAliases(KAFKA_HOSTNAME)) .bindWithFlinkContainer(flink.getFlinkContainer()) .build(); // Defines 2 External context Factories, so test cases will be invoked twice using these two // kinds of external contexts. @ExternalContextFactory KafkaSingleTopicExternalContext.Factory singleTopic = new KafkaSingleTopicExternalContext.Factory(kafka.getContainer()); // or even just kafka @ExternalContextFactory KafkaMultipleTopicExternalContext.Factory multipleTopic = new KafkaMultipleTopicExternalContext.Factory(kafka.getContainer()); } ``` ########## File path: flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/external/DefaultContainerizedExternalSystem.java ########## @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connectors.test.common.external; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connectors.test.common.TestResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; + +/** + * Default implementation of external system based on container. + * + * @param <C> Type of underlying container + */ +@Experimental +public class DefaultContainerizedExternalSystem<C extends GenericContainer<C>> + implements TestResource { + + private static final Logger LOG = + LoggerFactory.getLogger(DefaultContainerizedExternalSystem.class); + + private final GenericContainer<C> container; Review comment: ```suggestion private final C container; ``` Same with getter. ########## File path: flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/testutils/KafkaMultipleTopicExternalContext.java ########## @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source.testutils; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connectors.test.common.external.ExternalContext; +import org.apache.flink.connectors.test.common.external.SourceSplitDataWriter; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.StringDeserializer; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; + +/** + * Kafka external context that will create multiple topics with only one partitions as source + * splits. + */ +public class KafkaMultipleTopicExternalContext extends KafkaSingleTopicExternalContext { + + private int numTopics = 0; + + private final String topicPattern; + + private final Map<String, SourceSplitDataWriter<String>> topicNameToSplitWriters = + new HashMap<>(); + + public KafkaMultipleTopicExternalContext(String bootstrapServers) { + super(bootstrapServers); + this.topicPattern = + "kafka-multiple-topic-[0-9]+-" + + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); + } + + @Override + public SourceSplitDataWriter<String> createSourceSplitDataWriter() { + String topicName = getTopicName(); + createTopic(topicName, 1, (short) 1); + final KafkaPartitionDataWriter splitWriter = + new KafkaPartitionDataWriter( + getKafkaProducerProperties(numTopics), new TopicPartition(topicName, 0)); + topicNameToSplitWriters.put(topicName, splitWriter); + numTopics++; + return splitWriter; + } + + @Override + public Source<String, ?, ?> createSource(Boundedness boundedness) { + KafkaSourceBuilder<String> builder = KafkaSource.builder(); + + if (boundedness == Boundedness.BOUNDED) { + builder = builder.setBounded(OffsetsInitializer.latest()); + } + + return builder.setGroupId("flink-kafka-multiple-topic-test") + .setBootstrapServers(bootstrapServers) + .setTopicPattern(Pattern.compile(topicPattern)) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .build(); + } + + @Override + public void close() { + topicNameToSplitWriters.forEach( + (topicName, splitWriter) -> { + try { + splitWriter.close(); + deleteTopic(topicName); + } catch (Exception e) { + kafkaAdminClient.close(); + throw new RuntimeException("Cannot close split writer", e); + } + }); + topicNameToSplitWriters.clear(); + kafkaAdminClient.close(); + } + + private String getTopicName() { + return topicPattern.replace("[0-9]+", String.valueOf(numTopics)); + } + + @Override + public String toString() { + return "Multiple-topics Kafka"; + } + + /** . */ + public static class Factory implements ExternalContext.Factory<String> { + + private final String bootstrapServers; + + public Factory(String bootstrapServers) { Review comment: Here we should just get the `KafkaContainer` (or even `ExternalResource<KafkaContainer>`). ########## File path: flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java ########## @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import org.apache.flink.connector.kafka.source.testutils.KafkaMultipleTopicExternalContext; +import org.apache.flink.connector.kafka.source.testutils.KafkaSingleTopicExternalContext; +import org.apache.flink.connectors.test.common.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalContextFactory; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.TestEnv; +import org.apache.flink.connectors.test.common.testsuites.SourceTestSuiteBase; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.flink.FlinkContainerTestEnvironment; + +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Kafka E2E test based on connector testing framework. */ +public class KafkaSourceE2ECase extends SourceTestSuiteBase<String> { + private static final String KAFKA_HOSTNAME = "kafka"; + private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:5.5.2"; + + // Defines TestEnvironment + @TestEnv FlinkContainerTestEnvironment flink; + + // Defines ConnectorExternalSystem + @ExternalSystem DefaultContainerizedExternalSystem<KafkaContainer> kafka; + + // Defines 2 External context Factories, so test cases will be invoked twice using these two + // kinds of external contexts. + @ExternalContextFactory KafkaSingleTopicExternalContext.Factory singleTopic; + @ExternalContextFactory KafkaMultipleTopicExternalContext.Factory multipleTopic; + + /** Instantiate and preparing test resources. */ + public KafkaSourceE2ECase() throws Exception { + + // Flink on containers + flink = + new FlinkContainerTestEnvironment( + 1, + 6, + TestUtils.getResource("kafka-connector.jar").toAbsolutePath().toString(), + TestUtils.getResource("kafka-clients.jar").toAbsolutePath().toString()); + + // Kafka on container has to be bound with Flink containers so that they can access each + // other through internal network, so we have to launch Flink containers here. + flink.startUp(); Review comment: This should be unnecessary. Isn't the dependsOn in `Builder#bindWithFlinkContainer` enough? ########## File path: flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/testutils/KafkaMultipleTopicExternalContext.java ########## @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kafka.source.testutils; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connectors.test.common.external.ExternalContext; +import org.apache.flink.connectors.test.common.external.SourceSplitDataWriter; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.StringDeserializer; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; + +/** + * Kafka external context that will create multiple topics with only one partitions as source + * splits. + */ +public class KafkaMultipleTopicExternalContext extends KafkaSingleTopicExternalContext { + + private int numTopics = 0; + + private final String topicPattern; + + private final Map<String, SourceSplitDataWriter<String>> topicNameToSplitWriters = + new HashMap<>(); + + public KafkaMultipleTopicExternalContext(String bootstrapServers) { + super(bootstrapServers); + this.topicPattern = + "kafka-multiple-topic-[0-9]+-" + + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); + } + + @Override + public SourceSplitDataWriter<String> createSourceSplitDataWriter() { + String topicName = getTopicName(); + createTopic(topicName, 1, (short) 1); + final KafkaPartitionDataWriter splitWriter = + new KafkaPartitionDataWriter( + getKafkaProducerProperties(numTopics), new TopicPartition(topicName, 0)); + topicNameToSplitWriters.put(topicName, splitWriter); + numTopics++; + return splitWriter; + } + + @Override + public Source<String, ?, ?> createSource(Boundedness boundedness) { + KafkaSourceBuilder<String> builder = KafkaSource.builder(); + + if (boundedness == Boundedness.BOUNDED) { + builder = builder.setBounded(OffsetsInitializer.latest()); + } + + return builder.setGroupId("flink-kafka-multiple-topic-test") + .setBootstrapServers(bootstrapServers) + .setTopicPattern(Pattern.compile(topicPattern)) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .build(); + } + + @Override + public void close() { + topicNameToSplitWriters.forEach( + (topicName, splitWriter) -> { + try { + splitWriter.close(); + deleteTopic(topicName); + } catch (Exception e) { + kafkaAdminClient.close(); + throw new RuntimeException("Cannot close split writer", e); + } + }); + topicNameToSplitWriters.clear(); + kafkaAdminClient.close(); + } + + private String getTopicName() { + return topicPattern.replace("[0-9]+", String.valueOf(numTopics)); + } + + @Override + public String toString() { + return "Multiple-topics Kafka"; + } + + /** . */ + public static class Factory implements ExternalContext.Factory<String> { + + private final String bootstrapServers; + + public Factory(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + @Override + public ExternalContext<String> createExternalContext() { + return new KafkaMultipleTopicExternalContext(bootstrapServers); Review comment: Here we should extract the bootstrap server. (I'm assuming now that the external resources are running or we can easily force it while calling this factory) ########## File path: flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/external/DefaultContainerizedExternalSystem.java ########## @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connectors.test.common.external; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connectors.test.common.TestResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; + +/** + * Default implementation of external system based on container. + * + * @param <C> Type of underlying container + */ +@Experimental +public class DefaultContainerizedExternalSystem<C extends GenericContainer<C>> + implements TestResource { + + private static final Logger LOG = + LoggerFactory.getLogger(DefaultContainerizedExternalSystem.class); + + private final GenericContainer<C> container; + + /** + * Get a builder for {@link DefaultContainerizedExternalSystem}. + * + * @param <C> Type of underlying container + * @return An instance of builder + */ + public static <C extends GenericContainer<C>> Builder<C> builder() { + return new Builder<>(); + } + + private DefaultContainerizedExternalSystem(GenericContainer<C> container) { + this.container = container; + } + + @Override + public void startUp() throws Exception { + if (container.isRunning()) { + return; + } + container.start(); + } + + @Override + public void tearDown() throws Exception { + if (!container.isRunning()) { + return; + } + container.stop(); + } + + public GenericContainer<C> getContainer() { + return this.container; + } + + /** + * Builder for {@link DefaultContainerizedExternalSystem}. + * + * @param <C> Type of underlying container + */ + public static class Builder<C extends GenericContainer<C>> { + + private GenericContainer<C> container; + private GenericContainer<?> flinkContainer; + + public Builder<C> fromContainer(GenericContainer<C> container) { + this.container = container; + return this; + } Review comment: ```suggestion public <T extends GenericContainer<T>> Builder<T> fromContainer( GenericContainer<T> container) { @SuppressWarnings("unchecked") Builder<T> self = (Builder<T>) this; self.container = container; return self; } ``` This allows us to define kafka without `<KafkaContainer>builder` ``` DefaultContainerizedExternalSystem<KafkaContainer> kafka = DefaultContainerizedExternalSystem.builder() .fromContainer( new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) .withNetworkAliases(KAFKA_HOSTNAME)) // captures the type here! .bindWithFlinkContainer(flink.getFlinkContainer()) .build(); ``` ########## File path: flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java ########## @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import org.apache.flink.connector.kafka.source.testutils.KafkaMultipleTopicExternalContext; +import org.apache.flink.connector.kafka.source.testutils.KafkaSingleTopicExternalContext; +import org.apache.flink.connectors.test.common.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalContextFactory; +import org.apache.flink.connectors.test.common.junit.annotations.ExternalSystem; +import org.apache.flink.connectors.test.common.junit.annotations.TestEnv; +import org.apache.flink.connectors.test.common.testsuites.SourceTestSuiteBase; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.flink.FlinkContainerTestEnvironment; + +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Kafka E2E test based on connector testing framework. */ +public class KafkaSourceE2ECase extends SourceTestSuiteBase<String> { + private static final String KAFKA_HOSTNAME = "kafka"; + private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:5.5.2"; + + // Defines TestEnvironment + @TestEnv FlinkContainerTestEnvironment flink; + + // Defines ConnectorExternalSystem + @ExternalSystem DefaultContainerizedExternalSystem<KafkaContainer> kafka; + + // Defines 2 External context Factories, so test cases will be invoked twice using these two + // kinds of external contexts. + @ExternalContextFactory KafkaSingleTopicExternalContext.Factory singleTopic; + @ExternalContextFactory KafkaMultipleTopicExternalContext.Factory multipleTopic; + + /** Instantiate and preparing test resources. */ + public KafkaSourceE2ECase() throws Exception { + + // Flink on containers + flink = + new FlinkContainerTestEnvironment( + 1, + 6, + TestUtils.getResource("kafka-connector.jar").toAbsolutePath().toString(), + TestUtils.getResource("kafka-clients.jar").toAbsolutePath().toString()); + + // Kafka on container has to be bound with Flink containers so that they can access each + // other through internal network, so we have to launch Flink containers here. + flink.startUp(); + + final KafkaContainer kafkaContainer = + new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) + .withNetworkAliases(KAFKA_HOSTNAME); + kafka = + DefaultContainerizedExternalSystem.<KafkaContainer>builder() + .fromContainer(kafkaContainer) + .bindWithFlinkContainer(flink.getFlinkContainer()) + .build(); + + // The construction of Kafka external context requires bootstrap servers of Kafka, so we + // start Kafka cluster here to get bootstrap servers for the external context. + // Kafka on container + kafka.startUp(); Review comment: This should be unnecessary as well. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
