From 65e4c74471f087b207244e70c2a2487eb46325c5 Mon Sep 17 00:00:00 2001 From: mas-chen Date: Tue, 8 Aug 2023 18:01:40 -0700 Subject: [PATCH] [FLINK-32416] initial implementation of DynamicKafkaSource with bounded/unbounded support and unit/integration tests add dynamic kafka source docs --- .../docs/connectors/table/dynamic-kafka.md | 141 +++ .../connectors/datastream/dynamic-kafka.md | 141 +++ .../pom.xml | 4 +- .../984f05c0-ec82-405e-9bcc-d202dbe7202e | 357 +++++++ flink-connector-kafka/pom.xml | 5 + .../dynamic/metadata/ClusterMetadata.java | 92 ++ .../metadata/KafkaMetadataService.java | 53 + .../kafka/dynamic/metadata/KafkaStream.java | 94 ++ .../SingleClusterTopicMetadataService.java | 118 +++ .../dynamic/source/DynamicKafkaSource.java | 222 ++++ .../source/DynamicKafkaSourceBuilder.java | 328 ++++++ .../source/DynamicKafkaSourceOptions.java | 60 ++ .../source/GetMetadataUpdateEvent.java | 26 + .../dynamic/source/MetadataUpdateEvent.java | 77 ++ .../DynamicKafkaSourceEnumState.java | 58 ++ ...DynamicKafkaSourceEnumStateSerializer.java | 187 ++++ .../DynamicKafkaSourceEnumerator.java | 539 ++++++++++ .../StoppableKafkaEnumContextProxy.java | 297 ++++++ .../subscriber/KafkaStreamSetSubscriber.java | 42 + .../subscriber/KafkaStreamSubscriber.java | 42 + .../subscriber/StreamPatternSubscriber.java | 53 + .../metrics/KafkaClusterMetricGroup.java | 142 +++ .../KafkaClusterMetricGroupManager.java | 76 ++ .../reader/DynamicKafkaSourceReader.java | 539 ++++++++++ .../KafkaPartitionSplitReaderWrapper.java | 98 ++ .../source/split/DynamicKafkaSourceSplit.java | 85 ++ .../DynamicKafkaSourceSplitSerializer.java | 76 ++ .../kafka/source/KafkaPropertiesUtil.java | 67 ++ .../enumerator/KafkaSourceEnumState.java | 2 +- .../source/DynamicKafkaSourceITTest.java | 694 +++++++++++++ ...micKafkaSourceEnumStateSerializerTest.java | 118 +++ .../DynamicKafkaSourceEnumeratorTest.java | 964 ++++++++++++++++++ .../StoppableKafkaEnumContextProxyTest.java | 210 ++++ ...SingleClusterTopicMetadataServiceTest.java | 117 +++ .../metrics/KafkaClusterMetricGroupTest.java | 95 ++ .../reader/DynamicKafkaSourceReaderTest.java | 316 ++++++ ...DynamicKafkaSourceSplitSerializerTest.java | 47 + .../DynamicKafkaSourceExternalContext.java | 263 +++++ ...amicKafkaSourceExternalContextFactory.java | 63 ++ .../testutils/MockKafkaMetadataService.java | 93 ++ .../kafka/testutils/TwoKafkaContainers.java | 62 ++ .../testutils/YamlFileMetadataService.java | 361 +++++++ .../YamlFileMetadataServiceTest.java | 79 ++ .../kafka/DynamicKafkaSourceTestHelper.java | 229 +++++ .../connectors/kafka/KafkaTestBase.java | 89 +- .../kafka/KafkaTestEnvironment.java | 2 + .../kafka/KafkaTestEnvironmentImpl.java | 7 +- .../src/test/resources/stream-metadata.yaml | 19 + pom.xml | 7 + tools/maven/checkstyle.xml | 4 - 50 files changed, 7847 insertions(+), 13 deletions(-) create mode 100644 docs/content.zh/docs/connectors/table/dynamic-kafka.md create mode 100644 docs/content/docs/connectors/datastream/dynamic-kafka.md create mode 100644 flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java create mode 100644 flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java create mode 100644 flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java create mode 100644 flink-connector-kafka/src/test/resources/stream-metadata.yaml diff --git a/docs/content.zh/docs/connectors/table/dynamic-kafka.md b/docs/content.zh/docs/connectors/table/dynamic-kafka.md new file mode 100644 index 000000000..f00a2f226 --- /dev/null +++ b/docs/content.zh/docs/connectors/table/dynamic-kafka.md @@ -0,0 +1,141 @@ +--- +title: Kafka +weight: 3 +type: docs +aliases: + - /zh/dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and +writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic +fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in +topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without +Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration +with Hybrid Source. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka 3.1.0 >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +## Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "KafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setDeserializer(new SimpleStringSchema()) + .setProperties(properties) + .build(); + +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md new file mode 100644 index 000000000..903a26d6b --- /dev/null +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -0,0 +1,141 @@ +--- +title: Kafka +weight: 3 +type: docs +aliases: + - /dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from and +writing data to Kafka topics from one or more Kafka clusters. This connector achieves this in a dynamic +fashion, without requiring a job restart, using a Kafka metadata service to facilitate changes in +topics and/or clusters. This is especially useful in transparent Kafka cluster addition/removal without +Flink job restart, transparent Kafka topic addition/removal without Flink job restart, and direct integration +with Hybrid Source. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka 3.1.0 >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +## Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "KafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setDeserializer(new SimpleStringSchema()) + .setProperties(properties) + .build(); + +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the Kafka Source documentation. This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from 1+ cluster. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from 1+ clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 94227a4b4..cc0aa1bd9 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -171,7 +171,7 @@ under the License. com.google.guava guava - 32.1.2-jre + ${guava.version} @@ -250,7 +250,7 @@ under the License. com.google.guava guava - 32.1.2-jre + ${guava.version} guava.jar jar ${project.build.directory}/dependencies diff --git a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e new file mode 100644 index 000000000..28b74b8c8 --- /dev/null +++ b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e @@ -0,0 +1,357 @@ +Class extends class in (YamlFileMetadataService.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class extends class in (StreamKafkaShuffleSink.java:0) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:308) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:309) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:51) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:53) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:54) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:134) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:135) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:136) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:137) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) +Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:31) +Constructor (java.lang.Object, java.util.function.Consumer)> calls method in (Recyclable.java:32) +Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) +Constructor (java.lang.Class)> calls constructor ()> in (YamlFileMetadataService.java:270) +Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) +Constructor (java.lang.String, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext)> calls constructor (java.lang.String)> in (StoppableKafkaEnumContextProxy.java:90) +Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:111) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:253) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:251) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1733) +Constructor (java.util.Set)> calls method in (FlinkKafkaProducer.java:1599) +Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1879) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1630) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls constructor ()> in (FlinkKafkaProducerBase.java:120) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:144) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:146) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:584) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:586) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls constructor (org.apache.flink.api.common.eventtime.WatermarkOutput)> in (AbstractFetcher.java:154) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:152) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:156) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:159) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int)> calls method in (AbstractPartitionDiscoverer.java:81) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaConsumerThread, org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback)> calls method in (KafkaConsumerThread.java:539) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:136) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:137) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:138) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:139) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:140) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:142) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int, java.util.Properties)> calls method in (KafkaPartitionDiscoverer.java:50) +Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has generic parameter type >> with type argument depending on in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:45) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:51) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:56) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:57) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:58) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:59) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:61) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:56) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:59) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:71) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:74) +Constructor (org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer)> calls constructor (org.apache.flink.streaming.api.functions.sink.SinkFunction)> in (StreamKafkaShuffleSink.java:35) +Constructor (int, org.apache.flink.api.common.serialization.DeserializationSchema, [I, org.apache.flink.api.common.serialization.DeserializationSchema, [I, boolean, [Lorg.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema$MetadataConverter;, org.apache.flink.api.common.typeinfo.TypeInformation, boolean)> calls method in (DynamicKafkaDeserializationSchema.java:70) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:59) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:63) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:66) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:175) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:156) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:158) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:161) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:162) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:163) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:169) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:170) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:173) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:210) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:194) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:198) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:201) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:203) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:216) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:218) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:220) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:224) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:226) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:70) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:71) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:72) +Field has type in (YamlFileMetadataService.java:0) +Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) +Field has type in (DynamicKafkaSourceReader.java:0) +Field has type in (FlinkKafkaConsumerBase.java:0) +Field has type in (FlinkKafkaProducerBase.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (KafkaShuffleFetcher.java:0) +Field has type in (FlinkKafkaShuffleProducer.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:100) +Method calls method in (FlinkKafkaInternalProducer.java:109) +Method calls method in (FlinkKafkaInternalProducer.java:296) +Method calls method in (FlinkKafkaInternalProducer.java:297) +Method calls method in (FlinkKafkaInternalProducer.java:174) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:254) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:255) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:265) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:204) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:140) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:98) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:110) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:109) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:125) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:123) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:190) +Method calls method in (KafkaSerializerWrapper.java:71) +Method calls method in (KafkaSerializerWrapper.java:88) +Method is annotated with in (KafkaSink.java:0) +Method calls method in (KafkaSinkBuilder.java:194) +Method calls method in (KafkaSinkBuilder.java:202) +Method calls method in (KafkaSinkBuilder.java:198) +Method calls method in (KafkaSinkBuilder.java:111) +Method calls method in (KafkaSinkBuilder.java:97) +Method calls method in (KafkaSinkBuilder.java:123) +Method calls method in (KafkaSinkBuilder.java:133) +Method calls method in (KafkaSinkBuilder.java:152) +Method calls method in (KafkaSinkBuilder.java:151) +Method calls method in (KafkaSinkBuilder.java:175) +Method calls method in (KafkaSinkBuilder.java:176) +Method calls method in (KafkaWriter.java:244) +Method calls method in (KafkaWriter.java:245) +Method calls method in (KafkaWriter.java:246) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (KafkaWriter.java:0) +Method calls method in (KafkaWriter.java:311) +Method calls method in (Recyclable.java:36) +Method is annotated with in (DynamicKafkaSource.java:0) +Method calls method in (DynamicKafkaSourceBuilder.java:290) +Method calls method in (DynamicKafkaSourceBuilder.java:292) +Method calls method in (DynamicKafkaSourceBuilder.java:294) +Method calls method in (DynamicKafkaSourceBuilder.java:298) +Method calls method in (DynamicKafkaSourceBuilder.java:99) +Method calls method in (DynamicKafkaSourceBuilder.java:72) +Method calls method in (DynamicKafkaSourceBuilder.java:85) +Method calls method in (DynamicKafkaSourceOptions.java:69) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method calls method in (KafkaSourceBuilder.java:514) +Method calls method in (KafkaSourceBuilder.java:519) +Method calls method in (KafkaSourceBuilder.java:523) +Method calls method in (KafkaSourceBuilder.java:525) +Method calls method in (KafkaSourceBuilder.java:200) +Method calls method in (YamlFileMetadataService.java:279) +Method calls method in (YamlFileMetadataService.java:277) +Method checks instanceof in (YamlFileMetadataService.java:276) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:283) +Method calls method in (YamlFileMetadataService.java:283) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls constructor (java.lang.Class)> in (YamlFileMetadataService.java:260) +Method calls constructor (org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.BaseConstructor, org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer)> in (YamlFileMetadataService.java:263) +Method calls constructor ()> in (YamlFileMetadataService.java:258) +Method calls method in (YamlFileMetadataService.java:259) +Method calls method in (YamlFileMetadataService.java:261) +Method calls method in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:259) +Method has return type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:220) +Method is annotated with in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:165) +Method calls method in (DynamicKafkaSourceEnumStateSerializer.java:140) +Method calls method in (DynamicKafkaSourceEnumerator.java:506) +Method is annotated with in (KafkaSourceEnumStateSerializer.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method calls method in (StoppableKafkaEnumContextProxy.java:225) +Method calls method in (StoppableKafkaEnumContextProxy.java:247) +Method calls method in (ReaderHandledOffsetsInitializer.java:75) +Method calls method in (SpecifiedOffsetsInitializer.java:105) +Method calls method in (KafkaClusterMetricGroupManager.java:73) +Method calls method in (KafkaClusterMetricGroupManager.java:62) +Method checks instanceof in (KafkaClusterMetricGroupManager.java:42) +Method calls constructor (int)> in (DynamicKafkaSourceReader.java:474) +Method calls method in (DynamicKafkaSourceReader.java:473) +Method calls method in (DynamicKafkaSourceReader.java:488) +Method calls method in (DynamicKafkaSourceReader.java:417) +Method has return type in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:226) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:382) +Method calls method in (DynamicKafkaSourceReader.java:380) +Method calls method in (DynamicKafkaSourceReader.java:499) +Method calls method in (KafkaPartitionSplitReader.java:540) +Method calls method in (KafkaPartitionSplitReader.java:359) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method calls method in (KafkaValueOnlyDeserializerWrapper.java:65) +Method calls constructor (java.lang.Object)> in (FlinkKafkaConsumerBase.java:306) +Method calls method in (FlinkKafkaConsumerBase.java:304) +Method calls method in (FlinkKafkaConsumerBase.java:301) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks)> in (FlinkKafkaConsumerBase.java:398) +Method calls method in (FlinkKafkaConsumerBase.java:397) +Method calls method in (FlinkKafkaConsumerBase.java:390) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks)> in (FlinkKafkaConsumerBase.java:352) +Method calls method in (FlinkKafkaConsumerBase.java:351) +Method calls method in (FlinkKafkaConsumerBase.java:344) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:967) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (java.lang.Class, [Lorg.apache.flink.api.common.typeutils.TypeSerializer;)> in (FlinkKafkaConsumerBase.java:1225) +Method calls constructor (java.lang.Class, org.apache.flink.api.common.ExecutionConfig)> in (FlinkKafkaConsumerBase.java:1217) +Method gets field in (FlinkKafkaConsumerBase.java:1217) +Method has return type in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:752) +Method calls method in (FlinkKafkaConsumerBase.java:566) +Method calls method in (FlinkKafkaConsumerBase.java:535) +Method calls method in (FlinkKafkaConsumerBase.java:472) +Method calls method in (FlinkKafkaConsumerBase.java:477) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:404) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:223) +Method calls method in (FlinkKafkaProducerBase.java:260) +Method calls method in (AbstractFetcher.java:604) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:593) +Method calls method in (AbstractFetcher.java:593) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method calls method in (AbstractFetcher.java:430) +Method calls method in (AbstractFetcher.java:428) +Method calls method in (AbstractFetcher.java:426) +Method calls method in (AbstractFetcher.java:419) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method is annotated with in (FlinkKafkaInternalProducer.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:203) +Method calls method in (Handover.java:82) +Method calls method in (Handover.java:109) +Method calls method in (Handover.java:149) +Method is annotated with in (KafkaConsumerThread.java:0) +Method is annotated with in (KafkaConsumerThread.java:0) +Method calls method in (KafkaFetcher.java:223) +Method calls method in (KafkaFetcher.java:150) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:240) +Method calls method in (KafkaShuffleFetcher.java:244) +Method calls method in (KafkaShuffleFetcher.java:245) +Method calls method in (KafkaShuffleFetcher.java:254) +Method calls method in (KafkaShuffleFetcher.java:251) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:238) +Method gets field in (KafkaShuffleFetcher.java:244) +Method gets field in (KafkaShuffleFetcher.java:245) +Method gets field in (KafkaShuffleFetcher.java:253) +Method gets field in (KafkaShuffleFetcher.java:250) +Method gets field in (KafkaShuffleFetcher.java:254) +Method is annotated with in (KafkaShuffleFetcher.java:0) +Method calls method in (KafkaShuffleFetcher.java:279) +Method calls method in (FlinkFixedPartitioner.java:67) +Method calls method in (FlinkFixedPartitioner.java:69) +Method calls method in (FlinkFixedPartitioner.java:77) +Method calls constructor (org.apache.flink.api.dag.Transformation, java.lang.String, org.apache.flink.streaming.api.operators.StreamSink, int, boolean)> in (FlinkKafkaShuffle.java:380) +Method calls method in (FlinkKafkaShuffle.java:371) +Method calls method in (FlinkKafkaShuffle.java:383) +Method calls constructor ([I, org.apache.flink.api.common.typeinfo.TypeInformation)> in (FlinkKafkaShuffle.java:394) +Method calls method in (FlinkKafkaShuffle.java:392) +Method calls method in (FlinkKafkaShuffle.java:396) +Method calls method in (FlinkKafkaShuffle.java:151) +Method calls method in (FlinkKafkaShuffle.java:352) +Method calls method in (FlinkKafkaShuffle.java:340) +Method calls method in (FlinkKafkaShuffle.java:344) +Method calls method in (FlinkKafkaShuffle.java:256) +Method calls method in (FlinkKafkaShuffle.java:260) +Method calls method in (FlinkKafkaShuffleConsumer.java:81) +Method calls method in (FlinkKafkaShuffleConsumer.java:87) +Method calls method in (FlinkKafkaShuffleConsumer.java:80) +Method calls method in (FlinkKafkaShuffleConsumer.java:83) +Method calls method in (FlinkKafkaShuffleConsumer.java:82) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:186) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:204) +Method calls method in (FlinkKafkaShuffleProducer.java:190) +Method calls method in (FlinkKafkaShuffleProducer.java:193) +Method calls method in (FlinkKafkaShuffleProducer.java:195) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:212) +Method calls method in (FlinkKafkaShuffleProducer.java:225) +Method calls method in (FlinkKafkaShuffleProducer.java:224) +Method calls method in (FlinkKafkaShuffleProducer.java:216) +Method calls method in (FlinkKafkaShuffleProducer.java:217) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:160) +Method calls method in (FlinkKafkaShuffleProducer.java:99) +Method calls method in (StreamKafkaShuffleSink.java:40) +Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (KafkaConnectorOptionsUtil.java:543) +Method calls method in (KafkaConnectorOptionsUtil.java:518) +Method calls method in (KafkaConnectorOptionsUtil.java:587) +Method calls method in (KafkaConnectorOptionsUtil.java:585) +Method calls method in (KafkaConnectorOptionsUtil.java:498) +Method calls method in (KafkaDynamicSink.java:386) +Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) +Method calls method in (KafkaDynamicSource.java:566) +Method calls method in (ReducingUpsertWriter.java:177) diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 7c549fdac..ae5d99862 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -73,6 +73,11 @@ under the License. ${kafka.version} + + com.google.guava + guava + + diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java new file mode 100644 index 000000000..1ff18930e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java @@ -0,0 +1,92 @@ +/* + * 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.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +/** + * {@link ClusterMetadata} provides readers information about a cluster on what topics to read and + * how to connect to a cluster. + */ +@Experimental +public class ClusterMetadata implements Serializable { + private final Set topics; + private final Properties properties; + + /** + * Constructs the {@link ClusterMetadata} with the required properties. + * + * @param topics the topics belonging to a cluster. + * @param properties the properties to access a cluster. + */ + public ClusterMetadata(Set topics, Properties properties) { + this.topics = topics; + this.properties = properties; + } + + /** + * Get the topics. + * + * @return the topics. + */ + public Set getTopics() { + return topics; + } + + /** + * Get the properties. + * + * @return the properties. + */ + public Properties getProperties() { + return properties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("topics", topics) + .add("properties", properties) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterMetadata that = (ClusterMetadata) o; + return Objects.equals(topics, that.topics) && Objects.equals(properties, that.properties); + } + + @Override + public int hashCode() { + return Objects.hash(topics, properties); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java new file mode 100644 index 000000000..179277dad --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java @@ -0,0 +1,53 @@ +/* + * 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.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +/** Metadata service that returns Kafka details. */ +@Experimental +public interface KafkaMetadataService extends AutoCloseable, Serializable { + /** + * Get current metadata for all streams. + * + * @return set of all streams + */ + Set getAllStreams(); + + /** + * Get current metadata for queried streams. + * + * @param streamIds stream full names + * @return map of stream name to metadata + */ + Map describeStreams(Collection streamIds); + + /** + * Check if the cluster is active. + * + * @param kafkaClusterId Kafka cluster id + * @return boolean whether the cluster is active + */ + boolean isClusterActive(String kafkaClusterId); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java new file mode 100644 index 000000000..bea9872c7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java @@ -0,0 +1,94 @@ +/* + * 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.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Kafka stream represents multiple topics over multiple Kafka clusters and this class encapsulates + * all the necessary information to initiate Kafka consumers to read a stream. + */ +@Experimental +public class KafkaStream implements Serializable { + private final String streamId; + private final Map clusterMetadataMap; + + /** + * Construct a {@link KafkaStream} by passing Kafka information in order to connect to the + * stream. + * + * @param streamId the stream id. + * @param clusterMetadataMap the map of clusters to {@link ClusterMetadata} to connect to the + * stream. + */ + public KafkaStream(String streamId, Map clusterMetadataMap) { + this.streamId = streamId; + this.clusterMetadataMap = clusterMetadataMap; + } + + /** + * Get the stream id. + * + * @return the stream id. + */ + public String getStreamId() { + return streamId; + } + + /** + * Get the metadata to connect to the various cluster(s). + * + * @return the cluster metadata map. + */ + public Map getClusterMetadataMap() { + return clusterMetadataMap; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataMap", clusterMetadataMap) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaStream that = (KafkaStream) o; + return Objects.equals(streamId, that.streamId) + && Objects.equals(clusterMetadataMap, that.clusterMetadataMap); + } + + @Override + public int hashCode() { + return Objects.hash(streamId, clusterMetadataMap); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java new file mode 100644 index 000000000..6cef3ab31 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java @@ -0,0 +1,118 @@ +/* + * 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.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +/** + * A {@link KafkaMetadataService} that delegates metadata fetching to a single {@link AdminClient}, + * which is scoped to a single cluster. The stream ids are equivalent to topics. + */ +@Experimental +public class SingleClusterTopicMetadataService implements KafkaMetadataService { + + private final String kafkaClusterId; + private final Properties properties; + private transient AdminClient adminClient; + + /** + * Create a {@link SingleClusterTopicMetadataService}. + * + * @param kafkaClusterId the id of the Kafka cluster. + * @param properties the properties of the Kafka cluster. + */ + public SingleClusterTopicMetadataService(String kafkaClusterId, Properties properties) { + this.kafkaClusterId = kafkaClusterId; + this.properties = properties; + } + + /** {@inheritDoc} */ + @Override + public Set getAllStreams() { + try { + return getAdminClient().listTopics().names().get().stream() + .map(this::createKafkaStream) + .collect(Collectors.toSet()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + try { + return getAdminClient().describeTopics(new ArrayList<>(streamIds)).all().get().keySet() + .stream() + .collect(Collectors.toMap(topic -> topic, this::createKafkaStream)); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + private KafkaStream createKafkaStream(String topic) { + ClusterMetadata clusterMetadata = + new ClusterMetadata(Collections.singleton(topic), properties); + + return new KafkaStream(topic, Collections.singletonMap(kafkaClusterId, clusterMetadata)); + } + + private AdminClient getAdminClient() { + if (adminClient == null) { + Properties adminClientProps = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, adminClientProps); + String clientIdPrefix = + adminClientProps.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + adminClientProps.setProperty( + CommonClientConfigs.CLIENT_ID_CONFIG, + clientIdPrefix + "-single-cluster-topic-metadata-service"); + adminClient = AdminClient.create(adminClientProps); + } + + return adminClient; + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return this.kafkaClusterId.equals(kafkaClusterId); + } + + /** {@inheritDoc} */ + @Override + public void close() { + if (adminClient != null) { + adminClient.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java new file mode 100644 index 000000000..9a93d7b88 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java @@ -0,0 +1,222 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumerator; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.Properties; + +/** + * Factory class for the DynamicKafkaSource components. FLIP-246: DynamicKafkaSource + * + *

This source's key difference from {@link KafkaSource} is that it enables users to read + * dynamically, which does not require job restart, from streams (topics that belong to one or more + * clusters). If using {@link KafkaSource}, users need to restart the job by deleting the job and + * reconfiguring the topics and clusters. + * + *

This example shows how to configure a {@link DynamicKafkaSource} that emits Integer records: + * + *

{@code
+ * DynamicKafkaSource dynamicKafkaSource =
+ *                     DynamicKafkaSource.builder()
+ *                             .setStreamIds(Collections.singleton("MY_STREAM_ID"))
+ *                             // custom metadata service that resolves `MY_STREAM_ID` to the associated clusters and topics
+ *                             .setKafkaMetadataService(kafkaMetadataService)
+ *                             .setDeserializer(
+ *                                     KafkaRecordDeserializationSchema.valueOnly(
+ *                                             IntegerDeserializer.class))
+ *                             .setStartingOffsets(OffsetsInitializer.earliest())
+ *                             // common properties for all Kafka clusters
+ *                             .setProperties(properties)
+ *                             .build();
+ * }
+ * + *

See more configuration options in {@link DynamicKafkaSourceBuilder} and {@link + * DynamicKafkaSourceOptions}. + * + * @param Record type + */ +@Experimental +public class DynamicKafkaSource + implements Source, + ResultTypeQueryable { + + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final KafkaMetadataService kafkaMetadataService; + private final KafkaRecordDeserializationSchema deserializationSchema; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetsInitializer; + private final Properties properties; + private final Boundedness boundedness; + + DynamicKafkaSource( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + KafkaRecordDeserializationSchema deserializationSchema, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetsInitializer, + Properties properties, + Boundedness boundedness) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaMetadataService = kafkaMetadataService; + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + this.boundedness = boundedness; + } + + /** + * Get a builder for this source. + * + * @return a {@link DynamicKafkaSourceBuilder}. + */ + public static DynamicKafkaSourceBuilder builder() { + return new DynamicKafkaSourceBuilder<>(); + } + + /** + * Get the {@link Boundedness}. + * + * @return the {@link Boundedness}. + */ + @Override + public Boundedness getBoundedness() { + return boundedness; + } + + /** + * Create the {@link DynamicKafkaSourceReader}. + * + * @param readerContext The {@link SourceReaderContext context} for the source reader. + * @return the {@link DynamicKafkaSourceReader}. + */ + @Internal + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new DynamicKafkaSourceReader<>(readerContext, deserializationSchema, properties); + } + + /** + * Create the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + new DynamicKafkaSourceEnumState()); + } + + /** + * Restore the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the restored split + * enumerator. + * @param checkpoint The checkpoint to restore the SplitEnumerator from. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + DynamicKafkaSourceEnumState checkpoint) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + checkpoint); + } + + /** + * Get the {@link DynamicKafkaSourceSplitSerializer}. + * + * @return the {@link DynamicKafkaSourceSplitSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new DynamicKafkaSourceSplitSerializer(); + } + + /** + * Get the {@link DynamicKafkaSourceEnumStateSerializer}. + * + * @return the {@link DynamicKafkaSourceEnumStateSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer + getEnumeratorCheckpointSerializer() { + return new DynamicKafkaSourceEnumStateSerializer(); + } + + /** + * Get the {@link TypeInformation} of the source. + * + * @return the {@link TypeInformation}. + */ + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + @VisibleForTesting + public KafkaStreamSubscriber getKafkaStreamSubscriber() { + return kafkaStreamSubscriber; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java new file mode 100644 index 000000000..eab37c4ee --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java @@ -0,0 +1,328 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.StreamPatternSubscriber; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; + +/** A builder class to make it easier for users to construct a {@link DynamicKafkaSource}. */ +@Experimental +public class DynamicKafkaSourceBuilder { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceBuilder.class); + private KafkaStreamSubscriber kafkaStreamSubscriber; + private KafkaMetadataService kafkaMetadataService; + private KafkaRecordDeserializationSchema deserializationSchema; + private OffsetsInitializer startingOffsetsInitializer; + private OffsetsInitializer stoppingOffsetsInitializer; + private Boundedness boundedness; + private final Properties props; + + DynamicKafkaSourceBuilder() { + this.kafkaStreamSubscriber = null; + this.kafkaMetadataService = null; + this.deserializationSchema = null; + this.startingOffsetsInitializer = OffsetsInitializer.earliest(); + this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.props = new Properties(); + } + + /** + * Set the stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamIds the stream ids. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamIds(Set streamIds) { + Preconditions.checkNotNull(streamIds); + ensureSubscriberIsNull("streamIds"); + this.kafkaStreamSubscriber = new KafkaStreamSetSubscriber(streamIds); + return this; + } + + /** + * Set the stream pattern to determine stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamPattern the stream pattern. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamPattern(Pattern streamPattern) { + Preconditions.checkNotNull(streamPattern); + ensureSubscriberIsNull("stream pattern"); + this.kafkaStreamSubscriber = new StreamPatternSubscriber(streamPattern); + return this; + } + + /** + * Set a custom Kafka stream subscriber. + * + * @param kafkaStreamSubscriber the {@link KafkaStreamSubscriber}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaStreamSubscriber( + KafkaStreamSubscriber kafkaStreamSubscriber) { + Preconditions.checkNotNull(kafkaStreamSubscriber); + ensureSubscriberIsNull("custom"); + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + return this; + } + + /** + * Set the source in bounded mode and specify what offsets to end at. This is used for all + * clusters. + * + * @param stoppingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setBounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.BOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * Set the {@link KafkaMetadataService}. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaMetadataService( + KafkaMetadataService kafkaMetadataService) { + this.kafkaMetadataService = kafkaMetadataService; + return this; + } + + /** + * Set the {@link KafkaRecordDeserializationSchema}. + * + * @param recordDeserializer the {@link KafkaRecordDeserializationSchema}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setDeserializer( + KafkaRecordDeserializationSchema recordDeserializer) { + this.deserializationSchema = recordDeserializer; + return this; + } + + /** + * Set the starting offsets of the stream. This will be applied to all clusters. + * + * @param startingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStartingOffsets( + OffsetsInitializer startingOffsetsInitializer) { + this.startingOffsetsInitializer = startingOffsetsInitializer; + return this; + } + + /** + * Set the properties of the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param properties the properties. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperties(Properties properties) { + this.props.putAll(properties); + return this; + } + + /** + * Set a property for the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param key the property key. + * @param value the properties value. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperty(String key, String value) { + this.props.setProperty(key, value); + return this; + } + + /** + * Set the property for {@link CommonClientConfigs#GROUP_ID_CONFIG}. This will be applied to all + * clusters. + * + * @param groupId the group id. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setGroupId(String groupId) { + return setProperty(CommonClientConfigs.GROUP_ID_CONFIG, groupId); + } + + /** + * Set the client id prefix. This applies {@link KafkaSourceOptions#CLIENT_ID_PREFIX} to all + * clusters. + * + * @param prefix the client id prefix. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setClientIdPrefix(String prefix) { + return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); + } + + /** + * Construct the source with the configuration that was set. + * + * @return the {@link DynamicKafkaSource}. + */ + public DynamicKafkaSource build() { + logger.info("Building the DynamicKafkaSource"); + sanityCheck(); + setRequiredConsumerProperties(); + return new DynamicKafkaSource<>( + kafkaStreamSubscriber, + kafkaMetadataService, + deserializationSchema, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + boundedness); + } + + // Below are utility methods, code and structure are mostly copied over from KafkaSourceBuilder + + private void setRequiredConsumerProperties() { + maybeOverride( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + maybeOverride( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + logger.warn( + "Offset commit on checkpoint is disabled because {} is not specified", + ConsumerConfig.GROUP_ID_CONFIG); + maybeOverride(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false", false); + } + maybeOverride(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false", false); + maybeOverride( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase(), + true); + + // If the source is bounded, do not run periodic partition discovery. + maybeOverride( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + + // If the source is bounded, do not run periodic metadata discovery + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "0", + boundedness == Boundedness.BOUNDED); + + // If the client id prefix is not set, reuse the consumer group id as the client id prefix, + // or generate a random string if consumer group id is not specified. + maybeOverride( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) + ? props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) + : "DynamicKafkaSource-" + RandomStringUtils.randomAlphabetic(8), + false); + } + + private boolean maybeOverride(String key, String value, boolean override) { + boolean overridden = false; + String userValue = props.getProperty(key); + if (userValue != null) { + if (override) { + logger.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + props.setProperty(key, value); + overridden = true; + } + } else { + props.setProperty(key, value); + } + return overridden; + } + + private void sanityCheck() { + Preconditions.checkNotNull( + kafkaStreamSubscriber, "Kafka stream subscriber is required but not provided"); + Preconditions.checkNotNull( + kafkaMetadataService, "Kafka Metadata Service is required but not provided"); + Preconditions.checkNotNull( + deserializationSchema, "Deserialization schema is required but not provided."); + + // Check consumer group ID + Preconditions.checkState( + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(), + String.format( + "Property %s is required when offset commit is enabled", + ConsumerConfig.GROUP_ID_CONFIG)); + } + + private boolean offsetCommitEnabledManually() { + boolean autoCommit = + props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + && Boolean.parseBoolean( + props.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + boolean commitOnCheckpoint = + props.containsKey(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key()) + && Boolean.parseBoolean( + props.getProperty( + KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key())); + return autoCommit || commitOnCheckpoint; + } + + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (kafkaStreamSubscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, + kafkaStreamSubscriber.getClass().getSimpleName())); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java new file mode 100644 index 000000000..074798ce8 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java @@ -0,0 +1,60 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.Properties; +import java.util.function.Function; + +/** + * The connector options for {@link DynamicKafkaSource} that can be passed through the source + * properties e.g. {@link DynamicKafkaSourceBuilder#setProperties(Properties)}. + */ +@Internal +public class DynamicKafkaSourceOptions { + + private DynamicKafkaSourceOptions() {} + + public static final ConfigOption STREAM_METADATA_DISCOVERY_INTERVAL_MS = + ConfigOptions.key("stream-metadata-discovery-interval-ms") + .longType() + .defaultValue(-1L) + .withDescription( + "The interval in milliseconds for the sink to discover " + + "the changes in stream metadata. A non-positive value disables the stream metadata discovery."); + + public static final ConfigOption STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD = + ConfigOptions.key("stream-metadata-discovery-failure-threshold") + .intType() + .defaultValue(1) + .withDescription( + "The number of consecutive failures before letting the exception from Kafka metadata service discovery " + + "trigger jobmanager failure and global failover. The default is one to at least catch startup " + + "failures. This is only implemented for the source"); + + @Internal + public static T getOption( + Properties props, ConfigOption configOption, Function parser) { + String value = props.getProperty(configOption.key()); + return (T) (value == null ? configOption.defaultValue() : parser.apply(value)); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java new file mode 100644 index 000000000..4d7ff7b2d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java @@ -0,0 +1,26 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** Event to signal to enumerator that a reader needs to know the current metadata. */ +@Internal +public class GetMetadataUpdateEvent implements SourceEvent {} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java new file mode 100644 index 000000000..09b32d031 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java @@ -0,0 +1,77 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; +import java.util.Set; + +/** + * Signals {@link DynamicKafkaSourceReader} to stop their underlying readers. The restart process is + * as follows: + * + *

1. Detect metadata change in enumerator 2. Stop sub enumerators and don't snapshot state 3. + * Send this event to all readers 4. Stop sub readers and snapshot state (offsets) 5. Start new sub + * enumerators with clean state and do total split reassignment to readers 6. Readers obtain splits, + * starting sub readers dynamically, and do reconciliation of starting offsets with the cached + * offsets + * + *

We don't snapshot enumerator state because we want to reassign previously assigned splits. + * After restart, readers need to reinitialize the sub readers by using the received splits. + */ +@Internal +public class MetadataUpdateEvent implements SourceEvent { + private final Set kafkaStreams; + + public MetadataUpdateEvent(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("kafkaStreams", kafkaStreams).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MetadataUpdateEvent that = (MetadataUpdateEvent) o; + return Objects.equals(kafkaStreams, that.kafkaStreams); + } + + @Override + public int hashCode() { + return Objects.hash(kafkaStreams); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java new file mode 100644 index 000000000..0dd0a7744 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java @@ -0,0 +1,58 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * The enumerator state keeps track of the state of the sub enumerators assigned splits and + * metadata. + */ +@Internal +public class DynamicKafkaSourceEnumState { + private final Set kafkaStreams; + private final Map clusterEnumeratorStates; + + public DynamicKafkaSourceEnumState() { + this.kafkaStreams = new HashSet<>(); + this.clusterEnumeratorStates = new HashMap<>(); + } + + public DynamicKafkaSourceEnumState( + Set kafkaStreams, + Map clusterEnumeratorStates) { + this.kafkaStreams = kafkaStreams; + this.clusterEnumeratorStates = clusterEnumeratorStates; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + public Map getClusterEnumeratorStates() { + return clusterEnumeratorStates; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java new file mode 100644 index 000000000..b34e536c4 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java @@ -0,0 +1,187 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import org.apache.kafka.clients.CommonClientConfigs; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** (De)serializer for {@link DynamicKafkaSourceEnumState}. */ +@Internal +public class DynamicKafkaSourceEnumStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaSourceEnumStateSerializer kafkaSourceEnumStateSerializer; + + public DynamicKafkaSourceEnumStateSerializer() { + this.kafkaSourceEnumStateSerializer = new KafkaSourceEnumStateSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceEnumState state) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + Set kafkaStreams = state.getKafkaStreams(); + serialize(kafkaStreams, out); + + Map clusterEnumeratorStates = + state.getClusterEnumeratorStates(); + out.writeInt(kafkaSourceEnumStateSerializer.getVersion()); + + // write sub enumerator states + out.writeInt(clusterEnumeratorStates.size()); + for (Map.Entry clusterEnumeratorState : + clusterEnumeratorStates.entrySet()) { + String kafkaClusterId = clusterEnumeratorState.getKey(); + out.writeUTF(kafkaClusterId); + byte[] bytes = + kafkaSourceEnumStateSerializer.serialize(clusterEnumeratorState.getValue()); + // we need to know the exact size of the byte array since + // KafkaSourceEnumStateSerializer + // will throw exception if there are leftover unread bytes in deserialization. + out.writeInt(bytes.length); + out.write(bytes); + } + + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceEnumState deserialize(int version, byte[] serialized) + throws IOException { + if (version == VERSION_1) { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + Set kafkaStreams = deserialize(in); + + Map clusterEnumeratorStates = new HashMap<>(); + int kafkaSourceEnumStateSerializerVersion = in.readInt(); + + int clusterEnumeratorStateMapSize = in.readInt(); + for (int i = 0; i < clusterEnumeratorStateMapSize; i++) { + String kafkaClusterId = in.readUTF(); + int byteArraySize = in.readInt(); + KafkaSourceEnumState kafkaSourceEnumState = + kafkaSourceEnumStateSerializer.deserialize( + kafkaSourceEnumStateSerializerVersion, + readNBytes(in, byteArraySize)); + clusterEnumeratorStates.put(kafkaClusterId, kafkaSourceEnumState); + } + + return new DynamicKafkaSourceEnumState(kafkaStreams, clusterEnumeratorStates); + } + } + + throw new IOException( + String.format( + "The bytes are serialized with version %d, " + + "while this deserializer only supports version up to %d", + version, getVersion())); + } + + private void serialize(Set kafkaStreams, DataOutputStream out) throws IOException { + out.writeInt(kafkaStreams.size()); + for (KafkaStream kafkaStream : kafkaStreams) { + out.writeUTF(kafkaStream.getStreamId()); + Map clusterMetadataMap = kafkaStream.getClusterMetadataMap(); + out.writeInt(clusterMetadataMap.size()); + for (Map.Entry entry : clusterMetadataMap.entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + out.writeUTF(kafkaClusterId); + out.writeInt(clusterMetadata.getTopics().size()); + for (String topic : clusterMetadata.getTopics()) { + out.writeUTF(topic); + } + + // only write bootstrap server for now, can extend later to serialize the complete + // properties + out.writeUTF( + Preconditions.checkNotNull( + clusterMetadata + .getProperties() + .getProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + "Bootstrap servers must be specified in properties"))); + } + } + } + + private Set deserialize(DataInputStream in) throws IOException { + + Set kafkaStreams = new HashSet<>(); + int numStreams = in.readInt(); + for (int i = 0; i < numStreams; i++) { + String streamId = in.readUTF(); + Map clusterMetadataMap = new HashMap<>(); + int clusterMetadataMapSize = in.readInt(); + for (int j = 0; j < clusterMetadataMapSize; j++) { + String kafkaClusterId = in.readUTF(); + int topicsSize = in.readInt(); + Set topics = new HashSet<>(); + for (int k = 0; k < topicsSize; k++) { + topics.add(in.readUTF()); + } + + String bootstrapServers = in.readUTF(); + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + clusterMetadataMap.put(kafkaClusterId, new ClusterMetadata(topics, properties)); + } + + kafkaStreams.add(new KafkaStream(streamId, clusterMetadataMap)); + } + + return kafkaStreams; + } + + private static byte[] readNBytes(DataInputStream in, int size) throws IOException { + byte[] bytes = new byte[size]; + in.readFully(bytes); + return bytes; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java new file mode 100644 index 000000000..cce8ab28b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java @@ -0,0 +1,539 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.Preconditions; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.MapDifference; +import com.google.common.collect.Maps; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This enumerator manages multiple {@link KafkaSourceEnumerator}'s, which does not have any + * synchronization since it assumes single threaded execution. + */ +@Internal +public class DynamicKafkaSourceEnumerator + implements SplitEnumerator { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceEnumerator.class); + + // Each cluster will have its own sub enumerator + private final Map> + clusterEnumeratorMap; + + // The mapping that the split enumerator context needs to be able to forward certain requests. + private final Map clusterEnumContextMap; + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final SplitEnumeratorContext enumContext; + private final KafkaMetadataService kafkaMetadataService; + private final Properties properties; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetInitializer; + private final Boundedness boundedness; + private final StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory; + + // options + private final long kafkaMetadataServiceDiscoveryIntervalMs; + private final int kafkaMetadataServiceDiscoveryFailureThreshold; + + // state + private int kafkaMetadataServiceDiscoveryFailureCount; + private Map> latestClusterTopicsMap; + private Set latestKafkaStreams; + + public DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState) { + this( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetInitializer, + properties, + boundedness, + dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + .getDefaultFactory()); + } + + @VisibleForTesting + DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.boundedness = boundedness; + + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.properties = properties; + this.enumContext = enumContext; + + // options + this.kafkaMetadataServiceDiscoveryIntervalMs = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS, + Long::parseLong); + this.kafkaMetadataServiceDiscoveryFailureThreshold = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD, + Integer::parseInt); + this.kafkaMetadataServiceDiscoveryFailureCount = 0; + + this.kafkaMetadataService = kafkaMetadataService; + this.stoppableKafkaEnumContextProxyFactory = stoppableKafkaEnumContextProxyFactory; + + // handle checkpoint state and rebuild contexts + this.clusterEnumeratorMap = new HashMap<>(); + this.clusterEnumContextMap = new HashMap<>(); + this.latestKafkaStreams = dynamicKafkaSourceEnumState.getKafkaStreams(); + + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : latestKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterProperties.put(entry.getKey(), entry.getValue().getProperties()); + } + } + + this.latestClusterTopicsMap = new HashMap<>(); + for (Entry clusterEnumState : + dynamicKafkaSourceEnumState.getClusterEnumeratorStates().entrySet()) { + this.latestClusterTopicsMap.put( + clusterEnumState.getKey(), + clusterEnumState.getValue().assignedPartitions().stream() + .map(TopicPartition::topic) + .collect(Collectors.toSet())); + + createEnumeratorWithAssignedTopicPartitions( + clusterEnumState.getKey(), + this.latestClusterTopicsMap.get(clusterEnumState.getKey()), + clusterEnumState.getValue(), + clusterProperties.get(clusterEnumState.getKey())); + } + } + + /** + * Discover Kafka clusters and initialize sub enumerators. Bypass kafka metadata service + * discovery if there exists prior state. Exceptions with initializing Kafka source are treated + * the same as Kafka state and metadata inconsistency. + */ + @Override + public void start() { + // if there is checkpoint state, start all enumerators first. + if (!clusterEnumeratorMap.isEmpty()) { + startAllEnumerators(); + } + + if (kafkaMetadataServiceDiscoveryIntervalMs <= 0) { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch); + } else { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch, + 0, + kafkaMetadataServiceDiscoveryIntervalMs); + } + } + + private void handleNoMoreSplits() { + if (Boundedness.BOUNDED.equals(boundedness)) { + enumContext.runInCoordinatorThread( + () -> { + boolean allEnumeratorsHaveSignalledNoMoreSplits = true; + for (StoppableKafkaEnumContextProxy context : + clusterEnumContextMap.values()) { + allEnumeratorsHaveSignalledNoMoreSplits = + allEnumeratorsHaveSignalledNoMoreSplits + && context.isNoMoreSplits(); + } + + if (allEnumeratorsHaveSignalledNoMoreSplits) { + logger.info( + "Signal no more splits to all readers: {}", + enumContext.registeredReaders().keySet()); + enumContext + .registeredReaders() + .keySet() + .forEach(enumContext::signalNoMoreSplits); + } + }); + } + } + + // --------------- private methods for metadata discovery --------------- + + private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams, Throwable t) { + Set handledFetchKafkaStreams = + handleFetchSubscribedStreamsError(fetchedKafkaStreams, t); + + Map> newClustersTopicsMap = new HashMap<>(); + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : handledFetchKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + + newClustersTopicsMap + .computeIfAbsent(kafkaClusterId, (unused) -> new HashSet<>()) + .addAll(clusterMetadata.getTopics()); + clusterProperties.put(kafkaClusterId, clusterMetadata.getProperties()); + } + } + + // don't do anything if no change + if (latestClusterTopicsMap.equals(newClustersTopicsMap)) { + return; + } + + if (logger.isInfoEnabled()) { + MapDifference> metadataDifference = + Maps.difference(latestClusterTopicsMap, newClustersTopicsMap); + logger.info( + "Common cluster topics after metadata refresh: {}", + metadataDifference.entriesInCommon()); + logger.info( + "Removed cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnLeft()); + logger.info( + "Additional cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnRight()); + } + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState; + try { + dynamicKafkaSourceEnumState = snapshotState(-1); + } catch (Exception e) { + throw new RuntimeException("unable to snapshot state in metadata change", e); + } + + logger.info("Closing enumerators due to metadata change"); + + closeAllEnumeratorsAndContexts(); + latestClusterTopicsMap = newClustersTopicsMap; + latestKafkaStreams = handledFetchKafkaStreams; + sendMetadataUpdateEventToAvailableReaders(); + + // create enumerators + for (Entry> activeClusterTopics : latestClusterTopicsMap.entrySet()) { + final Set activeTopicPartitions = new HashSet<>(); + + if (dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .get(activeClusterTopics.getKey()) + != null) { + Set oldTopicPartitions = + dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .get(activeClusterTopics.getKey()) + .assignedPartitions(); + // filter out removed topics + for (TopicPartition oldTopicPartition : oldTopicPartitions) { + if (activeClusterTopics.getValue().contains(oldTopicPartition.topic())) { + activeTopicPartitions.add(oldTopicPartition); + } + } + } + + // restarts enumerator from state using only the active topic partitions, to avoid + // sending duplicate splits from enumerator + createEnumeratorWithAssignedTopicPartitions( + activeClusterTopics.getKey(), + activeClusterTopics.getValue(), + dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .getOrDefault( + activeClusterTopics.getKey(), + new KafkaSourceEnumState( + Collections.emptySet(), Collections.emptySet(), false)), + clusterProperties.get(activeClusterTopics.getKey())); + } + + startAllEnumerators(); + } + + private Set handleFetchSubscribedStreamsError( + Set fetchedKafkaStreams, @Nullable Throwable t) { + if (t != null) { + if (!latestKafkaStreams.isEmpty() + && ++kafkaMetadataServiceDiscoveryFailureCount + <= kafkaMetadataServiceDiscoveryFailureThreshold) { + logger.warn("Swallowing metadata service error", t); + // reuse state + return latestKafkaStreams; + } else { + throw new RuntimeException( + "Fetching subscribed Kafka streams failed and no metadata to fallback", t); + } + } else { + // reset count in absence of failure + kafkaMetadataServiceDiscoveryFailureCount = 0; + return fetchedKafkaStreams; + } + } + + /** NOTE: Must run on coordinator thread. */ + private void sendMetadataUpdateEventToAvailableReaders() { + for (int readerId : enumContext.registeredReaders().keySet()) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", readerId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(readerId, metadataUpdateEvent); + } + } + + /** + * Initialize KafkaEnumerators, maybe with the topic partitions that are already assigned to by + * readers, to avoid duplicate re-assignment of splits. This is especially important in the + * restart mechanism when duplicate split assignment can cause undesired starting offsets (e.g. + * not assigning to the offsets prior to reader restart). Split offset resolution is mostly + * managed by the readers. + * + *

NOTE: Must run on coordinator thread + */ + private KafkaSourceEnumerator createEnumeratorWithAssignedTopicPartitions( + String kafkaClusterId, + Set topics, + KafkaSourceEnumState kafkaSourceEnumState, + Properties fetchedProperties) { + StoppableKafkaEnumContextProxy context = + stoppableKafkaEnumContextProxyFactory.create( + enumContext, kafkaClusterId, kafkaMetadataService); + + Properties consumerProps = new Properties(); + KafkaPropertiesUtil.copyProperties(fetchedProperties, consumerProps); + KafkaPropertiesUtil.copyProperties(properties, consumerProps); + KafkaPropertiesUtil.setClientIdPrefix(consumerProps, kafkaClusterId); + + KafkaSourceEnumerator enumerator = + new KafkaSourceEnumerator( + KafkaSubscriber.getTopicListSubscriber(new ArrayList<>(topics)), + startingOffsetsInitializer, + stoppingOffsetInitializer, + consumerProps, + context, + boundedness, + kafkaSourceEnumState); + + clusterEnumContextMap.put(kafkaClusterId, context); + clusterEnumeratorMap.put(kafkaClusterId, enumerator); + + return enumerator; + } + + private void startAllEnumerators() { + for (String kafkaClusterId : latestClusterTopicsMap.keySet()) { + try { + // starts enumerators and handles split discovery and assignment + clusterEnumeratorMap.get(kafkaClusterId).start(); + } catch (KafkaException e) { + if (kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new RuntimeException( + String.format("Failed to create enumerator for %s", kafkaClusterId), e); + } else { + logger.info( + "Found inactive cluster {} while initializing, removing enumerator", + kafkaClusterId, + e); + try { + clusterEnumContextMap.remove(kafkaClusterId).close(); + clusterEnumeratorMap.remove(kafkaClusterId).close(); + } catch (Exception ex) { + // closing enumerator throws an exception, let error propagate and restart + // the job + throw new RuntimeException( + "Failed to close enum context for " + kafkaClusterId, ex); + } + } + } + } + } + + private void closeAllEnumeratorsAndContexts() { + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> { + try { + clusterEnumContextMap.get(cluster).close(); + subEnumerator.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + clusterEnumContextMap.clear(); + clusterEnumeratorMap.clear(); + } + + /** + * Multi cluster Kafka source readers will not request splits. Splits will be pushed to them, + * similarly for the sub enumerators. + */ + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + throw new UnsupportedOperationException("Kafka enumerators only assign splits to readers."); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + logger.debug("Adding splits back for {}", subtaskId); + // separate splits by cluster + ArrayListMultimap kafkaPartitionSplits = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + kafkaPartitionSplits.put(split.getKafkaClusterId(), split.getKafkaPartitionSplit()); + } + + // add splits back and assign pending splits for all enumerators + for (String kafkaClusterId : kafkaPartitionSplits.keySet()) { + if (clusterEnumeratorMap.containsKey(kafkaClusterId)) { + clusterEnumeratorMap + .get(kafkaClusterId) + .addSplitsBack(kafkaPartitionSplits.get(kafkaClusterId), subtaskId); + } else { + logger.warn( + "Split refers to inactive cluster {} with current clusters being {}", + kafkaClusterId, + clusterEnumeratorMap.keySet()); + } + } + + handleNoMoreSplits(); + } + + /** NOTE: this happens at startup and failover. */ + @Override + public void addReader(int subtaskId) { + logger.debug("Adding reader {}", subtaskId); + // assign pending splits from the sub enumerator + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> subEnumerator.addReader(subtaskId)); + handleNoMoreSplits(); + } + + /** + * Besides for checkpointing, this method is used in the restart sequence to retain the relevant + * assigned splits so that there is no reader duplicate split assignment. See {@link + * #createEnumeratorWithAssignedTopicPartitions(String, Set, KafkaSourceEnumState, Properties)}} + */ + @Override + public DynamicKafkaSourceEnumState snapshotState(long checkpointId) throws Exception { + Map subEnumeratorStateByCluster = new HashMap<>(); + + // populate map for all assigned splits + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + subEnumeratorStateByCluster.put( + clusterEnumerator.getKey(), + clusterEnumerator.getValue().snapshotState(checkpointId)); + } + + return new DynamicKafkaSourceEnumState(latestKafkaStreams, subEnumeratorStateByCluster); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof GetMetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + if (enumContext.registeredReaders().containsKey(subtaskId)) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", subtaskId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(subtaskId, metadataUpdateEvent); + } else { + logger.warn("Got get metadata update but subtask was unavailable"); + } + } + + @Override + public void close() throws IOException { + try { + // close contexts first since they may have running tasks + for (StoppableKafkaEnumContextProxy subEnumContext : clusterEnumContextMap.values()) { + subEnumContext.close(); + } + + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + clusterEnumerator.getValue().close(); + } + + kafkaMetadataService.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java new file mode 100644 index 000000000..5506c4446 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java @@ -0,0 +1,297 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +/** + * A proxy enumerator context that supports life cycle management of underlying threads related to a + * sub {@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator}. This is + * motivated by the need to cancel the periodic partition discovery in scheduled tasks when sub + * Kafka Enumerators are restarted. The worker thread pool in {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext} should not contain tasks of + * inactive KafkaSourceEnumerators, after source restart. + * + *

Due to the inability to cancel scheduled tasks from {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext}, this enumerator context + * will safely catch exceptions during enumerator restart and use a closeable proxy scheduler to + * invoke tasks on the coordinator main thread to maintain the single threaded property. + */ +@Internal +public class StoppableKafkaEnumContextProxy + implements SplitEnumeratorContext, AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(StoppableKafkaEnumContextProxy.class); + + private final String kafkaClusterId; + private final KafkaMetadataService kafkaMetadataService; + private final SplitEnumeratorContext enumContext; + private final ScheduledExecutorService subEnumeratorWorker; + private boolean noMoreSplits = false; + private volatile boolean isClosing; + + /** + * Constructor for the enumerator context. + * + * @param kafkaClusterId The Kafka cluster id in order to maintain the mapping to the sub + * KafkaSourceEnumerator + * @param kafkaMetadataService the Kafka metadata service to facilitate error handling + * @param enumContext the underlying enumerator context + */ + public StoppableKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext) { + this.kafkaClusterId = kafkaClusterId; + this.kafkaMetadataService = kafkaMetadataService; + this.enumContext = enumContext; + this.subEnumeratorWorker = + Executors.newScheduledThreadPool( + 1, new ExecutorThreadFactory(kafkaClusterId + "-enum-worker")); + this.isClosing = false; + } + + @Override + public SplitEnumeratorMetricGroup metricGroup() { + return enumContext.metricGroup(); + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + enumContext.sendEventToSourceReader(subtaskId, event); + } + + @Override + public int currentParallelism() { + return enumContext.currentParallelism(); + } + + @Override + public Map registeredReaders() { + return enumContext.registeredReaders(); + } + + /** Wrap splits with cluster metadata. */ + @Override + public void assignSplits(SplitsAssignment newSplitAssignments) { + if (logger.isInfoEnabled()) { + logger.info( + "Assigning {} splits for cluster {}: {}", + newSplitAssignments.assignment().values().stream() + .mapToLong(Collection::size) + .sum(), + kafkaClusterId, + newSplitAssignments); + } + + Map> readerToSplitsMap = new HashMap<>(); + newSplitAssignments + .assignment() + .forEach( + (subtask, splits) -> + readerToSplitsMap.put( + subtask, + splits.stream() + .map( + split -> + new DynamicKafkaSourceSplit( + kafkaClusterId, split)) + .collect(Collectors.toList()))); + + if (!readerToSplitsMap.isEmpty()) { + enumContext.assignSplits(new SplitsAssignment<>(readerToSplitsMap)); + } + } + + @Override + public void signalNoMoreSplits(int subtask) { + // there are no more splits for this cluster + noMoreSplits = true; + } + + /** Execute the one time callables in the coordinator. */ + @Override + public void callAsync(Callable callable, BiConsumer handler) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), wrapCallAsyncCallableHandler(handler)); + } + + /** + * Schedule task via internal thread pool to proxy task so that the task handler callback can + * execute in the single threaded source coordinator thread pool to avoid synchronization needs. + * + *

Having the scheduled task in the internal thread pool also allows us to cancel the task + * when the context needs to close due to dynamic enumerator restart. + * + *

In the case of KafkaEnumerator partition discovery, the callback modifies KafkaEnumerator + * object state. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + subEnumeratorWorker.scheduleAtFixedRate( + () -> callAsync(callable, handler), initialDelay, period, TimeUnit.MILLISECONDS); + } + + @Override + public void runInCoordinatorThread(Runnable runnable) { + enumContext.runInCoordinatorThread(runnable); + } + + public boolean isNoMoreSplits() { + return noMoreSplits; + } + + /** + * Note that we can't close the source coordinator here, because these contexts can be closed + * during metadata change when the coordinator still needs to continue to run. We can only close + * the coordinator context in Flink job shutdown, which Flink will do for us. That's why there + * is the complexity of the internal thread pools in this class. + * + *

TODO: Attach Flink JIRA ticket -- discuss with upstream how to cancel scheduled tasks + * belonging to enumerator. + */ + @Override + public void close() throws Exception { + logger.info("Closing enum context for {}", kafkaClusterId); + if (subEnumeratorWorker != null) { + // KafkaSubscriber worker thread will fail if admin client is closed in the middle. + // Swallow the error and set the context to closed state. + isClosing = true; + subEnumeratorWorker.shutdown(); + subEnumeratorWorker.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS); + } + } + + /** + * Wraps callable in call async executed in worker thread pool with exception propagation to + * optimize on doing IO in non-coordinator thread. + */ + protected Callable wrapCallAsyncCallable(Callable callable) { + return () -> { + try { + return callable.call(); + } catch (Exception e) { + if (isClosing) { + throw new HandledFlinkKafkaException(e, kafkaClusterId); + } + + Optional throwable = + ExceptionUtils.findThrowable(e, KafkaException.class); + // check if Kafka related and if Kafka cluster is inactive + if (throwable.isPresent() + && !kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new HandledFlinkKafkaException(throwable.get(), kafkaClusterId); + } + + throw e; + } + }; + } + + /** + * Handle exception that is propagated by a callable, executed on coordinator thread. Depending + * on condition(s) the exception may be swallowed or forwarded. This is the Kafka topic + * partition discovery callable handler. + */ + protected BiConsumer wrapCallAsyncCallableHandler( + BiConsumer mainHandler) { + return (result, t) -> { + // check if exception is handled + Optional throwable = + ExceptionUtils.findThrowable(t, HandledFlinkKafkaException.class); + if (throwable.isPresent()) { + logger.warn("Swallowed handled exception for {}.", kafkaClusterId, throwable.get()); + return; + } + + // let the main handler deal with the potential exception + mainHandler.accept(result, t); + }; + } + + /** + * General exception to signal to internal exception handling mechanisms that a benign error + * occurred. + */ + @Internal + public static class HandledFlinkKafkaException extends RuntimeException { + private static final String ERROR_MESSAGE = "An error occurred with %s"; + + private final String kafkaClusterId; + + public HandledFlinkKafkaException(Throwable cause, String kafkaClusterId) { + super(cause); + this.kafkaClusterId = kafkaClusterId; + } + + public String getMessage() { + return String.format(ERROR_MESSAGE, kafkaClusterId); + } + } + + /** + * This factory exposes a way to override the {@link StoppableKafkaEnumContextProxy} used in the + * enumerator. This pluggable factory is extended in unit tests to facilitate invoking the + * periodic discovery loops on demand. + */ + @Internal + public interface StoppableKafkaEnumContextProxyFactory { + + StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService); + + static StoppableKafkaEnumContextProxyFactory getDefaultFactory() { + return (enumContext, kafkaClusterId, kafkaMetadataService) -> + new StoppableKafkaEnumContextProxy( + kafkaClusterId, kafkaMetadataService, enumContext); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java new file mode 100644 index 000000000..01dabf59c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java @@ -0,0 +1,42 @@ +/* + * 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.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.util.HashSet; +import java.util.Set; + +/** Subscribe to streams based on the set of ids. */ +@Internal +public class KafkaStreamSetSubscriber implements KafkaStreamSubscriber { + + private final Set streamIds; + + public KafkaStreamSetSubscriber(Set streamIds) { + this.streamIds = streamIds; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + return new HashSet<>(kafkaMetadataService.describeStreams(streamIds).values()); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java new file mode 100644 index 000000000..bbd4e7f6c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java @@ -0,0 +1,42 @@ +/* + * 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.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.io.Serializable; +import java.util.Set; + +/** + * The subscriber interacts with {@link KafkaMetadataService} to find which Kafka streams the source + * will subscribe to. + */ +@Experimental +public interface KafkaStreamSubscriber extends Serializable { + + /** + * Get the subscribed {@link KafkaStream}s. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the subscribed {@link KafkaStream}s. + */ + Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java new file mode 100644 index 000000000..d95f7e0f0 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java @@ -0,0 +1,53 @@ +/* + * 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.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; +import java.util.regex.Pattern; + +/** To subscribe to streams based on a pattern. */ +@Internal +public class StreamPatternSubscriber implements KafkaStreamSubscriber { + + private final Pattern streamPattern; + + public StreamPatternSubscriber(Pattern streamPattern) { + this.streamPattern = streamPattern; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + Set allStreams = kafkaMetadataService.getAllStreams(); + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (KafkaStream kafkaStream : allStreams) { + String streamId = kafkaStream.getStreamId(); + if (streamPattern.matcher(streamId).find()) { + builder.add(kafkaStream); + } + } + + return builder.build(); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java new file mode 100644 index 000000000..451a05b3a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java @@ -0,0 +1,142 @@ +/* + * 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.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.runtime.metrics.MetricNames; + +import java.util.Map; + +/** + * A custom proxy metric group in order to group {@link + * org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics} by Kafka cluster. + * + *

Reports pending records per cluster under DynamicKafkaSource metric group, motivated by + * standardized connector metrics: + * https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics. + */ +@Internal +public class KafkaClusterMetricGroup implements SourceReaderMetricGroup { + + public static final String DYNAMIC_KAFKA_SOURCE_METRIC_GROUP = "DynamicKafkaSource"; + public static final String KAFKA_CLUSTER_GROUP_NAME = "kafkaCluster"; + + private final MetricGroup internalClusterSpecificMetricGroup; + private final OperatorIOMetricGroup delegateIOMetricGroup; + + public KafkaClusterMetricGroup( + MetricGroup dynamicKafkaSourceMetricGroup, + SourceReaderMetricGroup delegateSourceReaderMetricGroup, + String kafkaClusterId) { + this.internalClusterSpecificMetricGroup = + dynamicKafkaSourceMetricGroup.addGroup(KAFKA_CLUSTER_GROUP_NAME, kafkaClusterId); + this.delegateIOMetricGroup = delegateSourceReaderMetricGroup.getIOMetricGroup(); + } + + MetricGroup getInternalClusterSpecificMetricGroup() { + return internalClusterSpecificMetricGroup; + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingBytesGauge(Gauge gauge) { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) { + gauge(MetricNames.PENDING_RECORDS, pendingRecordsGauge); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return delegateIOMetricGroup; + } + + // --------------- MetricGroup methods --------------- + + @Override + public Counter counter(String name) { + return internalClusterSpecificMetricGroup.counter(name); + } + + @Override + public C counter(String name, C counter) { + return internalClusterSpecificMetricGroup.counter(name, counter); + } + + @Override + public > G gauge(String name, G gauge) { + return internalClusterSpecificMetricGroup.gauge(name, gauge); + } + + @Override + public H histogram(String name, H histogram) { + return internalClusterSpecificMetricGroup.histogram(name, histogram); + } + + @Override + public M meter(String name, M meter) { + return internalClusterSpecificMetricGroup.meter(name, meter); + } + + @Override + public MetricGroup addGroup(String name) { + return internalClusterSpecificMetricGroup.addGroup(name); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return internalClusterSpecificMetricGroup.addGroup(key, value); + } + + @Override + public String[] getScopeComponents() { + return internalClusterSpecificMetricGroup.getScopeComponents(); + } + + @Override + public Map getAllVariables() { + return internalClusterSpecificMetricGroup.getAllVariables(); + } + + @Override + public String getMetricIdentifier(String metricName) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName); + } + + @Override + public String getMetricIdentifier(String metricName, CharacterFilter filter) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName, filter); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java new file mode 100644 index 000000000..0659f528e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java @@ -0,0 +1,76 @@ +/* + * 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.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** Manages metric groups for each cluster. */ +@Internal +public class KafkaClusterMetricGroupManager implements AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(KafkaClusterMetricGroupManager.class); + private final Map metricGroups; + + public KafkaClusterMetricGroupManager() { + this.metricGroups = new HashMap<>(); + } + + public void register(String kafkaClusterId, KafkaClusterMetricGroup kafkaClusterMetricGroup) { + if (kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup() + instanceof AbstractMetricGroup) { + metricGroups.put( + kafkaClusterId, + (AbstractMetricGroup) + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup()); + } else { + logger.warn( + "MetricGroup {} is an instance of {}, which is not supported. Please use an implementation of AbstractMetricGroup.", + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup(), + kafkaClusterMetricGroup + .getInternalClusterSpecificMetricGroup() + .getClass() + .getSimpleName()); + } + } + + public void close(String kafkaClusterId) { + AbstractMetricGroup metricGroup = metricGroups.remove(kafkaClusterId); + if (metricGroup != null) { + metricGroup.close(); + } else { + logger.warn( + "Tried to close metric group for {} but it is not registered for lifecycle management", + kafkaClusterId); + } + } + + @Override + public void close() throws Exception { + for (AbstractMetricGroup metricGroup : metricGroups.values()) { + metricGroup.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java new file mode 100644 index 000000000..133d49a7b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java @@ -0,0 +1,539 @@ +/* + * 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.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroupManager; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; +import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.runtime.io.MultipleFuturesAvailabilityHelper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.UserCodeClassLoader; + +import com.google.common.collect.ArrayListMultimap; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Properties; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Manages state about underlying {@link KafkaSourceReader} to collect records and commit offsets + * from multiple Kafka clusters. This reader also handles changes to Kafka topology by reacting to + * restart sequence initiated by the enumerator and suspending inconsistent sub readers. + * + *

First, in the restart sequence, we will receive the {@link MetadataUpdateEvent} from the + * enumerator, stop all KafkaSourceReaders, and retain the relevant splits. Second, enumerator will + * send all new splits that readers should work on (old splits will not be sent again). + */ +@Internal +public class DynamicKafkaSourceReader implements SourceReader { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceReader.class); + private final KafkaRecordDeserializationSchema deserializationSchema; + private final Properties properties; + private final MetricGroup dynamicKafkaSourceMetricGroup; + private final Gauge kafkaClusterCount; + private final SourceReaderContext readerContext; + private final KafkaClusterMetricGroupManager kafkaClusterMetricGroupManager; + + // needs have a strict ordering for readers to guarantee availability future consistency + private final NavigableMap> clusterReaderMap; + private final Map clustersProperties; + private final List pendingSplits; + + private MultipleFuturesAvailabilityHelper availabilityHelper; + private boolean isActivelyConsumingSplits; + private boolean isNoMoreSplits; + private boolean restartingReaders; + + public DynamicKafkaSourceReader( + SourceReaderContext readerContext, + KafkaRecordDeserializationSchema deserializationSchema, + Properties properties) { + this.readerContext = readerContext; + this.clusterReaderMap = new TreeMap<>(); + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaClusterCount = clusterReaderMap::size; + this.dynamicKafkaSourceMetricGroup = + readerContext + .metricGroup() + .addGroup(KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + this.kafkaClusterMetricGroupManager = new KafkaClusterMetricGroupManager(); + this.pendingSplits = new ArrayList<>(); + this.availabilityHelper = new MultipleFuturesAvailabilityHelper(0); + this.isNoMoreSplits = false; + this.isActivelyConsumingSplits = false; + this.restartingReaders = false; + this.clustersProperties = new HashMap<>(); + } + + /** + * This is invoked first only in reader startup without state. In stateful startup, splits are + * added before this method is invoked. + */ + @Override + public void start() { + logger.trace("Starting reader for subtask index={}", readerContext.getIndexOfSubtask()); + // metrics cannot be registered in the enumerator + readerContext.metricGroup().gauge("kafkaClusterCount", kafkaClusterCount); + readerContext.sendSourceEventToCoordinator(new GetMetadataUpdateEvent()); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + // do not return end of input if no more splits has not yet been signaled + if (!isNoMoreSplits && clusterReaderMap.isEmpty()) { + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + if (restartingReaders) { + logger.info("Poll next invoked while restarting readers"); + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + boolean isMoreAvailable = false; + boolean isNothingAvailable = false; + + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + + InputStatus inputStatus = clusterReader.getValue().pollNext(readerOutput); + switch (inputStatus) { + case MORE_AVAILABLE: + isMoreAvailable = true; + break; + case NOTHING_AVAILABLE: + isNothingAvailable = true; + break; + } + } + + return logAndReturnInputStatus(consolidateInputStatus(isMoreAvailable, isNothingAvailable)); + } + + private InputStatus consolidateInputStatus( + boolean atLeastOneMoreAvailable, boolean atLeastOneNothingAvailable) { + final InputStatus inputStatus; + if (atLeastOneMoreAvailable) { + inputStatus = InputStatus.MORE_AVAILABLE; + } else if (atLeastOneNothingAvailable) { + inputStatus = InputStatus.NOTHING_AVAILABLE; + } else { + inputStatus = InputStatus.END_OF_INPUT; + } + return inputStatus; + } + + // we also need to filter splits at startup in case checkpoint is not consistent bwtn enumerator + // and reader + @Override + public void addSplits(List splits) { + logger.info("Adding splits to reader {}: {}", readerContext.getIndexOfSubtask(), splits); + // at startup, don't add splits until we get confirmation from enumerator of the current + // metadata + if (!isActivelyConsumingSplits) { + pendingSplits.addAll(splits); + return; + } + + ArrayListMultimap clusterSplitsMap = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + clusterSplitsMap.put(split.getKafkaClusterId(), split); + } + + Set kafkaClusterIds = clusterSplitsMap.keySet(); + + boolean newCluster = false; + for (String kafkaClusterId : kafkaClusterIds) { + // if a reader corresponding to the split doesn't exist, create it + // it is possible that the splits come before the source event + if (!clusterReaderMap.containsKey(kafkaClusterId)) { + try { + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + kafkaSourceReader.start(); + newCluster = true; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // add splits + KafkaSourceReader reader = clusterReaderMap.get(kafkaClusterId); + reader.addSplits(clusterSplitsMap.get(kafkaClusterId)); + } + + // reset the availability future to also depend on the new sub readers + if (newCluster) { + completeAndResetAvailabilityHelper(); + } + } + + /** + * Duplicate source events are handled with idempotency. No metadata change means we simply skip + * the restart logic. + */ + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof MetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + logger.info( + "Received source event {}: subtask={}", + sourceEvent, + readerContext.getIndexOfSubtask()); + Set newKafkaStreams = ((MetadataUpdateEvent) sourceEvent).getKafkaStreams(); + Map> newClustersAndTopics = new HashMap<>(); + Map newClustersProperties = new HashMap<>(); + for (KafkaStream kafkaStream : newKafkaStreams) { + for (Map.Entry clusterMetadataMapEntry : + kafkaStream.getClusterMetadataMap().entrySet()) { + newClustersAndTopics + .computeIfAbsent( + clusterMetadataMapEntry.getKey(), (unused) -> new HashSet<>()) + .addAll(clusterMetadataMapEntry.getValue().getTopics()); + + newClustersProperties.put( + clusterMetadataMapEntry.getKey(), + clusterMetadataMapEntry.getValue().getProperties()); + } + } + + // filter current splits with the metadata update + List currentSplitState = snapshotStateFromAllReaders(-1); + logger.info( + "Snapshotting split state for reader {}: {}", + readerContext.getIndexOfSubtask(), + currentSplitState); + Map> currentMetadataFromState = new HashMap<>(); + Map> filteredNewClusterSplitStateMap = new HashMap<>(); + + // the data structures above + for (DynamicKafkaSourceSplit split : currentSplitState) { + currentMetadataFromState + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new HashSet<>()) + .add(split.getKafkaPartitionSplit().getTopic()); + // check if cluster topic exists in the metadata update + if (newClustersAndTopics.containsKey(split.getKafkaClusterId()) + && newClustersAndTopics + .get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic())) { + filteredNewClusterSplitStateMap + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new ArrayList<>()) + .add(split); + } else { + logger.info("Skipping outdated split due to metadata changes: {}", split); + } + } + + // only restart if there was metadata change to handle duplicate MetadataUpdateEvent from + // enumerator. We can possibly only restart the readers whose metadata has changed but that + // comes at the cost of complexity and it is an optimization for a corner case. We can + // revisit if necessary. + if (!newClustersAndTopics.equals(currentMetadataFromState)) { + restartingReaders = true; + closeAllReadersAndClearState(); + + clustersProperties.putAll(newClustersProperties); + for (String kafkaClusterId : newClustersAndTopics.keySet()) { + try { + // restart kafka source readers with the relevant state + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + if (filteredNewClusterSplitStateMap.containsKey(kafkaClusterId)) { + kafkaSourceReader.addSplits( + filteredNewClusterSplitStateMap.get(kafkaClusterId)); + } + kafkaSourceReader.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // reset the availability future to also depend on the new sub readers + completeAndResetAvailabilityHelper(); + } else { + // update properties even on no metadata change + clustersProperties.clear(); + clustersProperties.putAll(newClustersProperties); + } + + // finally mark the reader as active, if not already and add pending splits + if (!isActivelyConsumingSplits) { + isActivelyConsumingSplits = true; + } + + if (!pendingSplits.isEmpty()) { + List validPendingSplits = + pendingSplits.stream() + // Pending splits is used to cache splits at startup, before metadata + // update event arrives. Splits in state could be old and it's possible + // to not have another metadata update event, so need to filter the + // splits at this point. + .filter( + pendingSplit -> { + boolean splitValid = + isSplitForActiveClusters( + pendingSplit, newClustersAndTopics); + if (!splitValid) { + logger.info( + "Removing invalid split for reader: {}", + pendingSplit); + } + return splitValid; + }) + .collect(Collectors.toList()); + + addSplits(validPendingSplits); + pendingSplits.clear(); + if (isNoMoreSplits) { + notifyNoMoreSplits(); + } + } + } + + private static boolean isSplitForActiveClusters( + DynamicKafkaSourceSplit split, Map> metadata) { + return metadata.containsKey(split.getKafkaClusterId()) + && metadata.get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic()); + } + + @Override + public List snapshotState(long checkpointId) { + List splits = snapshotStateFromAllReaders(checkpointId); + + // pending splits should be typically empty, since we do not add splits to pending splits if + // reader has started + splits.addAll(pendingSplits); + return splits; + } + + private List snapshotStateFromAllReaders(long checkpointId) { + List splits = new ArrayList<>(); + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + clusterReader + .getValue() + .snapshotState(checkpointId) + .forEach( + kafkaPartitionSplit -> + splits.add( + new DynamicKafkaSourceSplit( + clusterReader.getKey(), kafkaPartitionSplit))); + } + + return splits; + } + + @Override + public CompletableFuture isAvailable() { + availabilityHelper.resetToUnAvailable(); + syncAvailabilityHelperWithReaders(); + return (CompletableFuture) availabilityHelper.getAvailableFuture(); + } + + @Override + public void notifyNoMoreSplits() { + logger.info("notify no more splits for reader {}", readerContext.getIndexOfSubtask()); + if (pendingSplits.isEmpty()) { + clusterReaderMap.values().forEach(KafkaSourceReader::notifyNoMoreSplits); + } + + isNoMoreSplits = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + logger.debug("Notify checkpoint complete for {}", clusterReaderMap.keySet()); + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.notifyCheckpointComplete(checkpointId); + } + } + + @Override + public void close() throws Exception { + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.close(); + } + kafkaClusterMetricGroupManager.close(); + } + + private KafkaSourceReader createReader(String kafkaClusterId) throws Exception { + FutureCompletingBlockingQueue>> + elementsQueue = new FutureCompletingBlockingQueue<>(); + Properties readerSpecificProperties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, readerSpecificProperties); + KafkaPropertiesUtil.copyProperties( + Preconditions.checkNotNull( + clustersProperties.get(kafkaClusterId), + "Properties for cluster %s is not found. Current Kafka cluster ids: %s", + kafkaClusterId, + clustersProperties.keySet()), + readerSpecificProperties); + KafkaPropertiesUtil.setClientIdPrefix(readerSpecificProperties, kafkaClusterId); + + // layer a kafka cluster group to distinguish metrics by cluster + KafkaClusterMetricGroup kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + dynamicKafkaSourceMetricGroup, readerContext.metricGroup(), kafkaClusterId); + kafkaClusterMetricGroupManager.register(kafkaClusterId, kafkaClusterMetricGroup); + KafkaSourceReaderMetrics kafkaSourceReaderMetrics = + new KafkaSourceReaderMetrics(kafkaClusterMetricGroup); + + deserializationSchema.open( + new DeserializationSchema.InitializationContext() { + @Override + public MetricGroup getMetricGroup() { + // adding kafkaClusterMetricGroup instead of the sourceReaderMetricGroup + // since there could be metric collision, so `kafkaCluster` group is + // necessary to + // distinguish between instances of this metric + return kafkaClusterMetricGroup.addGroup("deserializer"); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + }); + + KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); + return new KafkaSourceReader<>( + elementsQueue, + new KafkaSourceFetcherManager( + elementsQueue, + () -> + new KafkaPartitionSplitReaderWrapper( + readerSpecificProperties, + readerContext, + kafkaSourceReaderMetrics, + kafkaClusterId), + (ignore) -> {}), + recordEmitter, + toConfiguration(readerSpecificProperties), + readerContext, + kafkaSourceReaderMetrics); + } + + /** + * In metadata change, we need to reset the availability helper since the number of Kafka source + * readers could have changed. + */ + private void completeAndResetAvailabilityHelper() { + CompletableFuture cachedPreviousFuture = availabilityHelper.getAvailableFuture(); + availabilityHelper = new MultipleFuturesAvailabilityHelper(clusterReaderMap.size()); + syncAvailabilityHelperWithReaders(); + + // We cannot immediately complete the previous future here. We must complete it only when + // the new readers have finished handling the split assignment. Completing the future too + // early can cause WakeupException (implicitly woken up by invocation to pollNext()) if the + // reader has not finished resolving the positions of the splits, as seen in flaky unit test + // errors. There is no error handling for WakeupException in SplitReader's + // handleSplitChanges. + availabilityHelper + .getAvailableFuture() + .whenComplete( + (ignore, t) -> { + restartingReaders = false; + cachedPreviousFuture.complete(null); + }); + } + + private void syncAvailabilityHelperWithReaders() { + int i = 0; + for (String kafkaClusterId : clusterReaderMap.navigableKeySet()) { + availabilityHelper.anyOf(i, clusterReaderMap.get(kafkaClusterId).isAvailable()); + i++; + } + } + + private void closeAllReadersAndClearState() { + for (Map.Entry> entry : clusterReaderMap.entrySet()) { + try { + logger.info( + "Closing sub reader in reader {} for cluster: {}", + readerContext.getIndexOfSubtask(), + entry.getKey()); + entry.getValue().close(); + kafkaClusterMetricGroupManager.close(entry.getKey()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + clusterReaderMap.clear(); + clustersProperties.clear(); + } + + static Configuration toConfiguration(Properties props) { + Configuration config = new Configuration(); + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); + return config; + } + + private InputStatus logAndReturnInputStatus(InputStatus inputStatus) { + logger.trace( + "inputStatus={}, subtaskIndex={}", inputStatus, readerContext.getIndexOfSubtask()); + return inputStatus; + } + + @VisibleForTesting + public MultipleFuturesAvailabilityHelper getAvailabilityHelper() { + return availabilityHelper; + } + + @VisibleForTesting + public boolean isActivelyConsumingSplits() { + return isActivelyConsumingSplits; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java new file mode 100644 index 000000000..cc250eace --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java @@ -0,0 +1,98 @@ +/* + * 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.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** This extends to Kafka Partition Split Reader to wrap split ids with the cluster name. */ +@Internal +public class KafkaPartitionSplitReaderWrapper extends KafkaPartitionSplitReader + implements AutoCloseable { + private final String kafkaClusterId; + + public KafkaPartitionSplitReaderWrapper( + Properties props, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics, + String kafkaClusterId) { + super(props, context, kafkaSourceReaderMetrics); + this.kafkaClusterId = kafkaClusterId; + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + return new WrappedRecordsWithSplitIds(super.fetch(), kafkaClusterId); + } + + private static final class WrappedRecordsWithSplitIds + implements RecordsWithSplitIds> { + + private final RecordsWithSplitIds> delegate; + private final String kafkaClusterId; + + public WrappedRecordsWithSplitIds( + RecordsWithSplitIds> delegate, + String kafkaClusterId) { + this.delegate = delegate; + this.kafkaClusterId = kafkaClusterId; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = delegate.nextSplit(); + if (nextSplit == null) { + return nextSplit; + } else { + return kafkaClusterId + "-" + nextSplit; + } + } + + @Nullable + @Override + public ConsumerRecord nextRecordFromSplit() { + return delegate.nextRecordFromSplit(); + } + + @Override + public Set finishedSplits() { + return delegate.finishedSplits().stream() + .map(finishedSplit -> kafkaClusterId + "-" + finishedSplit) + .collect(Collectors.toSet()); + } + + @Override + public void recycle() { + delegate.recycle(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java new file mode 100644 index 000000000..25ef25b99 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java @@ -0,0 +1,85 @@ +/* + * 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.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; + +/** Split that wraps {@link KafkaPartitionSplit} with Kafka cluster information. */ +@Internal +public class DynamicKafkaSourceSplit extends KafkaPartitionSplit { + + private final String kafkaClusterId; + private final KafkaPartitionSplit kafkaPartitionSplit; + + public DynamicKafkaSourceSplit(String kafkaClusterId, KafkaPartitionSplit kafkaPartitionSplit) { + super( + kafkaPartitionSplit.getTopicPartition(), + kafkaPartitionSplit.getStartingOffset(), + kafkaPartitionSplit.getStoppingOffset().orElse(NO_STOPPING_OFFSET)); + this.kafkaClusterId = kafkaClusterId; + this.kafkaPartitionSplit = kafkaPartitionSplit; + } + + @Override + public String splitId() { + return kafkaClusterId + "-" + kafkaPartitionSplit.splitId(); + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaPartitionSplit getKafkaPartitionSplit() { + return kafkaPartitionSplit; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaPartitionSplit", kafkaPartitionSplit) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + DynamicKafkaSourceSplit that = (DynamicKafkaSourceSplit) o; + return Objects.equals(kafkaClusterId, that.kafkaClusterId) + && Objects.equals(kafkaPartitionSplit, that.kafkaPartitionSplit); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), kafkaClusterId, kafkaPartitionSplit); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java new file mode 100644 index 000000000..852894f8b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java @@ -0,0 +1,76 @@ +/* + * 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.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import com.google.common.io.ByteStreams; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** (De)serializes the {@link DynamicKafkaSourceSplit}. */ +@Internal +public class DynamicKafkaSourceSplitSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaPartitionSplitSerializer kafkaPartitionSplitSerializer; + + public DynamicKafkaSourceSplitSerializer() { + this.kafkaPartitionSplitSerializer = new KafkaPartitionSplitSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceSplit split) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeUTF(split.getKafkaClusterId()); + out.writeInt(kafkaPartitionSplitSerializer.getVersion()); + out.write(kafkaPartitionSplitSerializer.serialize(split.getKafkaPartitionSplit())); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceSplit deserialize(int version, byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + String kafkaClusterId = in.readUTF(); + int kafkaPartitionSplitSerializerVersion = in.readInt(); + KafkaPartitionSplit kafkaPartitionSplit = + kafkaPartitionSplitSerializer.deserialize( + kafkaPartitionSplitSerializerVersion, ByteStreams.toByteArray(in)); + return new DynamicKafkaSourceSplit(kafkaClusterId, kafkaPartitionSplit); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java new file mode 100644 index 000000000..0e29576cc --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java @@ -0,0 +1,67 @@ +/* + * 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; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nonnull; + +import java.util.Properties; + +/** Utility class for modify Kafka properties. */ +@Internal +public class KafkaPropertiesUtil { + + private KafkaPropertiesUtil() {} + + public static void copyProperties(@Nonnull Properties from, @Nonnull Properties to) { + for (String key : from.stringPropertyNames()) { + to.setProperty(key, from.getProperty(key)); + } + } + + /** + * client.id is used for Kafka server side logging, see + * https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html#consumerconfigs_client.id + * + *

Set client id prefix to avoid mbean collision warning logs. There are multiple instances + * of the AdminClient/KafkaConsumer so each instance requires a different client id (i.e. also + * per cluster). + * + *

Flink internally configures the clientId, making this the only way to customize the Kafka + * client id parameter. + * + *

If this is not done, we will encounter warning logs of the form: + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.consumer:type=app-info,id=null-enumerator-consumer + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.admin.client:type=app-info,id=null-enumerator-admin-client + */ + public static void setClientIdPrefix(Properties properties, String kafkaClusterId) { + String userClientIdPrefix = + properties.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + properties.setProperty( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + userClientIdPrefix + "-" + kafkaClusterId); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java index 1375d9f60..70c435ee3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java @@ -42,7 +42,7 @@ public KafkaSourceEnumState( this.initialDiscoveryFinished = initialDiscoveryFinished; } - KafkaSourceEnumState( + public KafkaSourceEnumState( Set assignPartitions, Set unassignedInitialPartitions, boolean initialDiscoveryFinished) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java new file mode 100644 index 000000000..4ea1bd7dc --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java @@ -0,0 +1,694 @@ +/* + * 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.dynamic.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DynamicKafkaSourceExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.connector.kafka.testutils.TwoKafkaContainers; +import org.apache.flink.connector.kafka.testutils.YamlFileMetadataService; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for {@link org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource}. + */ +public class DynamicKafkaSourceITTest extends TestLogger { + + private static final String TOPIC = "DynamicKafkaSourceITTest"; + private static final int NUM_PARTITIONS = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata1; + MiniClusterWithClientResource miniClusterResource; + static InMemoryReporter reporter; + + @TempDir File testDir; + + @Nested + @TestInstance(TestInstance.Lifecycle.PER_CLASS) + class DynamicKafkaSourceSpecificTests { + @BeforeAll + void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_PARTITIONS, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + kafkaClusterTestEnvMetadata1 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1); + } + + @BeforeEach + void beforeEach() throws Exception { + reporter = InMemoryReporter.create(); + miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(2) + .setConfiguration( + reporter.addToConfiguration(new Configuration())) + .build()); + miniClusterResource.before(); + } + + @AfterEach + void afterEach() { + reporter.close(); + miniClusterResource.after(); + } + + @AfterAll + void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testBasicMultiClusterRead() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + mockKafkaMetadataService.getAllStreams().stream() + .map(KafkaStream::getStreamId) + .collect(Collectors.toSet())) + .setKafkaMetadataService(mockKafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() + < DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT + && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range( + 0, + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testSingleClusterTopicMetadataService() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + + KafkaMetadataService kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + // use topics as stream ids + Collections.singleton(TOPIC)) + .setKafkaMetadataService(kafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() < NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testMigrationUsingFileMetadataService() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-file-metadata-service"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + AtomicInteger latestValueOffset = + new AtomicInteger( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0)); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + results.add(iterator.next()); + + // trigger metadata update to consume from two clusters + if (results.size() == NUM_RECORDS_PER_SPLIT) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(0), + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + + // trigger another metadata update to remove old cluster + if (results.size() == latestValueOffset.get()) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + } catch (NoSuchElementException e) { + // swallow and wait + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + // we will produce 3x + return results.size() == NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3; + }, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + } + + // verify no data loss / duplication in metadata changes + // cluster0 contains 0-10 + // cluster 1 contains 10-30 + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testStreamPatternSubscriber() throws Throwable { + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-1", NUM_PARTITIONS); + int lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, "stream-pattern-test-1", NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-2", NUM_PARTITIONS); + lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + "stream-pattern-test-2", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + DynamicKafkaSourceTestHelper.createTopic(1, "stream-pattern-test-3", NUM_PARTITIONS); + final int totalRecords = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + "stream-pattern-test-3", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + + // create new metadata file to consume from 1 cluster + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + + Set kafkaStreams = + getKafkaStreams( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties(), + ImmutableSet.of("stream-pattern-test-1", "stream-pattern-test-2")); + + writeClusterMetadataToFile(metadataFile, kafkaStreams); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamPattern(Pattern.compile("stream-pattern-test-.+")) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + Integer record = iterator.next(); + results.add(record); + + // add third stream that matches the regex + if (results.size() == NUM_RECORDS_PER_SPLIT) { + kafkaStreams.add( + getKafkaStream( + kafkaClusterTestEnvMetadata1 + .getKafkaClusterId(), + kafkaClusterTestEnvMetadata1 + .getStandardProperties(), + "stream-pattern-test-3")); + writeClusterMetadataToFile(metadataFile, kafkaStreams); + } + } catch (NoSuchElementException e) { + // swallow + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return results.size() == totalRecords; + }, + Duration.ofSeconds(15), + "Could not obtain the required records within the timeout"); + } + // verify no data loss / duplication in metadata changes + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, totalRecords).boxed().collect(Collectors.toList())); + } + + @Test + void testMetricsLifecycleManagement() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-metrics-lifecycle-mgmt"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + env.setRestartStrategy(RestartStrategies.noRestart()); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty( + CommonClientConfigs.GROUP_ID_CONFIG, "testMetricsLifecycleManagement"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + + int latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + List results = new ArrayList<>(); + try (CloseableIterator iterator = stream.executeAndCollect()) { + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + assertThat(results) + .containsOnlyOnceElementsOf( + IntStream.range(0, latestValueOffset) + .boxed() + .collect(Collectors.toList())); + + // should contain cluster 0 metrics + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + // setup test data for cluster 1 and stop consuming from cluster 0 + latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1))); + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + // cluster 0 is not being consumed from, metrics should not appear + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .doesNotContainPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-1.*")); + } + } + + private void writeClusterMetadataToFile(File metadataFile, Set kafkaStreams) + throws IOException { + List streamMetadataList = new ArrayList<>(); + for (KafkaStream kafkaStream : kafkaStreams) { + List clusterMetadataList = + new ArrayList<>(); + + for (Map.Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + YamlFileMetadataService.StreamMetadata.ClusterMetadata clusterMetadata = + new YamlFileMetadataService.StreamMetadata.ClusterMetadata(); + clusterMetadata.setClusterId(entry.getKey()); + clusterMetadata.setBootstrapServers( + entry.getValue() + .getProperties() + .getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + clusterMetadata.setTopics(new ArrayList<>(entry.getValue().getTopics())); + clusterMetadataList.add(clusterMetadata); + } + + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(); + streamMetadata.setStreamId(kafkaStream.getStreamId()); + streamMetadata.setClusterMetadataList(clusterMetadataList); + streamMetadataList.add(streamMetadata); + } + + YamlFileMetadataService.saveToYaml(streamMetadataList, metadataFile); + } + + private void writeClusterMetadataToFile( + File metadataFile, + String streamId, + String topic, + List kafkaClusterTestEnvMetadataList) + throws IOException { + List clusterMetadata = + kafkaClusterTestEnvMetadataList.stream() + .map( + KafkaClusterTestEnvMetadata -> + new YamlFileMetadataService.StreamMetadata + .ClusterMetadata( + KafkaClusterTestEnvMetadata.getKafkaClusterId(), + KafkaClusterTestEnvMetadata + .getBrokerConnectionStrings(), + ImmutableList.of(topic))) + .collect(Collectors.toList()); + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(streamId, clusterMetadata); + YamlFileMetadataService.saveToYaml( + Collections.singletonList(streamMetadata), metadataFile); + } + + private Set findMetrics(InMemoryReporter inMemoryReporter, String groupPattern) { + Optional groups = inMemoryReporter.findGroup(groupPattern); + assertThat(groups).isPresent(); + return inMemoryReporter.getMetricsByGroup(groups.get()).keySet().stream() + .map(metricName -> groups.get().getMetricIdentifier(metricName)) + .collect(Collectors.toSet()); + } + + private Set getKafkaStreams( + String kafkaClusterId, Properties properties, Collection topics) { + return topics.stream() + .map(topic -> getKafkaStream(kafkaClusterId, properties, topic)) + .collect(Collectors.toSet()); + } + + private KafkaStream getKafkaStream( + String kafkaClusterId, Properties properties, String topic) { + return new KafkaStream( + topic, + Collections.singletonMap( + kafkaClusterId, + new ClusterMetadata(Collections.singleton(topic), properties))); + } + } + + /** Integration test based on connector testing framework. */ + @Nested + class IntegrationTests extends SourceTestSuiteBase { + @TestSemantics + CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + // Defines test environment on Flink MiniCluster + @SuppressWarnings("unused") + @TestEnv + MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + @TestExternalSystem + DefaultContainerizedExternalSystem twoKafkas = + DefaultContainerizedExternalSystem.builder() + .fromContainer(new TwoKafkaContainers()) + .build(); + + @SuppressWarnings("unused") + @TestContext + DynamicKafkaSourceExternalContextFactory twoClusters = + new DynamicKafkaSourceExternalContextFactory( + twoKafkas.getContainer().getKafka0(), + twoKafkas.getContainer().getKafka1(), + Collections.emptyList()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java new file mode 100644 index 000000000..66caec4c7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java @@ -0,0 +1,118 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.AssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test {@link + * org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer}. + */ +public class DynamicKafkaSourceEnumStateSerializerTest { + + @Test + public void testSerde() throws Exception { + DynamicKafkaSourceEnumStateSerializer dynamicKafkaSourceEnumStateSerializer = + new DynamicKafkaSourceEnumStateSerializer(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster0:9092"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster1:9092"); + + Set kafkaStreams = + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster1)))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = + new DynamicKafkaSourceEnumState( + kafkaStreams, + ImmutableMap.of( + "cluster0", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic0", 0), + AssignmentStatus.ASSIGNED), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic1", 1), + AssignmentStatus.UNASSIGNED_INITIAL)), + true), + "cluster1", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic2", 0), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic3", 1), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic4", 2), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic5", 3), + AssignmentStatus.UNASSIGNED_INITIAL)), + false))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumStateAfterSerde = + dynamicKafkaSourceEnumStateSerializer.deserialize( + 1, + dynamicKafkaSourceEnumStateSerializer.serialize( + dynamicKafkaSourceEnumState)); + + assertThat(dynamicKafkaSourceEnumState) + .usingRecursiveComparison() + .isEqualTo(dynamicKafkaSourceEnumStateAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java new file mode 100644 index 000000000..05046d406 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java @@ -0,0 +1,964 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** A test for {@link DynamicKafkaSourceEnumerator}. */ +public class DynamicKafkaSourceEnumeratorTest { + private static final int NUM_SUBTASKS = 3; + private static final String TOPIC = "DynamicKafkaSourceEnumeratorTest"; + private static final int NUM_SPLITS_PER_CLUSTER = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + @BeforeAll + public static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + } + + @AfterAll + public static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + public void testStartupWithoutContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getOneTimeCallables()).isEmpty(); + assertThat(context.getPeriodicCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_noPeriodicDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, new MockKafkaMetadataService(true), (properties) -> {})) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + assertThatThrownBy(() -> runAllOneTimeCallables(context)) + .as( + "Exception expected since periodic discovery is disabled and metadata is required for setting up the job") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_withContinuousDiscovery() + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(true), + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .as("Exception expected since there is no state") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void + testStartupWithKafkaMetadataServiceFailure_withContinuousDiscoveryAndCheckpointState() + throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService(true), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + // no exception + context.runPeriodicCallable(0); + + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testHandleMetadataServiceError() throws Throwable { + int failureThreshold = 5; + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + Integer.toString(failureThreshold)); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + context.runPeriodicCallable(0); + + // init splits + runAllOneTimeCallables(context); + + // swap to exceptional metadata service + mockKafkaMetadataService.setThrowException(true); + + for (int i = 0; i < failureThreshold; i++) { + context.runPeriodicCallable(0); + } + + for (int i = 0; i < 2; i++) { + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + // Need to reset internal throwable reference after each invocation of + // runPeriodicCallable, + // since the context caches the previous exceptions indefinitely + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(context, "errorInWorkerThread"); + errorInWorkerThread.set(null); + } + + mockKafkaMetadataService.setThrowException(false); + assertThatCode(() -> context.runPeriodicCallable(0)) + .as("Exception counter should have been reset") + .doesNotThrowAnyException(); + } + } + + @Test + public void testKafkaMetadataServiceDiscovery() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + mockKafkaMetadataService, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + context.runPeriodicCallable(0); + + // 1 callable for main enumerator and 2 for the sub enumerators since we have 2 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + int currentNumSplits = context.getSplitsAssignmentSequence().size(); + + // no changes to splits + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()).hasSize(currentNumSplits); + + // mock metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + + // changes should have occurred here + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "1 additional split assignment since there was 1 metadata update that caused a change") + .hasSize(currentNumSplits + 1); + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + } + } + + @Test + public void testReaderRegistrationAfterSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no readers have registered yet") + .isEmpty(); + assertThat(context.getSentSourceEvent()).as("no readers have registered yet").isEmpty(); + + // initialize readers 0 and 2 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSentSourceEvent().keySet()) + .as("reader 0 and 2 should have only received the source event") + .containsExactlyInAnyOrder(0, 2); + Set allReadersThatReceivedSplits = + context.getSplitsAssignmentSequence().stream() + .flatMap( + splitAssignment -> + splitAssignment.assignment().keySet().stream()) + .collect(Collectors.toSet()); + assertThat(allReadersThatReceivedSplits) + .as("reader 0 and 2 should hve only received splits") + .containsExactlyInAnyOrder(0, 2); + + // initialize readers 1 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + assertThat(context.getSentSourceEvent().keySet()) + .as("all readers should have received get metadata update event") + .containsExactlyInAnyOrder(0, 1, 2); + + for (List sourceEventsPerReader : context.getSentSourceEvent().values()) { + assertThat(sourceEventsPerReader) + .as("there should have been only 1 source event per reader") + .hasSize(1); + } + + // should have all splits assigned by now + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testReaderRegistrationBeforeSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testSnapshotState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + DynamicKafkaSourceEnumState stateBeforeSplitAssignment = enumerator.snapshotState(-1); + assertThat( + stateBeforeSplitAssignment.getClusterEnumeratorStates().values() + .stream() + .map(subState -> subState.assignedPartitions().stream()) + .count()) + .as("no readers registered, so state should be empty") + .isZero(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + DynamicKafkaSourceEnumState stateAfterSplitAssignment = enumerator.snapshotState(-1); + + assertThat( + stateAfterSplitAssignment.getClusterEnumeratorStates().values().stream() + .flatMap(enumState -> enumState.assignedPartitions().stream()) + .count()) + .isEqualTo( + NUM_SPLITS_PER_CLUSTER + * DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + } + } + + @Test + public void testStartupWithCheckpointState() throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + // start and check callables + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as( + "3 one time callables should have been scheduled. 1 for main enumerator and then 2 for each underlying enumerator") + .hasSize(1 + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers and do split assignment + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + runAllOneTimeCallables(context); + + assertThat(context.getSentSourceEvent()).as("3 readers registered").hasSize(3); + for (List sourceEventsReceived : context.getSentSourceEvent().values()) { + assertThat(sourceEventsReceived) + .as("each reader should have sent 1 source event") + .hasSize(1); + } + + assertThat(context.getSplitsAssignmentSequence()) + .as( + "there should not be new splits and we don't assign previously assigned splits at startup and there is no metadata/split changes") + .isEmpty(); + } + + // test with periodic discovery enabled + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + assertThat(context.getPeriodicCallables()) + .as("A periodic time partition discovery callable should have been scheduled") + .hasSize(1); + assertThat(context.getOneTimeCallables()) + .as( + "0 one time callables for main enumerator and 2 one time callables for each underlying enumerator should have been scheduled") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + // checkpoint state should have triggered split assignment + assertThat(context.getSplitsAssignmentSequence()) + .as( + "There is no split assignment since there are no new splits that are not contained in state") + .isEmpty(); + } + } + + @Test + public void testAddSplitsBack() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + runAllOneTimeCallables(context); + + Map> readerAssignmentsBeforeFailure = + getReaderAssignments(context); + assertThat(context.getSplitsAssignmentSequence()) + .as("we only expect splits have been assigned 2 times") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // simulate failures + context.unregisterReader(0); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(0), 0); + context.unregisterReader(2); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(2), 2); + assertThat(context.getSplitsAssignmentSequence()) + .as("Splits assignment should be unchanged") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // mock reader recovery + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + assertThat(getReaderAssignments(context)) + .containsAllEntriesOf(readerAssignmentsBeforeFailure); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "the readers came back up, so there should be 2 additional split assignments in the sequence") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + 2); + } + } + + @Test + public void testEnumeratorDoesNotAssignDuplicateSplitsInMetadataUpdate() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + // run all discovery + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + + // trigger metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + + Map splitAssignmentFrequencyMap = new HashMap<>(); + for (SplitsAssignment splitsAssignmentStep : + context.getSplitsAssignmentSequence()) { + for (List assignments : + splitsAssignmentStep.assignment().values()) { + for (DynamicKafkaSourceSplit assignment : assignments) { + splitAssignmentFrequencyMap.put( + assignment.splitId(), + splitAssignmentFrequencyMap.getOrDefault(assignment.splitId(), 0) + + 1); + } + } + } + + assertThat(splitAssignmentFrequencyMap.values()) + .as("all splits should have been assigned once") + .allMatch(count -> count == 1); + } + } + + @Test + public void testInitExceptionNonexistingKafkaCluster() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), + fakeProperties))))); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + @Test + public void testEnumeratorErrorPropagation() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + KafkaStream fakeStream = + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), fakeProperties))); + + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), fakeStream)); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService) { + return createEnumerator(context, kafkaMetadataService, (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + Consumer applyPropertiesConsumer) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + applyPropertiesConsumer); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService, + Consumer applyPropertiesConsumer) { + Properties properties = new Properties(); + applyPropertiesConsumer.accept(properties); + properties.putIfAbsent(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.putIfAbsent( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + return new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + kafkaMetadataService, + context, + OffsetsInitializer.earliest(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory()); + } + + private void mockRegisterReaderAndSendReaderStartupEvent( + MockSplitEnumeratorContext context, + DynamicKafkaSourceEnumerator enumerator, + int reader) { + context.registerReader(new ReaderInfo(reader, "location " + reader)); + enumerator.addReader(reader); + // readers send source event at startup + enumerator.handleSourceEvent(reader, new GetMetadataUpdateEvent()); + } + + private void verifyAllSplitsHaveBeenAssigned( + List> splitsAssignmentSequence, + KafkaStream kafkaStream) { + Map> clusterTopicMap = new HashMap<>(); + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterTopicMap + .computeIfAbsent(entry.getKey(), unused -> new HashSet<>()) + .addAll(entry.getValue().getTopics()); + } + + Set splitsAssigned = + splitsAssignmentSequence.stream() + .flatMap( + splitsAssignment -> + splitsAssignment.assignment().values().stream() + .flatMap(Collection::stream)) + .collect(Collectors.toSet()); + + assertThat(splitsAssignmentSequence).isNotEmpty(); + + Map> clusterToTopicPartition = new HashMap<>(); + for (SplitsAssignment split : splitsAssignmentSequence) { + for (Entry> assignments : + split.assignment().entrySet()) { + for (DynamicKafkaSourceSplit assignment : assignments.getValue()) { + clusterToTopicPartition + .computeIfAbsent(assignment.getKafkaClusterId(), key -> new HashSet<>()) + .add(assignment.getKafkaPartitionSplit().getTopicPartition()); + } + } + } + + assertThat(splitsAssigned) + .hasSize(NUM_SPLITS_PER_CLUSTER * clusterTopicMap.keySet().size()); + + // verify correct clusters + for (String kafkaClusterId : clusterTopicMap.keySet()) { + assertThat(clusterToTopicPartition) + .as("All Kafka clusters must be assigned in the splits.") + .containsKey(kafkaClusterId); + } + + // verify topic partitions + Set assignedTopicPartitionSet = + clusterToTopicPartition.values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + for (Set topics : clusterTopicMap.values()) { + for (String topic : topics) { + Set expectedTopicPartitions = new HashSet<>(); + for (int i = 0; i < NUM_SPLITS_PER_CLUSTER; i++) { + expectedTopicPartitions.add(new TopicPartition(topic, i)); + } + assertThat(assignedTopicPartitionSet) + .as("splits must contain all topics and 2 partitions per topic") + .containsExactlyInAnyOrderElementsOf(expectedTopicPartitions); + } + } + } + + private Map> getReaderAssignments( + MockSplitEnumeratorContext context) { + Map> readerToSplits = new HashMap<>(); + for (SplitsAssignment split : + context.getSplitsAssignmentSequence()) { + for (Entry> assignments : + split.assignment().entrySet()) { + readerToSplits + .computeIfAbsent(assignments.getKey(), key -> new HashSet<>()) + .addAll(assignments.getValue()); + } + } + return readerToSplits; + } + + private static void runAllOneTimeCallables(MockSplitEnumeratorContext context) + throws Throwable { + while (!context.getOneTimeCallables().isEmpty()) { + context.runNextOneTimeCallable(); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState(KafkaStream kafkaStream) + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(Collections.singleton(kafkaStream)), + (properties) -> {})) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned(context.getSplitsAssignmentSequence(), kafkaStream); + + return enumerator.snapshotState(-1); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + return enumerator.snapshotState(-1); + } + } + + private static class TestKafkaEnumContextProxyFactory + implements StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory { + @Override + public StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService) { + return new TestKafkaEnumContextProxy( + kafkaClusterId, + kafkaMetadataService, + (MockSplitEnumeratorContext) enumContext); + } + } + + private static class TestKafkaEnumContextProxy extends StoppableKafkaEnumContextProxy { + + private final SplitEnumeratorContext enumContext; + + public TestKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + MockSplitEnumeratorContext enumContext) { + super(kafkaClusterId, kafkaMetadataService, enumContext); + this.enumContext = enumContext; + } + + /** + * Schedule periodic callables under the coordinator executor, so we can use {@link + * MockSplitEnumeratorContext} to invoke the callable (split assignment) on demand to test + * the integration of KafkaSourceEnumerator. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), + wrapCallAsyncCallableHandler(handler), + initialDelay, + period); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java new file mode 100644 index 000000000..694c95c65 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java @@ -0,0 +1,210 @@ +/* + * 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.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.util.FlinkRuntimeException; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.common.errors.TimeoutException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.fail; + +/** A test for {@link StoppableKafkaEnumContextProxy}. */ +public class StoppableKafkaEnumContextProxyTest { + + private static final String ACTIVE_KAFKA_CLUSTER = "mock-kafka-cluster"; + private static final String INACTIVE_KAFKA_CLUSTER = "mock-inactive-kafka-cluster"; + + private volatile boolean throwExceptionFromMainCallable; + + @BeforeEach + public void beforeEach() { + throwExceptionFromMainCallable = true; + } + + @AfterAll + public static void afterAll() throws Exception {} + + @Test + public void testOneTimeCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + // not running the next periodic callable, since StoppableKafkaEnumContextProxy has the + // callable that periodically schedules the proxy task. So the proxy task (e.g. split + // discovery) is a one time callable in the context of source coordinator. + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableThrowsExceptionOnActiveCluster() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext, ACTIVE_KAFKA_CLUSTER)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + assertThatThrownBy(() -> runNextOneTimeCallableAndCatchException(enumContext)) + .isExactlyInstanceOf(FlinkRuntimeException.class); + assertThat(isCallbackInvoked) + .as("error callback should be invoked since splits have not been assigned yet.") + .isTrue(); + } + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext) { + return createStoppableKafkaEnumContextProxy(enumContext, INACTIVE_KAFKA_CLUSTER); + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext, String contextKafkaCluster) { + + KafkaStream mockStream = + new KafkaStream( + "mock-stream", + ImmutableMap.of( + ACTIVE_KAFKA_CLUSTER, + new ClusterMetadata( + ImmutableSet.of("mock-topic"), new Properties()))); + + return new StoppableKafkaEnumContextProxy( + contextKafkaCluster, + new MockKafkaMetadataService(Collections.singleton(mockStream)), + enumContext); + } + + // this modeled after `KafkaSourceEnumerator` topic partition subscription to throw the same + // exceptions + private void setupKafkaTopicPartitionDiscoveryMockCallable( + StoppableKafkaEnumContextProxy enumContextProxy, AtomicBoolean isCallbackInvoked) { + enumContextProxy.callAsync( + () -> { + if (throwExceptionFromMainCallable) { + // mock Kafka Exception + throw new TimeoutException("Kafka server timed out"); + } else { + // ignore output + return null; + } + }, + (res, t) -> { + isCallbackInvoked.set(true); + if (t != null) { + throw new FlinkRuntimeException(t); + } + }, + 0, + 1000); + } + + private void runNextOneTimeCallableAndCatchException(MockSplitEnumeratorContext enumContext) + throws Throwable { + try { + enumContext.runNextOneTimeCallable(); + fail("TimeoutException should have been thrown"); + } catch (TimeoutException e) { + // catch only Kafka Timeout exceptions since it will be rethrown by + // `MockSplitEnumeratorContext` + AtomicReference errorInMainThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInMainThread"); + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInWorkerThread"); + + assertThat(errorInMainThread.get()) + .as("Should be error in main executor thread for async io") + .isNotNull(); + assertThat(errorInWorkerThread.get()) + .as( + "Should not be error in worker thread that corresponds to source coordinator thread") + .isNull(); + } finally { + // reset MockSplitEnumeratorContext error state + Whitebox.setInternalState( + enumContext, "errorInMainThread", new AtomicReference()); + Whitebox.setInternalState( + enumContext, "errorInWorkerThread", new AtomicReference()); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java new file mode 100644 index 000000000..4e1fcf0ce --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java @@ -0,0 +1,117 @@ +/* + * 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.dynamic.source.metadata; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class SingleClusterTopicMetadataServiceTest { + + private static final String TOPIC0 = "SingleClusterTopicMetadataServiceTest-1"; + private static final String TOPIC1 = "SingleClusterTopicMetadataServiceTest-2"; + + private static KafkaMetadataService kafkaMetadataService; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC0, 3); + DynamicKafkaSourceTestHelper.createTopic(TOPIC1, 3); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + + kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void getAllStreams() { + Set allStreams = kafkaMetadataService.getAllStreams(); + assertThat(allStreams) + .as("stream names should be equal to topic names") + .containsExactlyInAnyOrder( + new KafkaStream( + TOPIC0, + ImmutableMap.of( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC0), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))), + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties())))); + } + + @Test + void describeStreams() { + Map streamMap = + kafkaMetadataService.describeStreams(Collections.singleton(TOPIC1)); + assertThat(streamMap) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of( + TOPIC1, + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))))); + + assertThatCode( + () -> + kafkaMetadataService.describeStreams( + Collections.singleton("unknown-stream"))) + .as("the stream topic cannot be found in kafka and we rethrow") + .hasRootCauseInstanceOf(UnknownTopicOrPartitionException.class); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java new file mode 100644 index 000000000..71feeb8a7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java @@ -0,0 +1,95 @@ +/* + * 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.dynamic.source.metrics; + +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup}. + */ +public class KafkaClusterMetricGroupTest { + + private static MetricListener metricListener; + private static InternalSourceReaderMetricGroup mockInternalSourceReaderMetricGroup; + private static KafkaClusterMetricGroup kafkaClusterMetricGroup; + + @BeforeEach + public void beforeEach() { + metricListener = new MetricListener(); + mockInternalSourceReaderMetricGroup = + InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()); + kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + mockInternalSourceReaderMetricGroup.addGroup( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP), + mockInternalSourceReaderMetricGroup, + "broker-bootstrap-server:443"); + } + + @Test + public void testGetAllVariables() { + // metric variables are wrapped in <...> + assertThat(kafkaClusterMetricGroup.getAllVariables()) + .as( + "variables should contain Kafka cluster info to distinguish multiple sub KafkaSourceReaders") + .containsEntry( + "<" + KafkaClusterMetricGroup.KAFKA_CLUSTER_GROUP_NAME + ">", + "broker-bootstrap-server:443"); + } + + @Test + public void testGetScopeComponents() { + assertThat(kafkaClusterMetricGroup.getScopeComponents()) + .as("scope components contains previously attached scope component") + .contains(DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + } + + @Test + public void testSetPendingRecordsGauge() { + kafkaClusterMetricGroup.setPendingRecordsGauge(() -> 5L); + + // these identifiers should be attached to distinguish distinguish multiple sub + // KafkaSourceReaders + Optional> pendingRecordsGauge = + metricListener.getGauge( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP, + "kafkaCluster", + "broker-bootstrap-server:443", + "pendingRecords"); + + assertThat(pendingRecordsGauge.get().getValue()).isEqualTo(5L); + } + + @Test + public void testGetIOMetricGroup() { + assertThat(kafkaClusterMetricGroup.getIOMetricGroup()) + .isEqualTo(mockInternalSourceReaderMetricGroup.getIOMetricGroup()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java new file mode 100644 index 000000000..aac057167 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java @@ -0,0 +1,316 @@ +/* + * 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.dynamic.source.reader; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.testutils.source.reader.SourceReaderTestBase; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.powermock.reflect.Whitebox; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test for {@link org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader}. + */ +public class DynamicKafkaSourceReaderTest extends SourceReaderTestBase { + private static final String TOPIC = "DynamicKafkaSourceReaderTest"; + + // we are testing two clusters and SourceReaderTestBase expects there to be a total of 10 splits + private static final int NUM_SPLITS_PER_CLUSTER = 5; + + private static String kafkaClusterId0; + private static String kafkaClusterId1; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + kafkaClusterId0 = DynamicKafkaSourceTestHelper.getKafkaClusterId(0); + kafkaClusterId1 = DynamicKafkaSourceTestHelper.getKafkaClusterId(1); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testHandleSourceEventWithRemovedMetadataAtStartup() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + // mock restoring state from Flink runtime + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + + // start reader + reader.start(); + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + // remove cluster 0 + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + List splitsWithoutCluster0 = + splits.stream() + .filter(split -> !split.getKafkaClusterId().equals(kafkaClusterId0)) + .collect(Collectors.toList()); + assertThat(reader.snapshotState(-1)) + .as("The splits should not contain any split related to cluster 0") + .containsExactlyInAnyOrderElementsOf(splitsWithoutCluster0); + } + } + + @Test + void testNoSubReadersInputStatus() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + InputStatus inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.NOTHING_AVAILABLE, + inputStatus, + "nothing available since there are no sub readers created, there could be sub readers created in the future"); + + // notify that this reader will not be assigned anymore splits + reader.notifyNoMoreSplits(); + + inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.END_OF_INPUT, + inputStatus, + "there will not be any more input from this reader since there are no splits"); + } + } + + @Test + void testAvailabilityFutureUpdates() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + CompletableFuture futureAtInit = reader.isAvailable(); + assertThat(reader.isActivelyConsumingSplits()).isFalse(); + assertThat(futureAtInit) + .as("future is not complete at fresh startup since no readers are created") + .isNotDone(); + assertThat(getAvailabilityHelperSize(reader)).isZero(); + + reader.start(); + MetadataUpdateEvent metadata = + DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC); + reader.handleSourceEvents(metadata); + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + CompletableFuture futureAfterSplitAssignment = reader.isAvailable(); + + assertThat(futureAtInit) + .as( + "New future should have been produced since metadata triggers reader creation") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(2); + + // remove cluster 0 + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + CompletableFuture futureAfterRemovingCluster0 = reader.isAvailable(); + assertThat(futureAfterRemovingCluster0) + .as("There should new future since the metadata has changed") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(1); + } + } + + private int getAvailabilityHelperSize(DynamicKafkaSourceReader reader) { + return ((CompletableFuture[]) + Whitebox.getInternalState( + reader.getAvailabilityHelper(), "futuresToCombine")) + .length; + } + + @Test + void testReaderMetadataChangeWhenOneTopicChanges() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + + // splits with offsets + DynamicKafkaSourceSplit cluster0Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(0), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + DynamicKafkaSourceSplit cluster1Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(1), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + reader.addSplits(ImmutableList.of(cluster0Split, cluster1Split)); + + // metadata change with a topic changing + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + Set topicsForCluster1 = + kafkaStream.getClusterMetadataMap().get(kafkaClusterId1).getTopics(); + topicsForCluster1.clear(); + topicsForCluster1.add("new topic"); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + // same split but earlier offset + DynamicKafkaSourceSplit newCluster0Split = + new DynamicKafkaSourceSplit( + kafkaClusterId0, + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + // new split + DynamicKafkaSourceSplit newCluster1Split = + new DynamicKafkaSourceSplit( + kafkaClusterId1, + new KafkaPartitionSplit(new TopicPartition("new topic", 0), 10)); + reader.addSplits(ImmutableList.of(newCluster0Split, newCluster1Split)); + + List assignedSplits = reader.snapshotState(-1); + + assertThat(assignedSplits) + .as( + "The new split for cluster 1 should be assigned and split for cluster 0 should retain offset 10") + .containsExactlyInAnyOrder(cluster0Split, newCluster1Split); + } + } + + @Override + protected SourceReader createReader() { + TestingReaderContext context = new TestingReaderContext(); + return startReader(createReaderWithoutStart(context), context); + } + + private DynamicKafkaSourceReader createReaderWithoutStart( + TestingReaderContext context) { + Properties properties = getRequiredProperties(); + return new DynamicKafkaSourceReader<>( + context, + KafkaRecordDeserializationSchema.valueOnly(IntegerDeserializer.class), + properties); + } + + private SourceReader startReader( + DynamicKafkaSourceReader reader, TestingReaderContext context) { + reader.start(); + assertThat(context.getSentEvents()) + .as("Reader sends GetMetadataUpdateEvent at startup") + .hasSize(1); + reader.handleSourceEvents(DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC)); + return reader; + } + + private static Properties getRequiredProperties() { + Properties properties = new Properties(); + properties.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + properties.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + return properties; + } + + @Override + protected List getSplits( + int numSplits, int numRecordsPerSplit, Boundedness boundedness) { + List splits = new ArrayList<>(); + for (int i = 0; i < numSplits; i++) { + splits.add(getSplit(i, numRecordsPerSplit, boundedness)); + } + return splits; + } + + @Override + protected DynamicKafkaSourceSplit getSplit( + int splitId, int numRecords, Boundedness boundedness) { + long stoppingOffset = + boundedness == Boundedness.BOUNDED + ? NUM_RECORDS_PER_SPLIT + : KafkaPartitionSplit.NO_STOPPING_OFFSET; + + String kafkaClusterId; + int splitIdForCluster = splitId % NUM_SPLITS_PER_CLUSTER; + if (splitId < NUM_SPLITS_PER_CLUSTER) { + kafkaClusterId = "kafka-cluster-0"; + } else { + kafkaClusterId = "kafka-cluster-1"; + } + + return new DynamicKafkaSourceSplit( + kafkaClusterId, + new KafkaPartitionSplit( + new TopicPartition(TOPIC, splitIdForCluster), 0L, stoppingOffset)); + } + + @Override + protected long getNextRecordIndex(DynamicKafkaSourceSplit split) { + return split.getKafkaPartitionSplit().getStartingOffset(); + } + + private Map> splitsToClusterTopicMap(List splits) { + Map> clusterTopicMap = new HashMap<>(); + + for (DynamicKafkaSourceSplit split : splits) { + Set topics = + clusterTopicMap.computeIfAbsent( + split.getKafkaClusterId(), (ignore) -> new HashSet<>()); + topics.add(split.getKafkaPartitionSplit().getTopic()); + } + + return clusterTopicMap; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java new file mode 100644 index 000000000..4125219a8 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java @@ -0,0 +1,47 @@ +/* + * 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.dynamic.source.split; + +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer}. + */ +public class DynamicKafkaSourceSplitSerializerTest { + + @Test + public void testSerde() throws IOException { + DynamicKafkaSourceSplitSerializer serializer = new DynamicKafkaSourceSplitSerializer(); + DynamicKafkaSourceSplit dynamicKafkaSourceSplit = + new DynamicKafkaSourceSplit( + "test-cluster", + new KafkaPartitionSplit(new TopicPartition("test-topic", 3), 1)); + DynamicKafkaSourceSplit dynamicKafkaSourceSplitAfterSerde = + serializer.deserialize(1, serializer.serialize(dynamicKafkaSourceSplit)); + assertEquals(dynamicKafkaSourceSplit, dynamicKafkaSourceSplitAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java new file mode 100644 index 000000000..e9bc77e83 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java @@ -0,0 +1,263 @@ +/* + * 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.testutils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** A external context for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContext implements DataStreamSourceExternalContext { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceExternalContext.class); + private static final int NUM_TEST_RECORDS_PER_SPLIT = 10; + private static final int NUM_PARTITIONS = 1; + + private static final Pattern STREAM_ID_PATTERN = Pattern.compile("stream-[0-9]+"); + private final List connectorJarPaths; + private final Set kafkaStreams = new HashSet<>(); + private final Map clusterPropertiesMap; + private final List splitDataWriters = new ArrayList<>(); + + // add random suffix to alleviate race conditions with Kafka deleting topics + private final long randomTopicSuffix; + + public DynamicKafkaSourceExternalContext( + List bootstrapServerList, List connectorJarPaths) { + this.connectorJarPaths = connectorJarPaths; + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(0)); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(1)); + + this.clusterPropertiesMap = + ImmutableMap.of( + "cluster0", propertiesForCluster0, "cluster1", propertiesForCluster1); + this.randomTopicSuffix = ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) + throws UnsupportedOperationException { + final DynamicKafkaSourceBuilder builder = DynamicKafkaSource.builder(); + + builder.setStreamPattern(STREAM_ID_PATTERN) + .setKafkaMetadataService(new MockKafkaMetadataService(kafkaStreams)) + .setGroupId("DynamicKafkaSourceExternalContext") + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); + + if (sourceSettings.getBoundedness().equals(Boundedness.BOUNDED)) { + builder.setBounded(OffsetsInitializer.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + int suffix = splitDataWriters.size(); + List> clusterTopics = setupSplits(String.valueOf(suffix)); + SplitDataWriter splitDataWriter = new SplitDataWriter(clusterPropertiesMap, clusterTopics); + this.splitDataWriters.add(splitDataWriter); + return splitDataWriter; + } + + private List> setupSplits(String suffix) { + KafkaStream kafkaStream = getKafkaStream(suffix + randomTopicSuffix); + logger.info("Setting up splits for {}", kafkaStream); + List> clusterTopics = + kafkaStream.getClusterMetadataMap().entrySet().stream() + .flatMap( + entry -> + entry.getValue().getTopics().stream() + .map(topic -> Tuple2.of(entry.getKey(), topic))) + .collect(Collectors.toList()); + + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + KafkaTestEnvironmentImpl.createNewTopic( + topic, NUM_PARTITIONS, 1, clusterPropertiesMap.get(cluster)); + } + + kafkaStreams.add(kafkaStream); + return clusterTopics; + } + + private KafkaStream getKafkaStream(String suffix) { + return new KafkaStream( + "stream-" + suffix, + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0-" + suffix, "topic1-" + suffix), + clusterPropertiesMap.get("cluster0")), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2-" + suffix, "topic3-" + suffix), + clusterPropertiesMap.get("cluster1")))); + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + return IntStream.range(0, NUM_TEST_RECORDS_PER_SPLIT * NUM_PARTITIONS) + .boxed() + .map(num -> Integer.toString(num)) + .collect(Collectors.toList()); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + + @Override + public List getConnectorJarPaths() { + return connectorJarPaths; + } + + @Override + public void close() throws Exception { + // need to clear topics + Map> clusterTopics = new HashMap<>(); + for (SplitDataWriter splitDataWriter : splitDataWriters) { + for (Tuple2 clusterTopic : splitDataWriter.getClusterTopics()) { + clusterTopics + .computeIfAbsent(clusterTopic.f0, unused -> new ArrayList<>()) + .add(clusterTopic.f1); + } + } + for (Map.Entry> entry : clusterTopics.entrySet()) { + String cluster = entry.getKey(); + List topics = entry.getValue(); + try (AdminClient adminClient = AdminClient.create(clusterPropertiesMap.get(cluster))) { + adminClient.deleteTopics(topics).all().get(); + CommonTestUtils.waitUtil( + () -> { + try { + return adminClient.listTopics().listings().get().stream() + .map(TopicListing::name) + .noneMatch(topics::contains); + } catch (Exception e) { + logger.warn("Exception caught when listing Kafka topics", e); + return false; + } + }, + Duration.ofSeconds(30), + String.format("Topics %s were not deleted within timeout", topics)); + } + + logger.info("topics {} are deleted from {}", topics, cluster); + } + } + + private static class SplitDataWriter implements ExternalSystemSplitDataWriter { + private final Map clusterPropertiesMap; + private final List> clusterTopics; + + public SplitDataWriter( + Map clusterPropertiesMap, + List> clusterTopics) { + this.clusterPropertiesMap = clusterPropertiesMap; + this.clusterTopics = clusterTopics; + } + + @Override + public void writeRecords(List records) { + int counter = 0; + try { + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + List> producerRecords = new ArrayList<>(); + for (int j = 0; j < NUM_PARTITIONS; j++) { + for (int k = 0; k < NUM_TEST_RECORDS_PER_SPLIT; k++) { + if (records.size() <= counter) { + break; + } + + producerRecords.add( + new ProducerRecord<>(topic, j, null, records.get(counter++))); + } + } + + logger.info("Writing producer records: {}", producerRecords); + + DynamicKafkaSourceTestHelper.produceToKafka( + clusterPropertiesMap.get(cluster), + producerRecords, + StringSerializer.class, + StringSerializer.class); + } + } catch (Throwable e) { + throw new RuntimeException("Failed to produce test data", e); + } + } + + @Override + public void close() throws Exception {} + + public List> getClusterTopics() { + return clusterTopics; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java new file mode 100644 index 000000000..71798e185 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java @@ -0,0 +1,63 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.testframe.external.ExternalContextFactory; + +import com.google.common.collect.ImmutableList; +import org.testcontainers.containers.KafkaContainer; + +import java.net.URL; +import java.util.List; +import java.util.stream.Collectors; + +/** A external context factory for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContextFactory + implements ExternalContextFactory { + + private final KafkaContainer kafkaContainer0; + private final KafkaContainer kafkaContainer1; + private final List connectorJars; + + public DynamicKafkaSourceExternalContextFactory( + KafkaContainer kafkaContainer0, + KafkaContainer kafkaContainer1, + List connectorJars) { + this.kafkaContainer0 = kafkaContainer0; + this.kafkaContainer1 = kafkaContainer1; + this.connectorJars = connectorJars; + } + + @Override + public DynamicKafkaSourceExternalContext createExternalContext(String testName) { + return new DynamicKafkaSourceExternalContext( + ImmutableList.of( + getBootstrapServers(kafkaContainer0), getBootstrapServers(kafkaContainer1)), + connectorJars); + } + + private static String getBootstrapServers(KafkaContainer kafkaContainer) { + final String internalEndpoints = + kafkaContainer.getNetworkAliases().stream() + .map(host -> String.join(":", host, Integer.toString(9092))) + .collect(Collectors.joining(",")); + return String.join(",", kafkaContainer.getBootstrapServers(), internalEndpoints); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java new file mode 100644 index 000000000..18854cf90 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java @@ -0,0 +1,93 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** A mock in-memory implementation of {@link KafkaMetadataService}. */ +public class MockKafkaMetadataService implements KafkaMetadataService { + + private Set kafkaStreams; + private Set kafkaClusterIds; + private boolean throwException = false; + + public MockKafkaMetadataService(boolean throwException) { + this.throwException = throwException; + } + + public MockKafkaMetadataService(Set kafkaStreams) { + setKafkaStreams(kafkaStreams); + } + + public void setKafkaStreams(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + this.kafkaClusterIds = + kafkaStreams.stream() + .flatMap( + kafkaStream -> + kafkaStream.getClusterMetadataMap().keySet().stream()) + .collect(Collectors.toSet()); + } + + public void setThrowException(boolean throwException) { + this.throwException = throwException; + } + + private void checkAndThrowException() { + if (throwException) { + throw new RuntimeException("Mock exception"); + } + } + + @Override + public Set getAllStreams() { + checkAndThrowException(); + return kafkaStreams; + } + + @Override + public Map describeStreams(Collection streamIds) { + checkAndThrowException(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (KafkaStream stream : getAllStreams()) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + @Override + public boolean isClusterActive(String kafkaClusterId) { + checkAndThrowException(); + return kafkaClusterIds.contains(kafkaClusterId); + } + + @Override + public void close() throws Exception {} +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java new file mode 100644 index 000000000..7085a480a --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java @@ -0,0 +1,62 @@ +/* + * 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.testutils; + +import org.apache.flink.util.DockerImageVersions; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Wraps 2 Kafka containers into one for test utilities that only accept one container. */ +public class TwoKafkaContainers extends GenericContainer { + private final KafkaContainer kafka0; + private final KafkaContainer kafka1; + + public TwoKafkaContainers() { + DockerImageName dockerImageName = DockerImageName.parse(DockerImageVersions.KAFKA); + this.kafka0 = new KafkaContainer(dockerImageName); + this.kafka1 = new KafkaContainer(dockerImageName); + } + + @Override + public boolean isRunning() { + return kafka0.isRunning() && kafka1.isRunning(); + } + + @Override + public void start() { + kafka0.start(); + kafka1.start(); + } + + @Override + public void stop() { + kafka0.stop(); + kafka1.stop(); + } + + public KafkaContainer getKafka0() { + return kafka0; + } + + public KafkaContainer getKafka1() { + return kafka1; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java new file mode 100644 index 000000000..32839f379 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java @@ -0,0 +1,361 @@ +/* + * 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.testutils; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.DumperOptions; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.TypeDescription; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.Yaml; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.Constructor; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Node; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.SequenceNode; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Tag; +import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.CommonClientConfigs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Reads metadata from yaml file and lazily refreshes periodically. This implementation assumes that + * specified topics exist in the clusters that are contained in the yaml metadata. Therefore, topic + * is used as the stream name. This is designed to integrate with K8s configmap and cluster + * migration. + * + *

Files must be of the form: + * + *

{@code
+ * - streamId: stream0
+ *   clusterMetadataList:
+ *     - clusterId: cluster0
+ *       bootstrapServers: bootstrap-server-0:443
+ *       topics:
+ *         - topic0
+ *         - topic1
+ *     - clusterId: cluster1
+ *       bootstrapServers: bootstrap-server-1:443
+ *       topics:
+ *         - topic2
+ *         - topic3
+ * - streamId: stream1
+ *   clusterMetadataList:
+ *     - clusterId: cluster2
+ *       bootstrapServers: bootstrap-server-2:443
+ *       topics:
+ *         - topic4
+ *         - topic5
+ * }
+ * + *

Typically, usage will look like: first consuming from one cluster, second adding new cluster + * and consuming from both clusters, and third consuming from only from the new cluster after all + * data from the old cluster has been read. + */ +public class YamlFileMetadataService implements KafkaMetadataService { + private static final Logger logger = LoggerFactory.getLogger(YamlFileMetadataService.class); + private final String metadataFilePath; + private final Duration refreshInterval; + private Instant lastRefresh; + // current metadata should be accessed from #getAllStreams() + private transient Set streamMetadata; + private transient Yaml yaml; + + /** + * Constructs a metadata service based on cluster information stored in a file. + * + * @param metadataFilePath location of the metadata file + * @param metadataTtl ttl of metadata that controls how often to refresh + */ + public YamlFileMetadataService(String metadataFilePath, Duration metadataTtl) { + this.metadataFilePath = metadataFilePath; + this.refreshInterval = metadataTtl; + this.lastRefresh = Instant.MIN; + } + + /** + * {@inheritDoc} + * + *

This obtains the all stream metadata and enforces the ttl configuration on the metadata. + */ + @Override + public Set getAllStreams() { + refreshIfNeeded(); + return streamMetadata; + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Set streams = getAllStreams(); + for (KafkaStream stream : streams) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return getAllStreams().stream() + .flatMap(kafkaStream -> kafkaStream.getClusterMetadataMap().keySet().stream()) + .anyMatch(cluster -> cluster.equals(kafkaClusterId)); + } + + /** {@inheritDoc} */ + @Override + public void close() throws Exception {} + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param streamMetadata list of {@link StreamMetadata} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYaml(List streamMetadata, File metadataFile) + throws IOException { + logger.debug("Writing stream infos to file: {}", streamMetadata); + Yaml yaml = initYamlParser(); + FileWriter fileWriter = new FileWriter(metadataFile, false); + yaml.dump(streamMetadata, fileWriter); + fileWriter.close(); + } + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param kafkaStreams list of {@link KafkaStream} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYamlFromKafkaStreams(List kafkaStreams, File metadataFile) + throws IOException { + saveToYaml( + kafkaStreams.stream() + .map(YamlFileMetadataService::convertToStreamMetadata) + .collect(Collectors.toList()), + metadataFile); + } + + private static StreamMetadata convertToStreamMetadata(KafkaStream kafkaStream) { + return new StreamMetadata( + kafkaStream.getStreamId(), + kafkaStream.getClusterMetadataMap().entrySet().stream() + .map( + entry -> + new StreamMetadata.ClusterMetadata( + entry.getKey(), + entry.getValue() + .getProperties() + .getProperty( + CommonClientConfigs + .BOOTSTRAP_SERVERS_CONFIG), + new ArrayList<>(entry.getValue().getTopics()))) + .collect(Collectors.toList())); + } + + private void refreshIfNeeded() { + Instant now = Instant.now(); + try { + if (now.isAfter(lastRefresh.plus(refreshInterval.toMillis(), ChronoUnit.MILLIS))) { + streamMetadata = parseFile(); + lastRefresh = now; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @VisibleForTesting + Set parseFile() throws IOException { + if (yaml == null) { + yaml = initYamlParser(); + } + + List streamMetadataList = + yaml.load(Files.newInputStream(Paths.get(metadataFilePath))); + if (logger.isDebugEnabled()) { + logger.debug( + "Input stream of metadata file has size: {}", + Files.newInputStream(Paths.get(metadataFilePath)).available()); + } + Set kafkaStreams = new HashSet<>(); + + for (StreamMetadata streamMetadata : streamMetadataList) { + Map clusterMetadataMap = new HashMap<>(); + + for (StreamMetadata.ClusterMetadata clusterMetadata : + streamMetadata.getClusterMetadataList()) { + final String kafkaClusterId; + if (clusterMetadata.getClusterId() != null) { + kafkaClusterId = clusterMetadata.getClusterId(); + } else { + kafkaClusterId = clusterMetadata.getBootstrapServers(); + } + + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + clusterMetadata.getBootstrapServers()); + clusterMetadataMap.put( + kafkaClusterId, + new ClusterMetadata( + new HashSet<>(clusterMetadata.getTopics()), properties)); + } + + kafkaStreams.add(new KafkaStream(streamMetadata.getStreamId(), clusterMetadataMap)); + } + + logger.debug("From {} loaded metadata: {}", metadataFilePath, kafkaStreams); + return kafkaStreams; + } + + private static Yaml initYamlParser() { + Representer representer = new Representer(); + representer.addClassTag(StreamMetadata.class, Tag.MAP); + TypeDescription typeDescription = new TypeDescription(StreamMetadata.class); + representer.addTypeDescription(typeDescription); + representer.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK); + return new Yaml(new ListConstructor<>(StreamMetadata.class), representer); + } + + /** A custom constructor is required to read yaml lists at the root. */ + private static class ListConstructor extends Constructor { + private final Class clazz; + + public ListConstructor(final Class clazz) { + this.clazz = clazz; + } + + @Override + protected Object constructObject(final Node node) { + if (node instanceof SequenceNode && isRootNode(node)) { + ((SequenceNode) node).setListType(clazz); + } + return super.constructObject(node); + } + + private boolean isRootNode(final Node node) { + return node.getStartMark().getIndex() == 0; + } + } + + /** Internal class for snake yaml parsing. A mutable, no arg, public class is required. */ + public static class StreamMetadata { + + private String streamId; + private List clusterMetadataList; + + public StreamMetadata() {} + + public StreamMetadata(String streamId, List clusterMetadataList) { + this.streamId = streamId; + this.clusterMetadataList = clusterMetadataList; + } + + public String getStreamId() { + return streamId; + } + + public void setStreamId(String streamId) { + this.streamId = streamId; + } + + public List getClusterMetadataList() { + return clusterMetadataList; + } + + public void setClusterMetadataList(List clusterMetadata) { + this.clusterMetadataList = clusterMetadata; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataList", clusterMetadataList) + .toString(); + } + + /** Information to connect to a particular cluster. */ + public static class ClusterMetadata { + private String clusterId; + private String bootstrapServers; + private List topics; + + public ClusterMetadata() {} + + public ClusterMetadata(String clusterId, String bootstrapServers, List topics) { + this.clusterId = clusterId; + this.bootstrapServers = bootstrapServers; + this.topics = topics; + } + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public List getTopics() { + return topics; + } + + public void setTopics(List topics) { + this.topics = topics; + } + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java new file mode 100644 index 000000000..f0012d181 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java @@ -0,0 +1,79 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.Resources; +import org.apache.kafka.clients.CommonClientConfigs; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** A test class for {@link YamlFileMetadataService}. */ +public class YamlFileMetadataServiceTest { + + @Test + public void testParseFile() throws IOException { + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService( + Resources.getResource("stream-metadata.yaml").getPath(), Duration.ZERO); + Set kafkaStreams = yamlFileMetadataService.parseFile(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-0:443"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-1:443"); + Properties propertiesForCluster2 = new Properties(); + propertiesForCluster2.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-2:443"); + + assertThat(kafkaStreams) + .containsExactlyInAnyOrderElementsOf( + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster2", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster2))))); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java new file mode 100644 index 000000000..c6ecfd061 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java @@ -0,0 +1,229 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** Brings up multiple kafka clusters and provides utilities to setup test data. */ +public class DynamicKafkaSourceTestHelper extends KafkaTestBase { + public static final int NUM_KAFKA_CLUSTERS = 2; + + public static void setup() throws Throwable { + setNumKafkaClusters(NUM_KAFKA_CLUSTERS); + prepare(); + } + + public static void tearDown() throws Exception { + shutDownServices(); + } + + public static KafkaClusterTestEnvMetadata getKafkaClusterTestEnvMetadata(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx); + } + + public static MetadataUpdateEvent getMetadataUpdateEvent(String topic) { + return new MetadataUpdateEvent(Collections.singleton(getKafkaStream(topic))); + } + + public static String getKafkaClusterId(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx).getKafkaClusterId(); + } + + /** Stream is a topic across multiple clusters. */ + public static KafkaStream getKafkaStream(String topic) { + Map clusterMetadataMap = new HashMap<>(); + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + getKafkaClusterTestEnvMetadata(i); + + Set topics = new HashSet<>(); + topics.add(topic); + + ClusterMetadata clusterMetadata = + new ClusterMetadata( + topics, kafkaClusterTestEnvMetadata.getStandardProperties()); + clusterMetadataMap.put( + kafkaClusterTestEnvMetadata.getKafkaClusterId(), clusterMetadata); + } + + return new KafkaStream(topic, clusterMetadataMap); + } + + public static void createTopic(String topic, int numPartitions, int replicationFactor) { + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + createTopic(i, topic, numPartitions, replicationFactor); + } + } + + public static void createTopic(String topic, int numPartitions) { + createTopic(topic, numPartitions, 1); + } + + public static void createTopic(int kafkaClusterIdx, String topic, int numPartitions) { + createTopic(kafkaClusterIdx, topic, numPartitions, 1); + } + + private static void createTopic( + int kafkaClusterIdx, String topic, int numPartitions, int replicationFactor) { + kafkaClusters + .get(kafkaClusterIdx) + .getKafkaTestEnvironment() + .createTestTopic(topic, numPartitions, replicationFactor); + } + + /** Produces [0, numPartitions*numRecordsPerSplit) range of records to the specified topic. */ + public static List> produceToKafka( + String topic, int numPartitions, int numRecordsPerSplit) throws Throwable { + List> records = new ArrayList<>(); + + int counter = 0; + for (int kafkaClusterIdx = 0; kafkaClusterIdx < NUM_KAFKA_CLUSTERS; kafkaClusterIdx++) { + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes( + StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + records.addAll(recordsForCluster); + } + + return records; + } + + /** + * Produces [recordValueStartingOffset, recordValueStartingOffset + + * numPartitions*numRecordsPerSplit) range of records to the specified topic and cluster. + */ + public static int produceToKafka( + int kafkaClusterIdx, + String topic, + int numPartitions, + int numRecordsPerSplit, + int recordValueStartingOffset) + throws Throwable { + int counter = recordValueStartingOffset; + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes(StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + + return counter; + } + + public static void produceToKafka( + int kafkaClusterIdx, Collection> records) + throws Throwable { + produceToKafka(kafkaClusterIdx, records, StringSerializer.class, IntegerSerializer.class); + } + + public static void produceToKafka( + int id, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + produceToKafka( + kafkaClusters.get(id).getStandardProperties(), + records, + keySerializerClass, + valueSerializerClass); + } + + public static void produceToKafka( + Properties clusterProperties, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + Properties props = new Properties(); + props.putAll(clusterProperties); + props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName()); + props.setProperty( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); + + AtomicReference sendingError = new AtomicReference<>(); + Callback callback = + (metadata, exception) -> { + if (exception != null) { + if (!sendingError.compareAndSet(null, exception)) { + sendingError.get().addSuppressed(exception); + } + } + }; + try (KafkaProducer producer = new KafkaProducer<>(props)) { + for (ProducerRecord record : records) { + producer.send(record, callback).get(); + } + } + if (sendingError.get() != null) { + throw sendingError.get(); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index b4fd54ab2..deafb7d6d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -32,6 +32,7 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; +import com.google.common.base.MoreObjects; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -83,6 +84,7 @@ public abstract class KafkaTestBase extends TestLogger { public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); public static final int NUMBER_OF_KAFKA_SERVERS = 1; + private static int numKafkaClusters = 1; public static String brokerConnectionStrings; @@ -92,6 +94,8 @@ public abstract class KafkaTestBase extends TestLogger { public static KafkaTestEnvironment kafkaServer; + public static List kafkaClusters = new ArrayList<>(); + @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); public static Properties secureProps = new Properties(); @@ -108,7 +112,7 @@ public static void prepare() throws Exception { LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); - startClusters(false); + startClusters(false, numKafkaClusters); } @AfterClass @@ -140,11 +144,21 @@ public static void startClusters() throws Exception { KafkaTestEnvironment.createConfig().setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS)); } - public static void startClusters(boolean secureMode) throws Exception { + public static void startClusters(boolean secureMode, int numKafkaClusters) throws Exception { startClusters( - KafkaTestEnvironment.createConfig() - .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS) - .setSecureMode(secureMode)); + KafkaTestEnvironment.createConfig().setSecureMode(secureMode), numKafkaClusters); + } + + public static void startClusters( + KafkaTestEnvironment.Config environmentConfig, int numKafkaClusters) throws Exception { + for (int i = 0; i < numKafkaClusters; i++) { + startClusters(environmentConfig); + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + new KafkaClusterTestEnvMetadata( + i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); + kafkaClusters.add(kafkaClusterTestEnvMetadata); + LOG.info("Created Kafka cluster with configuration: {}", kafkaClusterTestEnvMetadata); + } } public static void startClusters(KafkaTestEnvironment.Config environmentConfig) @@ -183,6 +197,13 @@ public static void shutdownClusters() throws Exception { if (kafkaServer != null) { kafkaServer.shutdown(); } + + if (kafkaClusters != null && !kafkaClusters.isEmpty()) { + for (KafkaClusterTestEnvMetadata value : kafkaClusters) { + value.getKafkaTestEnvironment().shutdown(); + } + kafkaClusters.clear(); + } } // ------------------------------------------------------------------------ @@ -338,4 +359,62 @@ private String formatElements(List elements) { return String.format("elements: <%s>", elements); } } + + public static void setNumKafkaClusters(int size) { + numKafkaClusters = size; + } + + /** Metadata generated by this test utility. */ + public static class KafkaClusterTestEnvMetadata { + + private final String kafkaClusterId; + private final KafkaTestEnvironment kafkaTestEnvironment; + private final Properties standardProperties; + private final String brokerConnectionStrings; + private final Properties secureProperties; + + private KafkaClusterTestEnvMetadata( + int kafkaClusterIdx, + KafkaTestEnvironment kafkaTestEnvironment, + Properties standardProperties, + String brokerConnectionStrings, + Properties secureProperties) { + this.kafkaClusterId = "kafka-cluster-" + kafkaClusterIdx; + this.kafkaTestEnvironment = kafkaTestEnvironment; + this.standardProperties = standardProperties; + this.brokerConnectionStrings = brokerConnectionStrings; + this.secureProperties = secureProperties; + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaTestEnvironment getKafkaTestEnvironment() { + return kafkaTestEnvironment; + } + + public Properties getStandardProperties() { + return standardProperties; + } + + public String getBrokerConnectionStrings() { + return brokerConnectionStrings; + } + + public Properties getSecureProperties() { + return secureProperties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaTestEnvironment", kafkaTestEnvironment) + .add("standardProperties", standardProperties) + .add("brokerConnectionStrings", brokerConnectionStrings) + .add("secureProperties", secureProperties) + .toString(); + } + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index ee38e8501..6687cd525 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -39,6 +39,8 @@ public abstract class KafkaTestEnvironment { /** Configuration class for {@link KafkaTestEnvironment}. */ public static class Config { + + private int numKafkaClusters = 1; private int kafkaServersNumber = 1; private Properties kafkaServerProperties = null; private boolean secureMode = false; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index c5bc3b003..b776cad22 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -169,8 +169,13 @@ private void tryDelete(AdminClient adminClient, String topic) throws Exception { @Override public void createTestTopic( String topic, int numberOfPartitions, int replicationFactor, Properties properties) { + createNewTopic(topic, numberOfPartitions, replicationFactor, getStandardProperties()); + } + + public static void createNewTopic( + String topic, int numberOfPartitions, int replicationFactor, Properties properties) { LOG.info("Creating topic {}", topic); - try (AdminClient adminClient = AdminClient.create(getStandardProperties())) { + try (AdminClient adminClient = AdminClient.create(properties)) { NewTopic topicObj = new NewTopic(topic, numberOfPartitions, (short) replicationFactor); adminClient.createTopics(Collections.singleton(topicObj)).all().get(); CommonTestUtils.waitUtil( diff --git a/flink-connector-kafka/src/test/resources/stream-metadata.yaml b/flink-connector-kafka/src/test/resources/stream-metadata.yaml new file mode 100644 index 000000000..9502351b5 --- /dev/null +++ b/flink-connector-kafka/src/test/resources/stream-metadata.yaml @@ -0,0 +1,19 @@ +- streamId: stream0 + clusterMetadataList: + - clusterId: cluster0 + bootstrapServers: bootstrap-server-0:443 + topics: + - topic0 + - topic1 + - clusterId: cluster1 + bootstrapServers: bootstrap-server-1:443 + topics: + - topic2 + - topic3 +- streamId: stream1 + clusterMetadataList: + - clusterId: cluster2 + bootstrapServers: bootstrap-server-2:443 + topics: + - topic4 + - topic5 diff --git a/pom.xml b/pom.xml index ff8efac36..d73d2ebd2 100644 --- a/pom.xml +++ b/pom.xml @@ -69,6 +69,7 @@ under the License. 2.12.7 1.1.10.5 1.11.3 + 32.1.2-jre false 1.17.0 @@ -263,6 +264,12 @@ under the License. 3.3.2 + + com.google.guava + guava + ${guava.version} + + org.apache.flink diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 2048fd186..7d1f9fd71 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -224,10 +224,6 @@ This file is based on the checkstyle file of Apache Beam. - - - -