[WIP] Remove all Kafka connector code from main repo
diff --git a/flink-architecture-tests/pom.xml b/flink-architecture-tests/pom.xml
index 0ce107c..aeed3bc 100644
--- a/flink-architecture-tests/pom.xml
+++ b/flink-architecture-tests/pom.xml
@@ -177,7 +177,7 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
<scope>test</scope>
</dependency>
diff --git a/flink-connectors/flink-connector-kafka/archunit-violations/8248e31e-b706-4471-bb65-4c927c65fccf b/flink-connectors/flink-connector-kafka/archunit-violations/8248e31e-b706-4471-bb65-4c927c65fccf
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-kafka/archunit-violations/8248e31e-b706-4471-bb65-4c927c65fccf
+++ /dev/null
diff --git a/flink-connectors/flink-connector-kafka/archunit-violations/97dda445-f6bc-43e2-8106-5876ca0cd052 b/flink-connectors/flink-connector-kafka/archunit-violations/97dda445-f6bc-43e2-8106-5876ca0cd052
deleted file mode 100644
index 50c10b7..0000000
--- a/flink-connectors/flink-connector-kafka/archunit-violations/97dda445-f6bc-43e2-8106-5876ca0cd052
+++ /dev/null
@@ -1,78 +0,0 @@
-org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.kafka.sink.KafkaSinkITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.connector.kafka.source.KafkaSourceLegacyITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.KafkaITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.KafkaProducerAtLeastOnceITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.KafkaProducerExactlyOnceITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleExactlyOnceITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
-org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-kafka/archunit-violations/stored.rules b/flink-connectors/flink-connector-kafka/archunit-violations/stored.rules
deleted file mode 100644
index a9c54e2..0000000
--- a/flink-connectors/flink-connector-kafka/archunit-violations/stored.rules
+++ /dev/null
@@ -1,4 +0,0 @@
-#
-#Tue Feb 22 12:18:13 CET 2022
-Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=8248e31e-b706-4471-bb65-4c927c65fccf
-ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=97dda445-f6bc-43e2-8106-5876ca0cd052
diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml
deleted file mode 100644
index 607af88..0000000
--- a/flink-connectors/flink-connector-kafka/pom.xml
+++ /dev/null
@@ -1,332 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>flink-connectors</artifactId>
- <groupId>org.apache.flink</groupId>
- <version>1.18-SNAPSHOT</version>
- </parent>
-
- <artifactId>flink-connector-kafka</artifactId>
- <name>Flink : Connectors : Kafka</name>
-
- <packaging>jar</packaging>
-
- <properties>
- <kafka.version>3.2.3</kafka.version>
- </properties>
-
- <dependencies>
-
- <!-- Core -->
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
-
- <!-- Connectors -->
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-base</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <!-- Table ecosystem -->
-
- <!-- Projects depending on this project won't depend on flink-table-*. -->
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-table-api-java-bridge</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- <optional>true</optional>
- </dependency>
-
- <!-- Kafka -->
-
- <!-- Add Kafka 2.x as a dependency -->
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka-clients</artifactId>
- <version>${kafka.version}</version>
- </dependency>
-
- <!-- Tests -->
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-core</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
-
-
- <dependency>
- <!-- include 2.0 server for tests -->
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka_${scala.binary.version}</artifactId>
- <version>${kafka.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </exclusion>
- </exclusions>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.zookeeper</groupId>
- <artifactId>zookeeper</artifactId>
- <version>${zookeeper.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.testcontainers</groupId>
- <artifactId>kafka</artifactId>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.testcontainers</groupId>
- <artifactId>junit-jupiter</artifactId>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-test-utils</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-test-utils</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-runtime</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-base</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-metrics-jmx</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <!-- Kafka table factory testing -->
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-table-common</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-table-test-utils</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-table-api-scala-bridge_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
-
- <!-- Kafka SQL IT test with formats -->
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-json</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-avro</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-avro-confluent-registry</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-csv</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>fink-migration-test-utils</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <!-- ArchUit test dependencies -->
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-architecture-tests-test</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <executions>
- <execution>
- <goals>
- <goal>test-jar</goal>
- </goals>
- <configuration>
- <includes>
- <include>**/KafkaTestEnvironment*</include>
- <include>**/testutils/*</include>
- <include>META-INF/LICENSE</include>
- <include>META-INF/NOTICE</include>
- </includes>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-source-plugin</artifactId>
- <executions>
- <execution>
- <id>attach-test-sources</id>
- <goals>
- <goal>test-jar-no-fork</goal>
- </goals>
- <configuration>
- <archive>
- <!-- Globally exclude maven metadata, because it may accidentally bundle files we don't intend to -->
- <addMavenDescriptor>false</addMavenDescriptor>
- </archive>
- <includes>
- <include>**/KafkaTestEnvironment*</include>
- <include>**/testutils/*</include>
- <include>META-INF/LICENSE</include>
- <include>META-INF/NOTICE</include>
- </includes>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-
- <profiles>
- <profile>
- <id>generate-migration-test-data</id>
- <build>
- <plugins>
- <plugin>
- <artifactId>maven-antrun-plugin</artifactId>
- <executions>
- <execution>
- <id>generate-migration-test-data</id>
- <phase>package</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <target>
- <condition property="optional.classes" value="--classes '${generate.classes}'"
- else="">
- <isset property="generate.classes"/>
- </condition>
- <condition property="optional.prefixes"
- value="--prefixes '${generate.prefixes}'" else="">
- <isset property="generate.prefixes"/>
- </condition>
- <java classname="org.apache.flink.test.migration.MigrationTestsSnapshotGenerator"
- fork="true" failonerror="true" dir="${project.basedir}">
- <classpath refid="maven.test.classpath"/>
- <arg value="--dir"/>
- <arg line="${project.basedir}"/>
- <arg value="--version"/>
- <arg value="${generate.version}"/>
- <arg line="${optional.classes}"/>
- <arg line="${optional.prefixes}"/>
- </java>
- </target>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
- </profile>
- </profiles>
-</project>
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/MetricUtil.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/MetricUtil.java
deleted file mode 100644
index 330315f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/MetricUtil.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.metrics.Counter;
-
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-
-import java.util.Map;
-import java.util.function.Predicate;
-
-/** Collection of methods to interact with Kafka's client metric system. */
-@Internal
-public class MetricUtil {
-
- /**
- * Tries to find the Kafka {@link Metric} in the provided metrics.
- *
- * @return {@link Metric} which exposes continuous updates
- * @throws IllegalStateException if the metric is not part of the provided metrics
- */
- public static Metric getKafkaMetric(
- Map<MetricName, ? extends Metric> metrics, String metricGroup, String metricName) {
- return getKafkaMetric(
- metrics,
- e ->
- e.getKey().group().equals(metricGroup)
- && e.getKey().name().equals(metricName));
- }
-
- /**
- * Tries to find the Kafka {@link Metric} in the provided metrics matching a given filter.
- *
- * @return {@link Metric} which exposes continuous updates
- * @throws IllegalStateException if no metric matches the given filter
- */
- public static Metric getKafkaMetric(
- Map<MetricName, ? extends Metric> metrics,
- Predicate<Map.Entry<MetricName, ? extends Metric>> filter) {
- return metrics.entrySet().stream()
- .filter(filter)
- .map(Map.Entry::getValue)
- .findFirst()
- .orElseThrow(
- () ->
- new IllegalStateException(
- "Cannot find Kafka metric matching current filter."));
- }
-
- /**
- * Ensures that the counter has the same value as the given Kafka metric.
- *
- * <p>Do not use this method for every record because {@link Metric#metricValue()} is an
- * expensive operation.
- *
- * @param from Kafka's {@link Metric} to query
- * @param to {@link Counter} to write the value to
- */
- public static void sync(Metric from, Counter to) {
- to.inc(((Number) from.metricValue()).longValue() - to.getCount());
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/DefaultKafkaSinkContext.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/DefaultKafkaSinkContext.java
deleted file mode 100644
index bec5ab6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/DefaultKafkaSinkContext.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.common.PartitionInfo;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Context providing information to assist constructing a {@link
- * org.apache.kafka.clients.producer.ProducerRecord}.
- */
-public class DefaultKafkaSinkContext implements KafkaRecordSerializationSchema.KafkaSinkContext {
-
- private final int subtaskId;
- private final int numberOfParallelInstances;
- private final Properties kafkaProducerConfig;
-
- private final Map<String, int[]> cachedPartitions = new HashMap<>();
-
- public DefaultKafkaSinkContext(
- int subtaskId, int numberOfParallelInstances, Properties kafkaProducerConfig) {
- this.subtaskId = subtaskId;
- this.numberOfParallelInstances = numberOfParallelInstances;
- this.kafkaProducerConfig = kafkaProducerConfig;
- }
-
- @Override
- public int getParallelInstanceId() {
- return subtaskId;
- }
-
- @Override
- public int getNumberOfParallelInstances() {
- return numberOfParallelInstances;
- }
-
- @Override
- public int[] getPartitionsForTopic(String topic) {
- return cachedPartitions.computeIfAbsent(topic, this::fetchPartitionsForTopic);
- }
-
- private int[] fetchPartitionsForTopic(String topic) {
- try (final Producer<?, ?> producer = new KafkaProducer<>(kafkaProducerConfig)) {
- // the fetched list is immutable, so we're creating a mutable copy in order to sort
- // it
- final List<PartitionInfo> partitionsList =
- new ArrayList<>(producer.partitionsFor(topic));
-
- return partitionsList.stream()
- .sorted(Comparator.comparing(PartitionInfo::partition))
- .map(PartitionInfo::partition)
- .mapToInt(Integer::intValue)
- .toArray();
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java
deleted file mode 100644
index a023cdd..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java
+++ /dev/null
@@ -1,375 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.internals.TransactionManager;
-import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.time.Duration;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state.
- */
-class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class);
- private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager";
- private static final String TRANSACTION_MANAGER_STATE_ENUM =
- "org.apache.kafka.clients.producer.internals.TransactionManager$State";
- private static final String PRODUCER_ID_AND_EPOCH_FIELD_NAME = "producerIdAndEpoch";
-
- @Nullable private String transactionalId;
- private volatile boolean inTransaction;
- private volatile boolean closed;
-
- public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) {
- super(withTransactionalId(properties, transactionalId));
- this.transactionalId = transactionalId;
- }
-
- private static Properties withTransactionalId(
- Properties properties, @Nullable String transactionalId) {
- if (transactionalId == null) {
- return properties;
- }
- Properties props = new Properties();
- props.putAll(properties);
- props.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
- return props;
- }
-
- @Override
- public void flush() {
- super.flush();
- if (inTransaction) {
- flushNewPartitions();
- }
- }
-
- @Override
- public void beginTransaction() throws ProducerFencedException {
- super.beginTransaction();
- inTransaction = true;
- }
-
- @Override
- public void abortTransaction() throws ProducerFencedException {
- LOG.debug("abortTransaction {}", transactionalId);
- checkState(inTransaction, "Transaction was not started");
- inTransaction = false;
- super.abortTransaction();
- }
-
- @Override
- public void commitTransaction() throws ProducerFencedException {
- LOG.debug("commitTransaction {}", transactionalId);
- checkState(inTransaction, "Transaction was not started");
- inTransaction = false;
- super.commitTransaction();
- }
-
- public boolean isInTransaction() {
- return inTransaction;
- }
-
- @Override
- public void close() {
- closed = true;
- if (inTransaction) {
- // This is state is most likely reached in case of a failure.
- // If this producer is still in transaction, it should be committing.
- // However, at this point, we cannot decide that and we shouldn't prolong cancellation.
- // So hard kill this producer with all resources.
- super.close(Duration.ZERO);
- } else {
- // If this is outside of a transaction, we should be able to cleanly shutdown.
- super.close(Duration.ofHours(1));
- }
- }
-
- @Override
- public void close(Duration timeout) {
- closed = true;
- super.close(timeout);
- }
-
- public boolean isClosed() {
- return closed;
- }
-
- @Nullable
- public String getTransactionalId() {
- return transactionalId;
- }
-
- public short getEpoch() {
- Object transactionManager = getTransactionManager();
- Object producerIdAndEpoch = getField(transactionManager, PRODUCER_ID_AND_EPOCH_FIELD_NAME);
- return (short) getField(producerIdAndEpoch, "epoch");
- }
-
- public long getProducerId() {
- Object transactionManager = getTransactionManager();
- Object producerIdAndEpoch = getField(transactionManager, PRODUCER_ID_AND_EPOCH_FIELD_NAME);
- return (long) getField(producerIdAndEpoch, "producerId");
- }
-
- public void initTransactionId(String transactionalId) {
- if (!transactionalId.equals(this.transactionalId)) {
- setTransactionId(transactionalId);
- initTransactions();
- }
- }
-
- public void setTransactionId(String transactionalId) {
- if (!transactionalId.equals(this.transactionalId)) {
- checkState(
- !inTransaction,
- String.format("Another transaction %s is still open.", transactionalId));
- LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId);
- Object transactionManager = getTransactionManager();
- synchronized (transactionManager) {
- setField(transactionManager, "transactionalId", transactionalId);
- setField(
- transactionManager,
- "currentState",
- getTransactionManagerState("UNINITIALIZED"));
- this.transactionalId = transactionalId;
- }
- }
- }
-
- /**
- * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction}
- * is also adding new partitions to the transaction. flushNewPartitions method is moving this
- * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise resumeTransaction
- * would require to restore state of the not yet added/"in-flight" partitions.
- */
- private void flushNewPartitions() {
- LOG.info("Flushing new partitions");
- TransactionalRequestResult result = enqueueNewPartitions();
- Object sender = getField("sender");
- invoke(sender, "wakeup");
- result.await();
- }
-
- /**
- * Enqueues new transactions at the transaction manager and returns a {@link
- * TransactionalRequestResult} that allows waiting on them.
- *
- * <p>If there are no new transactions we return a {@link TransactionalRequestResult} that is
- * already done.
- */
- private TransactionalRequestResult enqueueNewPartitions() {
- Object transactionManager = getTransactionManager();
- synchronized (transactionManager) {
- Object newPartitionsInTransaction =
- getField(transactionManager, "newPartitionsInTransaction");
- Object newPartitionsInTransactionIsEmpty =
- invoke(newPartitionsInTransaction, "isEmpty");
- TransactionalRequestResult result;
- if (newPartitionsInTransactionIsEmpty instanceof Boolean
- && !((Boolean) newPartitionsInTransactionIsEmpty)) {
- Object txnRequestHandler =
- invoke(transactionManager, "addPartitionsToTransactionHandler");
- invoke(
- transactionManager,
- "enqueueRequest",
- new Class[] {txnRequestHandler.getClass().getSuperclass()},
- new Object[] {txnRequestHandler});
- result =
- (TransactionalRequestResult)
- getField(
- txnRequestHandler,
- txnRequestHandler.getClass().getSuperclass(),
- "result");
- } else {
- // we don't have an operation but this operation string is also used in
- // addPartitionsToTransactionHandler.
- result = new TransactionalRequestResult("AddPartitionsToTxn");
- result.done();
- }
- return result;
- }
- }
-
- private static Object invoke(Object object, String methodName, Object... args) {
- Class<?>[] argTypes = new Class[args.length];
- for (int i = 0; i < args.length; i++) {
- argTypes[i] = args[i].getClass();
- }
- return invoke(object, methodName, argTypes, args);
- }
-
- private static Object invoke(
- Object object, String methodName, Class<?>[] argTypes, Object[] args) {
- try {
- Method method = object.getClass().getDeclaredMethod(methodName, argTypes);
- method.setAccessible(true);
- return method.invoke(object, args);
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- private Object getField(String fieldName) {
- return getField(this, KafkaProducer.class, fieldName);
- }
-
- /**
- * Gets and returns the field {@code fieldName} from the given Object {@code object} using
- * reflection.
- */
- private static Object getField(Object object, String fieldName) {
- return getField(object, object.getClass(), fieldName);
- }
-
- /**
- * Gets and returns the field {@code fieldName} from the given Object {@code object} using
- * reflection.
- */
- private static Object getField(Object object, Class<?> clazz, String fieldName) {
- try {
- Field field = clazz.getDeclaredField(fieldName);
- field.setAccessible(true);
- return field.get(object);
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- /**
- * Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously
- * obtained ones, so that we can resume transaction after a restart. Implementation of this
- * method is based on {@link KafkaProducer#initTransactions}.
- * https://github.com/apache/kafka/commit/5d2422258cb975a137a42a4e08f03573c49a387e#diff-f4ef1afd8792cd2a2e9069cd7ddea630
- */
- public void resumeTransaction(long producerId, short epoch) {
- checkState(!inTransaction, "Already in transaction %s", transactionalId);
- checkState(
- producerId >= 0 && epoch >= 0,
- "Incorrect values for producerId %s and epoch %s",
- producerId,
- epoch);
- LOG.info(
- "Attempting to resume transaction {} with producerId {} and epoch {}",
- transactionalId,
- producerId,
- epoch);
-
- Object transactionManager = getTransactionManager();
- synchronized (transactionManager) {
- Object topicPartitionBookkeeper =
- getField(transactionManager, "topicPartitionBookkeeper");
-
- transitionTransactionManagerStateTo(transactionManager, "INITIALIZING");
- invoke(topicPartitionBookkeeper, "reset");
-
- setField(
- transactionManager,
- PRODUCER_ID_AND_EPOCH_FIELD_NAME,
- createProducerIdAndEpoch(producerId, epoch));
-
- transitionTransactionManagerStateTo(transactionManager, "READY");
-
- transitionTransactionManagerStateTo(transactionManager, "IN_TRANSACTION");
- setField(transactionManager, "transactionStarted", true);
- this.inTransaction = true;
- }
- }
-
- private static Object createProducerIdAndEpoch(long producerId, short epoch) {
- try {
- Field field =
- TransactionManager.class.getDeclaredField(PRODUCER_ID_AND_EPOCH_FIELD_NAME);
- Class<?> clazz = field.getType();
- Constructor<?> constructor = clazz.getDeclaredConstructor(Long.TYPE, Short.TYPE);
- constructor.setAccessible(true);
- return constructor.newInstance(producerId, epoch);
- } catch (InvocationTargetException
- | InstantiationException
- | IllegalAccessException
- | NoSuchFieldException
- | NoSuchMethodException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- /**
- * Sets the field {@code fieldName} on the given Object {@code object} to {@code value} using
- * reflection.
- */
- private static void setField(Object object, String fieldName, Object value) {
- setField(object, object.getClass(), fieldName, value);
- }
-
- private static void setField(Object object, Class<?> clazz, String fieldName, Object value) {
- try {
- Field field = clazz.getDeclaredField(fieldName);
- field.setAccessible(true);
- field.set(object, value);
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- private static Enum<?> getTransactionManagerState(String enumName) {
- try {
- Class<Enum> cl = (Class<Enum>) Class.forName(TRANSACTION_MANAGER_STATE_ENUM);
- return Enum.valueOf(cl, enumName);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- private Object getTransactionManager() {
- return getField(TRANSACTION_MANAGER_FIELD_NAME);
- }
-
- private static void transitionTransactionManagerStateTo(
- Object transactionManager, String state) {
- invoke(transactionManager, "transitionTo", getTransactionManagerState(state));
- }
-
- @Override
- public String toString() {
- return "FlinkKafkaInternalProducer{"
- + "transactionalId='"
- + transactionalId
- + "', inTransaction="
- + inTransaction
- + ", closed="
- + closed
- + '}';
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java
deleted file mode 100644
index 72a3281..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.sink;
-
-import javax.annotation.Nullable;
-
-import java.util.Objects;
-import java.util.Optional;
-import java.util.function.Consumer;
-
-/**
- * This class holds the necessary information to construct a new {@link FlinkKafkaInternalProducer}
- * to commit transactions in {@link KafkaCommitter}.
- */
-class KafkaCommittable {
-
- private final long producerId;
- private final short epoch;
- private final String transactionalId;
- @Nullable private Recyclable<? extends FlinkKafkaInternalProducer<?, ?>> producer;
-
- public KafkaCommittable(
- long producerId,
- short epoch,
- String transactionalId,
- @Nullable Recyclable<? extends FlinkKafkaInternalProducer<?, ?>> producer) {
- this.producerId = producerId;
- this.epoch = epoch;
- this.transactionalId = transactionalId;
- this.producer = producer;
- }
-
- public static <K, V> KafkaCommittable of(
- FlinkKafkaInternalProducer<K, V> producer,
- Consumer<FlinkKafkaInternalProducer<K, V>> recycler) {
- return new KafkaCommittable(
- producer.getProducerId(),
- producer.getEpoch(),
- producer.getTransactionalId(),
- new Recyclable<>(producer, recycler));
- }
-
- public long getProducerId() {
- return producerId;
- }
-
- public short getEpoch() {
- return epoch;
- }
-
- public String getTransactionalId() {
- return transactionalId;
- }
-
- public Optional<Recyclable<? extends FlinkKafkaInternalProducer<?, ?>>> getProducer() {
- return Optional.ofNullable(producer);
- }
-
- @Override
- public String toString() {
- return "KafkaCommittable{"
- + "producerId="
- + producerId
- + ", epoch="
- + epoch
- + ", transactionalId="
- + transactionalId
- + '}';
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- KafkaCommittable that = (KafkaCommittable) o;
- return producerId == that.producerId
- && epoch == that.epoch
- && transactionalId.equals(that.transactionalId);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(producerId, epoch, transactionalId);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializer.java
deleted file mode 100644
index 78f1472..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializer.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-class KafkaCommittableSerializer implements SimpleVersionedSerializer<KafkaCommittable> {
-
- @Override
- public int getVersion() {
- return 1;
- }
-
- @Override
- public byte[] serialize(KafkaCommittable state) throws IOException {
- try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- final DataOutputStream out = new DataOutputStream(baos)) {
- out.writeShort(state.getEpoch());
- out.writeLong(state.getProducerId());
- out.writeUTF(state.getTransactionalId());
- out.flush();
- return baos.toByteArray();
- }
- }
-
- @Override
- public KafkaCommittable deserialize(int version, byte[] serialized) throws IOException {
- try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
- final DataInputStream in = new DataInputStream(bais)) {
- final short epoch = in.readShort();
- final long producerId = in.readLong();
- final String transactionalId = in.readUTF();
- return new KafkaCommittable(producerId, epoch, transactionalId, null);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java
deleted file mode 100644
index 4dbeaf9..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.connector.sink2.Committer;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.errors.InvalidTxnStateException;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.errors.RetriableException;
-import org.apache.kafka.common.errors.UnknownProducerIdException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Optional;
-import java.util.Properties;
-
-/**
- * Committer implementation for {@link KafkaSink}
- *
- * <p>The committer is responsible to finalize the Kafka transactions by committing them.
- */
-class KafkaCommitter implements Committer<KafkaCommittable>, Closeable {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaCommitter.class);
- public static final String UNKNOWN_PRODUCER_ID_ERROR_MESSAGE =
- "because of a bug in the Kafka broker (KAFKA-9310). Please upgrade to Kafka 2.5+. If you are running with concurrent checkpoints, you also may want to try without them.\n"
- + "To avoid data loss, the application will restart.";
-
- private final Properties kafkaProducerConfig;
-
- @Nullable private FlinkKafkaInternalProducer<?, ?> recoveryProducer;
-
- KafkaCommitter(Properties kafkaProducerConfig) {
- this.kafkaProducerConfig = kafkaProducerConfig;
- }
-
- @Override
- public void commit(Collection<CommitRequest<KafkaCommittable>> requests)
- throws IOException, InterruptedException {
- for (CommitRequest<KafkaCommittable> request : requests) {
- final KafkaCommittable committable = request.getCommittable();
- final String transactionalId = committable.getTransactionalId();
- LOG.debug("Committing Kafka transaction {}", transactionalId);
- Optional<Recyclable<? extends FlinkKafkaInternalProducer<?, ?>>> recyclable =
- committable.getProducer();
- FlinkKafkaInternalProducer<?, ?> producer;
- try {
- producer =
- recyclable
- .<FlinkKafkaInternalProducer<?, ?>>map(Recyclable::getObject)
- .orElseGet(() -> getRecoveryProducer(committable));
- producer.commitTransaction();
- producer.flush();
- recyclable.ifPresent(Recyclable::close);
- } catch (RetriableException e) {
- LOG.warn(
- "Encountered retriable exception while committing {}.", transactionalId, e);
- request.retryLater();
- } catch (ProducerFencedException e) {
- // initTransaction has been called on this transaction before
- LOG.error(
- "Unable to commit transaction ({}) because its producer is already fenced."
- + " This means that you either have a different producer with the same '{}' (this is"
- + " unlikely with the '{}' as all generated ids are unique and shouldn't be reused)"
- + " or recovery took longer than '{}' ({}ms). In both cases this most likely signals data loss,"
- + " please consult the Flink documentation for more details.",
- request,
- ProducerConfig.TRANSACTIONAL_ID_CONFIG,
- KafkaSink.class.getSimpleName(),
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
- kafkaProducerConfig.getProperty(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG),
- e);
- recyclable.ifPresent(Recyclable::close);
- request.signalFailedWithKnownReason(e);
- } catch (InvalidTxnStateException e) {
- // This exception only occurs when aborting after a commit or vice versa.
- // It does not appear on double commits or double aborts.
- LOG.error(
- "Unable to commit transaction ({}) because it's in an invalid state. "
- + "Most likely the transaction has been aborted for some reason. Please check the Kafka logs for more details.",
- request,
- e);
- recyclable.ifPresent(Recyclable::close);
- request.signalFailedWithKnownReason(e);
- } catch (UnknownProducerIdException e) {
- LOG.error(
- "Unable to commit transaction ({}) " + UNKNOWN_PRODUCER_ID_ERROR_MESSAGE,
- request,
- e);
- recyclable.ifPresent(Recyclable::close);
- request.signalFailedWithKnownReason(e);
- } catch (Exception e) {
- LOG.error(
- "Transaction ({}) encountered error and data has been potentially lost.",
- request,
- e);
- recyclable.ifPresent(Recyclable::close);
- request.signalFailedWithUnknownReason(e);
- }
- }
- }
-
- @Override
- public void close() {
- if (recoveryProducer != null) {
- recoveryProducer.close();
- }
- }
-
- /**
- * Creates a producer that can commit into the same transaction as the upstream producer that
- * was serialized into {@link KafkaCommittable}.
- */
- private FlinkKafkaInternalProducer<?, ?> getRecoveryProducer(KafkaCommittable committable) {
- if (recoveryProducer == null) {
- recoveryProducer =
- new FlinkKafkaInternalProducer<>(
- kafkaProducerConfig, committable.getTransactionalId());
- } else {
- recoveryProducer.setTransactionId(committable.getTransactionalId());
- }
- recoveryProducer.resumeTransaction(committable.getProducerId(), committable.getEpoch());
- return recoveryProducer;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java
deleted file mode 100644
index 9d081c7..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-
-/**
- * A serialization schema which defines how to convert a value of type {@code T} to {@link
- * ProducerRecord}.
- *
- * @param <T> the type of values being serialized
- */
-@PublicEvolving
-public interface KafkaRecordSerializationSchema<T> extends Serializable {
-
- /**
- * Initialization method for the schema. It is called before the actual working methods {@link
- * #serialize(Object, KafkaSinkContext, Long)} and thus suitable for one time setup work.
- *
- * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
- * additional features such as e.g. registering user metrics.
- *
- * @param context Contextual information that can be used during initialization.
- * @param sinkContext runtime information i.e. partitions, subtaskId
- */
- default void open(
- SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext)
- throws Exception {}
-
- /**
- * Serializes given element and returns it as a {@link ProducerRecord}.
- *
- * @param element element to be serialized
- * @param context context to possibly determine target partition
- * @param timestamp timestamp
- * @return Kafka {@link ProducerRecord} or null if the given element cannot be serialized
- */
- @Nullable
- ProducerRecord<byte[], byte[]> serialize(T element, KafkaSinkContext context, Long timestamp);
-
- /** Context providing information of the kafka record target location. */
- @Internal
- interface KafkaSinkContext {
-
- /**
- * Get the ID of the subtask the KafkaSink is running on. The numbering starts from 0 and
- * goes up to parallelism-1. (parallelism as returned by {@link
- * #getNumberOfParallelInstances()}
- *
- * @return ID of subtask
- */
- int getParallelInstanceId();
-
- /** @return number of parallel KafkaSink tasks. */
- int getNumberOfParallelInstances();
-
- /**
- * For a given topic id retrieve the available partitions.
- *
- * <p>After the first retrieval the returned partitions are cached. If the partitions are
- * updated the job has to be restarted to make the change visible.
- *
- * @param topic kafka topic with partitions
- * @return the ids of the currently available partitions
- */
- int[] getPartitionsForTopic(String topic);
- }
-
- /**
- * Creates a default schema builder to provide common building blocks i.e. key serialization,
- * value serialization, partitioning.
- *
- * @param <T> type of incoming elements
- * @return {@link KafkaRecordSerializationSchemaBuilder}
- */
- static <T> KafkaRecordSerializationSchemaBuilder<T> builder() {
- return new KafkaRecordSerializationSchemaBuilder<>();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java
deleted file mode 100644
index 59864a3..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.Serializer;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.OptionalInt;
-import java.util.function.Function;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * Builder to construct {@link KafkaRecordSerializationSchema}.
- *
- * <p>This class should give a first entrypoint when trying to serialize elements to {@link
- * ProducerRecord}. The following examples show some of the possibilities.
- *
- * <pre>Simple key-value serialization:
- * {@code
- * KafkaRecordSerializationSchema.builder()
- * .setTopic("topic)
- * .setKeySerializationSchema(new SimpleStringSchema())
- * .setValueSerializationSchema(new SimpleStringSchema())
- * .build()
- * }</pre>
- *
- * <pre>Using Kafka's serialization stack:
- * {@code
- * KafkaRecordSerializationSchema.builder()
- * .setTopic("topic)
- * .setKeySerializer(StringSerializer.class)
- * .setKafkaValueSerializer(StringSerializer.class)
- * .build()
- * }</pre>
- *
- * <pre>With custom partitioner:
- * {@code
- * KafkaRecordSerializationSchema.builder()
- * .setTopic("topic)
- * .setPartitioner(MY_FLINK_PARTITIONER)
- * .setValueSerializationSchema(StringSerializer.class)
- * .build()
- * }</pre>
- *
- * <p>The different serialization methods for key and value are mutually exclusive thus i.e. it is
- * not possible to use {@link #setKeySerializationSchema(SerializationSchema)} and {@link
- * #setKafkaKeySerializer(Class)} on the same builder instance.
- *
- * <p>It is necessary to configure exactly one serialization method for the value and a topic.
- *
- * @param <IN> type of records to be serialized
- * @see KafkaRecordSerializationSchema#builder()
- */
-@PublicEvolving
-public class KafkaRecordSerializationSchemaBuilder<IN> {
-
- @Nullable private Function<? super IN, String> topicSelector;
- @Nullable private SerializationSchema<? super IN> valueSerializationSchema;
- @Nullable private FlinkKafkaPartitioner<? super IN> partitioner;
- @Nullable private SerializationSchema<? super IN> keySerializationSchema;
-
- /**
- * Sets a custom partitioner determining the target partition of the target topic.
- *
- * @param partitioner
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setPartitioner(
- FlinkKafkaPartitioner<? super T> partitioner) {
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.partitioner = checkNotNull(partitioner);
- return self;
- }
-
- /**
- * Sets a fixed topic which used as destination for all records.
- *
- * @param topic
- * @return {@code this}
- */
- public KafkaRecordSerializationSchemaBuilder<IN> setTopic(String topic) {
- checkState(this.topicSelector == null, "Topic selector already set.");
- checkNotNull(topic);
- this.topicSelector = new CachingTopicSelector<>((e) -> topic);
- return this;
- }
-
- /**
- * Sets a topic selector which computes the target topic for every incoming record.
- *
- * @param topicSelector
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setTopicSelector(
- TopicSelector<? super T> topicSelector) {
- checkState(this.topicSelector == null, "Topic selector already set.");
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.topicSelector = new CachingTopicSelector<>(checkNotNull(topicSelector));
- return self;
- }
-
- /**
- * Sets a {@link SerializationSchema} which is used to serialize the incoming element to the key
- * of the {@link ProducerRecord}.
- *
- * @param keySerializationSchema
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setKeySerializationSchema(
- SerializationSchema<? super T> keySerializationSchema) {
- checkKeySerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.keySerializationSchema = checkNotNull(keySerializationSchema);
- return self;
- }
-
- /**
- * Sets Kafka's {@link Serializer} to serialize incoming elements to the key of the {@link
- * ProducerRecord}.
- *
- * @param keySerializer
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setKafkaKeySerializer(
- Class<? extends Serializer<? super T>> keySerializer) {
- checkKeySerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.keySerializationSchema =
- new KafkaSerializerWrapper<>(keySerializer, true, topicSelector);
- return self;
- }
-
- /**
- * Sets a configurable Kafka {@link Serializer} and pass a configuration to serialize incoming
- * elements to the key of the {@link ProducerRecord}.
- *
- * @param keySerializer
- * @param configuration
- * @param <S> type of the used serializer class
- * @return {@code this}
- */
- public <T extends IN, S extends Serializer<? super T>>
- KafkaRecordSerializationSchemaBuilder<T> setKafkaKeySerializer(
- Class<S> keySerializer, Map<String, String> configuration) {
- checkKeySerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.keySerializationSchema =
- new KafkaSerializerWrapper<>(keySerializer, true, configuration, topicSelector);
- return self;
- }
-
- /**
- * Sets a {@link SerializationSchema} which is used to serialize the incoming element to the
- * value of the {@link ProducerRecord}.
- *
- * @param valueSerializationSchema
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setValueSerializationSchema(
- SerializationSchema<T> valueSerializationSchema) {
- checkValueSerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.valueSerializationSchema = checkNotNull(valueSerializationSchema);
- return self;
- }
-
- @SuppressWarnings("unchecked")
- private <T extends IN> KafkaRecordSerializationSchemaBuilder<T> self() {
- return (KafkaRecordSerializationSchemaBuilder<T>) this;
- }
-
- /**
- * Sets Kafka's {@link Serializer} to serialize incoming elements to the value of the {@link
- * ProducerRecord}.
- *
- * @param valueSerializer
- * @return {@code this}
- */
- public <T extends IN> KafkaRecordSerializationSchemaBuilder<T> setKafkaValueSerializer(
- Class<? extends Serializer<? super T>> valueSerializer) {
- checkValueSerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.valueSerializationSchema =
- new KafkaSerializerWrapper<>(valueSerializer, false, topicSelector);
- return self;
- }
-
- /**
- * Sets a configurable Kafka {@link Serializer} and pass a configuration to serialize incoming
- * elements to the value of the {@link ProducerRecord}.
- *
- * @param valueSerializer
- * @param configuration
- * @param <S> type of the used serializer class
- * @return {@code this}
- */
- public <T extends IN, S extends Serializer<? super T>>
- KafkaRecordSerializationSchemaBuilder<T> setKafkaValueSerializer(
- Class<S> valueSerializer, Map<String, String> configuration) {
- checkValueSerializerNotSet();
- KafkaRecordSerializationSchemaBuilder<T> self = self();
- self.valueSerializationSchema =
- new KafkaSerializerWrapper<>(valueSerializer, false, configuration, topicSelector);
- return self;
- }
-
- /**
- * Constructs the {@link KafkaRecordSerializationSchemaBuilder} with the configured properties.
- *
- * @return {@link KafkaRecordSerializationSchema}
- */
- public KafkaRecordSerializationSchema<IN> build() {
- checkState(valueSerializationSchema != null, "No value serializer is configured.");
- checkState(topicSelector != null, "No topic selector is configured.");
- return new KafkaRecordSerializationSchemaWrapper<>(
- topicSelector, valueSerializationSchema, keySerializationSchema, partitioner);
- }
-
- private void checkValueSerializerNotSet() {
- checkState(valueSerializationSchema == null, "Value serializer already set.");
- }
-
- private void checkKeySerializerNotSet() {
- checkState(keySerializationSchema == null, "Key serializer already set.");
- }
-
- private static class CachingTopicSelector<IN> implements Function<IN, String>, Serializable {
-
- private static final int CACHE_RESET_SIZE = 5;
- private final Map<IN, String> cache;
- private final TopicSelector<IN> topicSelector;
-
- CachingTopicSelector(TopicSelector<IN> topicSelector) {
- this.topicSelector = topicSelector;
- this.cache = new HashMap<>();
- }
-
- @Override
- public String apply(IN in) {
- final String topic = cache.getOrDefault(in, topicSelector.apply(in));
- cache.put(in, topic);
- if (cache.size() == CACHE_RESET_SIZE) {
- cache.clear();
- }
- return topic;
- }
- }
-
- private static class KafkaRecordSerializationSchemaWrapper<IN>
- implements KafkaRecordSerializationSchema<IN> {
- private final SerializationSchema<? super IN> valueSerializationSchema;
- private final Function<? super IN, String> topicSelector;
- private final FlinkKafkaPartitioner<? super IN> partitioner;
- private final SerializationSchema<? super IN> keySerializationSchema;
-
- KafkaRecordSerializationSchemaWrapper(
- Function<? super IN, String> topicSelector,
- SerializationSchema<? super IN> valueSerializationSchema,
- @Nullable SerializationSchema<? super IN> keySerializationSchema,
- @Nullable FlinkKafkaPartitioner<? super IN> partitioner) {
- this.topicSelector = checkNotNull(topicSelector);
- this.valueSerializationSchema = checkNotNull(valueSerializationSchema);
- this.partitioner = partitioner;
- this.keySerializationSchema = keySerializationSchema;
- }
-
- @Override
- public void open(
- SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext)
- throws Exception {
- valueSerializationSchema.open(context);
- if (keySerializationSchema != null) {
- keySerializationSchema.open(context);
- }
- if (partitioner != null) {
- partitioner.open(
- sinkContext.getParallelInstanceId(),
- sinkContext.getNumberOfParallelInstances());
- }
- }
-
- @Override
- public ProducerRecord<byte[], byte[]> serialize(
- IN element, KafkaSinkContext context, Long timestamp) {
- final String targetTopic = topicSelector.apply(element);
- final byte[] value = valueSerializationSchema.serialize(element);
- byte[] key = null;
- if (keySerializationSchema != null) {
- key = keySerializationSchema.serialize(element);
- }
- final OptionalInt partition =
- partitioner != null
- ? OptionalInt.of(
- partitioner.partition(
- element,
- key,
- value,
- targetTopic,
- context.getPartitionsForTopic(targetTopic)))
- : OptionalInt.empty();
-
- return new ProducerRecord<>(
- targetTopic,
- partition.isPresent() ? partition.getAsInt() : null,
- timestamp == null || timestamp < 0L ? null : timestamp,
- key,
- value);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapper.java
deleted file mode 100644
index f212056..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapper.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.TemporaryClassLoaderContext;
-
-import org.apache.kafka.common.Configurable;
-import org.apache.kafka.common.serialization.Serializer;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.function.Function;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/** Wrapper for Kafka {@link Serializer}. */
-class KafkaSerializerWrapper<IN> implements SerializationSchema<IN> {
-
- private final Class<? extends Serializer<? super IN>> serializerClass;
- // Whether the serializer is for key or value.
- private final boolean isKey;
- private final Map<String, String> config;
- private final Function<? super IN, String> topicSelector;
-
- private transient Serializer<? super IN> serializer;
-
- KafkaSerializerWrapper(
- Class<? extends Serializer<? super IN>> serializerClass,
- boolean isKey,
- Map<String, String> config,
- Function<? super IN, String> topicSelector) {
- this.serializerClass = checkNotNull(serializerClass);
- this.isKey = isKey;
- this.config = checkNotNull(config);
- this.topicSelector = checkNotNull(topicSelector);
- }
-
- KafkaSerializerWrapper(
- Class<? extends Serializer<? super IN>> serializerClass,
- boolean isKey,
- Function<? super IN, String> topicSelector) {
- this(serializerClass, isKey, Collections.emptyMap(), topicSelector);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void open(InitializationContext context) throws Exception {
- final ClassLoader userCodeClassLoader = context.getUserCodeClassLoader().asClassLoader();
- try (TemporaryClassLoaderContext ignored =
- TemporaryClassLoaderContext.of(userCodeClassLoader)) {
- serializer =
- InstantiationUtil.instantiate(
- serializerClass.getName(),
- Serializer.class,
- getClass().getClassLoader());
-
- if (serializer instanceof Configurable) {
- ((Configurable) serializer).configure(config);
- } else {
- serializer.configure(config, isKey);
- }
- } catch (Exception e) {
- throw new IOException("Failed to instantiate the serializer of class " + serializer, e);
- }
- }
-
- @Override
- public byte[] serialize(IN element) {
- checkState(serializer != null, "Call open() once before trying to serialize elements.");
- return serializer.serialize(topicSelector.apply(element), element);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java
deleted file mode 100644
index 6f74aae..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.connector.sink2.Committer;
-import org.apache.flink.api.connector.sink2.StatefulSink;
-import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Properties;
-
-/**
- * Flink Sink to produce data into a Kafka topic. The sink supports all delivery guarantees
- * described by {@link DeliveryGuarantee}.
- * <li>{@link DeliveryGuarantee#NONE} does not provide any guarantees: messages may be lost in case
- * of issues on the Kafka broker and messages may be duplicated in case of a Flink failure.
- * <li>{@link DeliveryGuarantee#AT_LEAST_ONCE} the sink will wait for all outstanding records in the
- * Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. No messages will be
- * lost in case of any issue with the Kafka brokers but messages may be duplicated when Flink
- * restarts.
- * <li>{@link DeliveryGuarantee#EXACTLY_ONCE}: In this mode the KafkaSink will write all messages in
- * a Kafka transaction that will be committed to Kafka on a checkpoint. Thus, if the consumer
- * reads only committed data (see Kafka consumer config isolation.level), no duplicates will be
- * seen in case of a Flink restart. However, this delays record writing effectively until a
- * checkpoint is written, so adjust the checkpoint duration accordingly. Please ensure that you
- * use unique {@link #transactionalIdPrefix}s across your applications running on the same Kafka
- * cluster such that multiple running jobs do not interfere in their transactions! Additionally,
- * it is highly recommended to tweak Kafka transaction timeout (link) >> maximum checkpoint
- * duration + maximum restart duration or data loss may happen when Kafka expires an uncommitted
- * transaction.
- *
- * @param <IN> type of the records written to Kafka
- * @see KafkaSinkBuilder on how to construct a KafkaSink
- */
-@PublicEvolving
-public class KafkaSink<IN>
- implements StatefulSink<IN, KafkaWriterState>,
- TwoPhaseCommittingSink<IN, KafkaCommittable> {
-
- private final DeliveryGuarantee deliveryGuarantee;
-
- private final KafkaRecordSerializationSchema<IN> recordSerializer;
- private final Properties kafkaProducerConfig;
- private final String transactionalIdPrefix;
-
- KafkaSink(
- DeliveryGuarantee deliveryGuarantee,
- Properties kafkaProducerConfig,
- String transactionalIdPrefix,
- KafkaRecordSerializationSchema<IN> recordSerializer) {
- this.deliveryGuarantee = deliveryGuarantee;
- this.kafkaProducerConfig = kafkaProducerConfig;
- this.transactionalIdPrefix = transactionalIdPrefix;
- this.recordSerializer = recordSerializer;
- }
-
- /**
- * Create a {@link KafkaSinkBuilder} to construct a new {@link KafkaSink}.
- *
- * @param <IN> type of incoming records
- * @return {@link KafkaSinkBuilder}
- */
- public static <IN> KafkaSinkBuilder<IN> builder() {
- return new KafkaSinkBuilder<>();
- }
-
- @Internal
- @Override
- public Committer<KafkaCommittable> createCommitter() throws IOException {
- return new KafkaCommitter(kafkaProducerConfig);
- }
-
- @Internal
- @Override
- public SimpleVersionedSerializer<KafkaCommittable> getCommittableSerializer() {
- return new KafkaCommittableSerializer();
- }
-
- @Internal
- @Override
- public KafkaWriter<IN> createWriter(InitContext context) throws IOException {
- return new KafkaWriter<IN>(
- deliveryGuarantee,
- kafkaProducerConfig,
- transactionalIdPrefix,
- context,
- recordSerializer,
- context.asSerializationSchemaInitializationContext(),
- Collections.emptyList());
- }
-
- @Internal
- @Override
- public KafkaWriter<IN> restoreWriter(
- InitContext context, Collection<KafkaWriterState> recoveredState) throws IOException {
- return new KafkaWriter<>(
- deliveryGuarantee,
- kafkaProducerConfig,
- transactionalIdPrefix,
- context,
- recordSerializer,
- context.asSerializationSchemaInitializationContext(),
- recoveredState);
- }
-
- @Internal
- @Override
- public SimpleVersionedSerializer<KafkaWriterState> getWriterStateSerializer() {
- return new KafkaWriterStateSerializer();
- }
-
- @VisibleForTesting
- protected Properties getKafkaProducerConfig() {
- return kafkaProducerConfig;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java
deleted file mode 100644
index f0c20cf..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.charset.StandardCharsets;
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * Builder to construct {@link KafkaSink}.
- *
- * <p>The following example shows the minimum setup to create a KafkaSink that writes String values
- * to a Kafka topic.
- *
- * <pre>{@code
- * KafkaSink<String> sink = KafkaSink
- * .<String>builder
- * .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
- * .setRecordSerializer(MY_RECORD_SERIALIZER)
- * .build();
- * }</pre>
- *
- * <p>One can also configure different {@link DeliveryGuarantee} by using {@link
- * #setDeliveryGuarantee(DeliveryGuarantee)} but keep in mind when using {@link
- * DeliveryGuarantee#EXACTLY_ONCE} one must set the transactionalIdPrefix {@link
- * #setTransactionalIdPrefix(String)}.
- *
- * @see KafkaSink for a more detailed explanation of the different guarantees.
- * @param <IN> type of the records written to Kafka
- */
-@PublicEvolving
-public class KafkaSinkBuilder<IN> {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkBuilder.class);
- private static final Duration DEFAULT_KAFKA_TRANSACTION_TIMEOUT = Duration.ofHours(1);
- private static final String[] warnKeys =
- new String[] {
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG
- };
- private static final int MAXIMUM_PREFIX_BYTES = 64000;
-
- private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE;
- private String transactionalIdPrefix = "kafka-sink";
-
- private final Properties kafkaProducerConfig;
- private KafkaRecordSerializationSchema<IN> recordSerializer;
-
- KafkaSinkBuilder() {
- kafkaProducerConfig = new Properties();
- kafkaProducerConfig.put(
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
- kafkaProducerConfig.put(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
- kafkaProducerConfig.put(
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
- (int) DEFAULT_KAFKA_TRANSACTION_TIMEOUT.toMillis());
- }
-
- /**
- * Sets the wanted the {@link DeliveryGuarantee}. The default delivery guarantee is {@link
- * #deliveryGuarantee}.
- *
- * @param deliveryGuarantee
- * @return {@link KafkaSinkBuilder}
- */
- public KafkaSinkBuilder<IN> setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
- this.deliveryGuarantee = checkNotNull(deliveryGuarantee, "deliveryGuarantee");
- return this;
- }
-
- /**
- * Sets the wanted the {@link DeliveryGuarantee}. The default delivery guarantee is {@link
- * #deliveryGuarantee}.
- *
- * @param deliveryGuarantee
- * @return {@link KafkaSinkBuilder}
- * @deprecated Will be removed in future versions. Use {@link #setDeliveryGuarantee} instead.
- */
- @Deprecated
- public KafkaSinkBuilder<IN> setDeliverGuarantee(DeliveryGuarantee deliveryGuarantee) {
- this.deliveryGuarantee = checkNotNull(deliveryGuarantee, "deliveryGuarantee");
- return this;
- }
-
- /**
- * Sets the configuration which used to instantiate all used {@link
- * org.apache.kafka.clients.producer.KafkaProducer}.
- *
- * @param props
- * @return {@link KafkaSinkBuilder}
- */
- public KafkaSinkBuilder<IN> setKafkaProducerConfig(Properties props) {
- checkNotNull(props);
- Arrays.stream(warnKeys)
- .filter(props::containsKey)
- .forEach(k -> LOG.warn("Overwriting the '{}' is not recommended", k));
-
- kafkaProducerConfig.putAll(props);
- return this;
- }
-
- public KafkaSinkBuilder<IN> setProperty(String key, String value) {
- checkNotNull(key);
- Arrays.stream(warnKeys)
- .filter(key::equals)
- .forEach(k -> LOG.warn("Overwriting the '{}' is not recommended", k));
-
- kafkaProducerConfig.setProperty(key, value);
- return this;
- }
-
- /**
- * Sets the {@link KafkaRecordSerializationSchema} that transforms incoming records to {@link
- * org.apache.kafka.clients.producer.ProducerRecord}s.
- *
- * @param recordSerializer
- * @return {@link KafkaSinkBuilder}
- */
- public KafkaSinkBuilder<IN> setRecordSerializer(
- KafkaRecordSerializationSchema<IN> recordSerializer) {
- this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer");
- ClosureCleaner.clean(
- this.recordSerializer, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- return this;
- }
-
- /**
- * Sets the prefix for all created transactionalIds if {@link DeliveryGuarantee#EXACTLY_ONCE} is
- * configured.
- *
- * <p>It is mandatory to always set this value with {@link DeliveryGuarantee#EXACTLY_ONCE} to
- * prevent corrupted transactions if multiple jobs using the KafkaSink run against the same
- * Kafka Cluster. The default prefix is {@link #transactionalIdPrefix}.
- *
- * <p>The size of the prefix is capped by {@link #MAXIMUM_PREFIX_BYTES} formatted with UTF-8.
- *
- * <p>It is important to keep the prefix stable across application restarts. If the prefix
- * changes it might happen that lingering transactions are not correctly aborted and newly
- * written messages are not immediately consumable until the transactions timeout.
- *
- * @param transactionalIdPrefix
- * @return {@link KafkaSinkBuilder}
- */
- public KafkaSinkBuilder<IN> setTransactionalIdPrefix(String transactionalIdPrefix) {
- this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix");
- checkState(
- transactionalIdPrefix.getBytes(StandardCharsets.UTF_8).length
- <= MAXIMUM_PREFIX_BYTES,
- "The configured prefix is too long and the resulting transactionalId might exceed Kafka's transactionalIds size.");
- return this;
- }
-
- /**
- * Sets the Kafka bootstrap servers.
- *
- * @param bootstrapServers a comma separated list of valid URIs to reach the Kafka broker
- * @return {@link KafkaSinkBuilder}
- */
- public KafkaSinkBuilder<IN> setBootstrapServers(String bootstrapServers) {
- return setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- }
-
- private void sanityCheck() {
- checkNotNull(
- kafkaProducerConfig.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG),
- "bootstrapServers");
- if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
- checkState(
- transactionalIdPrefix != null,
- "EXACTLY_ONCE delivery guarantee requires a transactionIdPrefix to be set to provide unique transaction names across multiple KafkaSinks writing to the same Kafka cluster.");
- }
- checkNotNull(recordSerializer, "recordSerializer");
- }
-
- /**
- * Constructs the {@link KafkaSink} with the configured properties.
- *
- * @return {@link KafkaSink}
- */
- public KafkaSink<IN> build() {
- sanityCheck();
- return new KafkaSink<>(
- deliveryGuarantee, kafkaProducerConfig, transactionalIdPrefix, recordSerializer);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java
deleted file mode 100644
index 29d7cb4..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java
+++ /dev/null
@@ -1,473 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.operators.MailboxExecutor;
-import org.apache.flink.api.common.operators.ProcessingTimeService;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.connector.sink2.Sink;
-import org.apache.flink.api.connector.sink2.StatefulSink;
-import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.MetricUtil;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
-import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricMutableWrapper;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
-import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
-import org.apache.flink.shaded.guava30.com.google.common.io.Closer;
-
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.errors.UnknownProducerIdException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.function.Consumer;
-
-import static org.apache.flink.util.IOUtils.closeAll;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * This class is responsible to write records in a Kafka topic and to handle the different delivery
- * {@link DeliveryGuarantee}s.
- *
- * @param <IN> The type of the input elements.
- */
-class KafkaWriter<IN>
- implements StatefulSink.StatefulSinkWriter<IN, KafkaWriterState>,
- TwoPhaseCommittingSink.PrecommittingSinkWriter<IN, KafkaCommittable> {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaWriter.class);
- private static final String KAFKA_PRODUCER_METRIC_NAME = "KafkaProducer";
- private static final long METRIC_UPDATE_INTERVAL_MILLIS = 500;
-
- private static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
- private static final String KEY_REGISTER_METRICS = "register.producer.metrics";
- private static final String KAFKA_PRODUCER_METRICS = "producer-metrics";
-
- private final DeliveryGuarantee deliveryGuarantee;
- private final Properties kafkaProducerConfig;
- private final String transactionalIdPrefix;
- private final KafkaRecordSerializationSchema<IN> recordSerializer;
- private final Callback deliveryCallback;
- private final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext;
- private volatile Exception asyncProducerException;
- private final Map<String, KafkaMetricMutableWrapper> previouslyCreatedMetrics = new HashMap<>();
- private final SinkWriterMetricGroup metricGroup;
- private final boolean disabledMetrics;
- private final Counter numRecordsOutCounter;
- private final Counter numBytesOutCounter;
- private final Counter numRecordsOutErrorsCounter;
- private final ProcessingTimeService timeService;
-
- // Number of outgoing bytes at the latest metric sync
- private long latestOutgoingByteTotal;
- private Metric byteOutMetric;
- private FlinkKafkaInternalProducer<byte[], byte[]> currentProducer;
- private final KafkaWriterState kafkaWriterState;
- // producer pool only used for exactly once
- private final Deque<FlinkKafkaInternalProducer<byte[], byte[]>> producerPool =
- new ArrayDeque<>();
- private final Closer closer = Closer.create();
- private long lastCheckpointId;
-
- private boolean closed = false;
- private long lastSync = System.currentTimeMillis();
-
- /**
- * Constructor creating a kafka writer.
- *
- * <p>It will throw a {@link RuntimeException} if {@link
- * KafkaRecordSerializationSchema#open(SerializationSchema.InitializationContext,
- * KafkaRecordSerializationSchema.KafkaSinkContext)} fails.
- *
- * @param deliveryGuarantee the Sink's delivery guarantee
- * @param kafkaProducerConfig the properties to configure the {@link FlinkKafkaInternalProducer}
- * @param transactionalIdPrefix used to create the transactionalIds
- * @param sinkInitContext context to provide information about the runtime environment
- * @param recordSerializer serialize to transform the incoming records to {@link ProducerRecord}
- * @param schemaContext context used to initialize the {@link KafkaRecordSerializationSchema}
- * @param recoveredStates state from an previous execution which was covered
- */
- KafkaWriter(
- DeliveryGuarantee deliveryGuarantee,
- Properties kafkaProducerConfig,
- String transactionalIdPrefix,
- Sink.InitContext sinkInitContext,
- KafkaRecordSerializationSchema<IN> recordSerializer,
- SerializationSchema.InitializationContext schemaContext,
- Collection<KafkaWriterState> recoveredStates) {
- this.deliveryGuarantee = checkNotNull(deliveryGuarantee, "deliveryGuarantee");
- this.kafkaProducerConfig = checkNotNull(kafkaProducerConfig, "kafkaProducerConfig");
- this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix");
- this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer");
- checkNotNull(sinkInitContext, "sinkInitContext");
- this.deliveryCallback =
- new WriterCallback(
- sinkInitContext.getMailboxExecutor(),
- sinkInitContext.<RecordMetadata>metadataConsumer().orElse(null));
- this.disabledMetrics =
- kafkaProducerConfig.containsKey(KEY_DISABLE_METRICS)
- && Boolean.parseBoolean(
- kafkaProducerConfig.get(KEY_DISABLE_METRICS).toString())
- || kafkaProducerConfig.containsKey(KEY_REGISTER_METRICS)
- && !Boolean.parseBoolean(
- kafkaProducerConfig.get(KEY_REGISTER_METRICS).toString());
- this.timeService = sinkInitContext.getProcessingTimeService();
- this.metricGroup = sinkInitContext.metricGroup();
- this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter();
- this.numRecordsOutCounter = metricGroup.getIOMetricGroup().getNumRecordsOutCounter();
- this.numRecordsOutErrorsCounter = metricGroup.getNumRecordsOutErrorsCounter();
- this.kafkaSinkContext =
- new DefaultKafkaSinkContext(
- sinkInitContext.getSubtaskId(),
- sinkInitContext.getNumberOfParallelSubtasks(),
- kafkaProducerConfig);
- try {
- recordSerializer.open(schemaContext, kafkaSinkContext);
- } catch (Exception e) {
- throw new FlinkRuntimeException("Cannot initialize schema.", e);
- }
-
- this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix);
- this.lastCheckpointId =
- sinkInitContext
- .getRestoredCheckpointId()
- .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1);
- if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
- abortLingeringTransactions(
- checkNotNull(recoveredStates, "recoveredStates"), lastCheckpointId + 1);
- this.currentProducer = getTransactionalProducer(lastCheckpointId + 1);
- this.currentProducer.beginTransaction();
- } else if (deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE
- || deliveryGuarantee == DeliveryGuarantee.NONE) {
- this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig, null);
- closer.register(this.currentProducer);
- initKafkaMetrics(this.currentProducer);
- } else {
- throw new UnsupportedOperationException(
- "Unsupported Kafka writer semantic " + this.deliveryGuarantee);
- }
-
- initFlinkMetrics();
- }
-
- @Override
- public void write(@Nullable IN element, Context context) throws IOException {
- checkAsyncException();
- final ProducerRecord<byte[], byte[]> record =
- recordSerializer.serialize(element, kafkaSinkContext, context.timestamp());
- if (record != null) {
- currentProducer.send(record, deliveryCallback);
- numRecordsOutCounter.inc();
- }
- }
-
- @Override
- public void flush(boolean endOfInput) throws IOException, InterruptedException {
- if (deliveryGuarantee != DeliveryGuarantee.NONE || endOfInput) {
- LOG.debug("final flush={}", endOfInput);
- currentProducer.flush();
- }
-
- checkAsyncException();
- }
-
- @Override
- public Collection<KafkaCommittable> prepareCommit() {
- if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
- final List<KafkaCommittable> committables =
- Collections.singletonList(
- KafkaCommittable.of(currentProducer, producerPool::add));
- LOG.debug("Committing {} committables.", committables);
- return committables;
- }
- return Collections.emptyList();
- }
-
- @Override
- public List<KafkaWriterState> snapshotState(long checkpointId) throws IOException {
- if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
- currentProducer = getTransactionalProducer(checkpointId + 1);
- currentProducer.beginTransaction();
- }
- return ImmutableList.of(kafkaWriterState);
- }
-
- @Override
- public void close() throws Exception {
- closed = true;
- LOG.debug("Closing writer with {}", currentProducer);
- closeAll(
- this::abortCurrentProducer,
- closer,
- producerPool::clear,
- () -> {
- checkState(currentProducer.isClosed());
- currentProducer = null;
- });
-
- // Rethrow exception for the case in which close is called before writer() and flush().
- checkAsyncException();
- }
-
- private void abortCurrentProducer() {
- if (currentProducer.isInTransaction()) {
- try {
- currentProducer.abortTransaction();
- } catch (ProducerFencedException e) {
- LOG.debug(
- "Producer {} fenced while aborting", currentProducer.getTransactionalId());
- }
- }
- }
-
- @VisibleForTesting
- Deque<FlinkKafkaInternalProducer<byte[], byte[]>> getProducerPool() {
- return producerPool;
- }
-
- @VisibleForTesting
- FlinkKafkaInternalProducer<byte[], byte[]> getCurrentProducer() {
- return currentProducer;
- }
-
- void abortLingeringTransactions(
- Collection<KafkaWriterState> recoveredStates, long startCheckpointId) {
- List<String> prefixesToAbort = Lists.newArrayList(transactionalIdPrefix);
-
- final Optional<KafkaWriterState> lastStateOpt = recoveredStates.stream().findFirst();
- if (lastStateOpt.isPresent()) {
- KafkaWriterState lastState = lastStateOpt.get();
- if (!lastState.getTransactionalIdPrefix().equals(transactionalIdPrefix)) {
- prefixesToAbort.add(lastState.getTransactionalIdPrefix());
- LOG.warn(
- "Transactional id prefix from previous execution {} has changed to {}.",
- lastState.getTransactionalIdPrefix(),
- transactionalIdPrefix);
- }
- }
-
- try (TransactionAborter transactionAborter =
- new TransactionAborter(
- kafkaSinkContext.getParallelInstanceId(),
- kafkaSinkContext.getNumberOfParallelInstances(),
- this::getOrCreateTransactionalProducer,
- producerPool::add)) {
- transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId);
- }
- }
-
- /**
- * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions
- * will not clash with transactions created during previous checkpoints ({@code
- * producer.initTransactions()} assures that we obtain new producerId and epoch counters).
- *
- * <p>Ensures that all transaction ids in between lastCheckpointId and checkpointId are
- * initialized.
- */
- private FlinkKafkaInternalProducer<byte[], byte[]> getTransactionalProducer(long checkpointId) {
- checkState(
- checkpointId > lastCheckpointId,
- "Expected %s > %s",
- checkpointId,
- lastCheckpointId);
- FlinkKafkaInternalProducer<byte[], byte[]> producer = null;
- // in case checkpoints have been aborted, Flink would create non-consecutive transaction ids
- // this loop ensures that all gaps are filled with initialized (empty) transactions
- for (long id = lastCheckpointId + 1; id <= checkpointId; id++) {
- String transactionalId =
- TransactionalIdFactory.buildTransactionalId(
- transactionalIdPrefix, kafkaSinkContext.getParallelInstanceId(), id);
- producer = getOrCreateTransactionalProducer(transactionalId);
- }
- this.lastCheckpointId = checkpointId;
- assert producer != null;
- LOG.info("Created new transactional producer {}", producer.getTransactionalId());
- return producer;
- }
-
- private FlinkKafkaInternalProducer<byte[], byte[]> getOrCreateTransactionalProducer(
- String transactionalId) {
- FlinkKafkaInternalProducer<byte[], byte[]> producer = producerPool.poll();
- if (producer == null) {
- producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId);
- closer.register(producer);
- producer.initTransactions();
- initKafkaMetrics(producer);
- } else {
- producer.initTransactionId(transactionalId);
- }
- return producer;
- }
-
- private void initFlinkMetrics() {
- metricGroup.setCurrentSendTimeGauge(this::computeSendTime);
- registerMetricSync();
- }
-
- private void initKafkaMetrics(FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- byteOutMetric =
- MetricUtil.getKafkaMetric(
- producer.metrics(), KAFKA_PRODUCER_METRICS, "outgoing-byte-total");
- if (disabledMetrics) {
- return;
- }
- final MetricGroup kafkaMetricGroup = metricGroup.addGroup(KAFKA_PRODUCER_METRIC_NAME);
- producer.metrics().entrySet().forEach(initMetric(kafkaMetricGroup));
- }
-
- private Consumer<Map.Entry<MetricName, ? extends Metric>> initMetric(
- MetricGroup kafkaMetricGroup) {
- return (entry) -> {
- final String name = entry.getKey().name();
- final Metric metric = entry.getValue();
- if (previouslyCreatedMetrics.containsKey(name)) {
- final KafkaMetricMutableWrapper wrapper = previouslyCreatedMetrics.get(name);
- wrapper.setKafkaMetric(metric);
- } else {
- final KafkaMetricMutableWrapper wrapper = new KafkaMetricMutableWrapper(metric);
- previouslyCreatedMetrics.put(name, wrapper);
- kafkaMetricGroup.gauge(name, wrapper);
- }
- };
- }
-
- private long computeSendTime() {
- FlinkKafkaInternalProducer<byte[], byte[]> producer = this.currentProducer;
- if (producer == null) {
- return -1L;
- }
- final Metric sendTime =
- MetricUtil.getKafkaMetric(
- producer.metrics(), KAFKA_PRODUCER_METRICS, "request-latency-avg");
- final Metric queueTime =
- MetricUtil.getKafkaMetric(
- producer.metrics(), KAFKA_PRODUCER_METRICS, "record-queue-time-avg");
- return ((Number) sendTime.metricValue()).longValue()
- + ((Number) queueTime.metricValue()).longValue();
- }
-
- private void registerMetricSync() {
- timeService.registerTimer(
- lastSync + METRIC_UPDATE_INTERVAL_MILLIS,
- (time) -> {
- if (closed) {
- return;
- }
- long outgoingBytesUntilNow = ((Number) byteOutMetric.metricValue()).longValue();
- long outgoingBytesSinceLastUpdate =
- outgoingBytesUntilNow - latestOutgoingByteTotal;
- numBytesOutCounter.inc(outgoingBytesSinceLastUpdate);
- latestOutgoingByteTotal = outgoingBytesUntilNow;
- lastSync = time;
- registerMetricSync();
- });
- }
-
- /**
- * This method should only be invoked in the mailbox thread since the counter is not volatile.
- * Logic needs to be invoked by write AND flush since we support various semantics.
- */
- private void checkAsyncException() throws IOException {
- // reset this exception since we could close the writer later on
- Exception e = asyncProducerException;
- if (e != null) {
-
- asyncProducerException = null;
- numRecordsOutErrorsCounter.inc();
- throw new IOException(
- "One or more Kafka Producer send requests have encountered exception", e);
- }
- }
-
- private class WriterCallback implements Callback {
- private final MailboxExecutor mailboxExecutor;
- @Nullable private final Consumer<RecordMetadata> metadataConsumer;
-
- public WriterCallback(
- MailboxExecutor mailboxExecutor,
- @Nullable Consumer<RecordMetadata> metadataConsumer) {
- this.mailboxExecutor = mailboxExecutor;
- this.metadataConsumer = metadataConsumer;
- }
-
- @Override
- public void onCompletion(RecordMetadata metadata, Exception exception) {
- if (exception != null) {
- FlinkKafkaInternalProducer<byte[], byte[]> producer =
- KafkaWriter.this.currentProducer;
-
- // Propagate the first exception since amount of exceptions could be large. Need to
- // do this in Producer IO thread since flush() guarantees that the future will
- // complete. The same guarantee does not hold for tasks executed in separate
- // executor e.g. mailbox executor. flush() needs to have the exception immediately
- // available to fail the checkpoint.
- if (asyncProducerException == null) {
- asyncProducerException = decorateException(metadata, exception, producer);
- }
-
- mailboxExecutor.submit(
- () -> {
- // Checking for exceptions from previous writes
- checkAsyncException();
- },
- "Update error metric");
- }
-
- if (metadataConsumer != null) {
- metadataConsumer.accept(metadata);
- }
- }
-
- private FlinkRuntimeException decorateException(
- RecordMetadata metadata,
- Exception exception,
- FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- String message =
- String.format("Failed to send data to Kafka %s with %s ", metadata, producer);
- if (exception instanceof UnknownProducerIdException) {
- message += KafkaCommitter.UNKNOWN_PRODUCER_ID_ERROR_MESSAGE;
- }
- return new FlinkRuntimeException(message, exception);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterState.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterState.java
deleted file mode 100644
index b4482c6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterState.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.sink;
-
-import java.util.Objects;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-class KafkaWriterState {
- private final String transactionalIdPrefix;
-
- KafkaWriterState(String transactionalIdPrefix) {
- this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix");
- }
-
- public String getTransactionalIdPrefix() {
- return transactionalIdPrefix;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- KafkaWriterState that = (KafkaWriterState) o;
- return transactionalIdPrefix.equals(that.transactionalIdPrefix);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(transactionalIdPrefix);
- }
-
- @Override
- public String toString() {
- return "KafkaWriterState{"
- + ", transactionalIdPrefix='"
- + transactionalIdPrefix
- + '\''
- + '}';
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializer.java
deleted file mode 100644
index 5c91967..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-/** A serializer used to serialize {@link KafkaWriterState}. */
-class KafkaWriterStateSerializer implements SimpleVersionedSerializer<KafkaWriterState> {
-
- @Override
- public int getVersion() {
- return 1;
- }
-
- @Override
- public byte[] serialize(KafkaWriterState state) throws IOException {
- try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- final DataOutputStream out = new DataOutputStream(baos)) {
- out.writeUTF(state.getTransactionalIdPrefix());
- out.flush();
- return baos.toByteArray();
- }
- }
-
- @Override
- public KafkaWriterState deserialize(int version, byte[] serialized) throws IOException {
- try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
- final DataInputStream in = new DataInputStream(bais)) {
- final String transactionalIdPrefx = in.readUTF();
- return new KafkaWriterState(transactionalIdPrefx);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java
deleted file mode 100644
index 012fa99..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.sink;
-
-import java.io.Closeable;
-import java.util.function.Consumer;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-class Recyclable<T> implements Closeable {
- private T object;
- private final Consumer<T> recycler;
-
- public Recyclable(T object, Consumer<T> recycler) {
- this.object = checkNotNull(object);
- this.recycler = checkNotNull(recycler);
- }
-
- public T getObject() {
- checkState(!isRecycled(), "Already recycled");
- return object;
- }
-
- boolean isRecycled() {
- return object == null;
- }
-
- @Override
- public void close() {
- recycler.accept(object);
- object = null;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TopicSelector.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TopicSelector.java
deleted file mode 100644
index 2a20754..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TopicSelector.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-import java.util.function.Function;
-
-/**
- * Selects a topic for the incoming record.
- *
- * @param <IN> type of the incoming record
- */
-@PublicEvolving
-public interface TopicSelector<IN> extends Function<IN, String>, Serializable {}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java
deleted file mode 100644
index cae6ca6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * 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.sink;
-
-import javax.annotation.Nullable;
-
-import java.io.Closeable;
-import java.util.List;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Aborts lingering transactions on restart.
- *
- * <p>Transactions are lingering if they are not tracked anywhere. For example, if a job is started
- * transactions are opened. A restart without checkpoint would not allow Flink to abort old
- * transactions. Since Kafka's transactions are sequential, newly produced data does not become
- * visible for read_committed consumers. However, Kafka has no API for querying open transactions,
- * so they become lingering.
- *
- * <p>Flink solves this by assuming consecutive transaction ids. On restart of checkpoint C on
- * subtask S, it will sequentially cancel transaction C+1, C+2, ... of S until it finds the first
- * unused transaction.
- *
- * <p>Additionally, to cover for weird downscaling cases without checkpoints, it also checks for
- * transactions of subtask S+P where P is the current parallelism until it finds a subtask without
- * transactions.
- */
-class TransactionAborter implements Closeable {
- private final int subtaskId;
- private final int parallelism;
- private final Function<String, FlinkKafkaInternalProducer<byte[], byte[]>> producerFactory;
- private final Consumer<FlinkKafkaInternalProducer<byte[], byte[]>> closeAction;
- @Nullable FlinkKafkaInternalProducer<byte[], byte[]> producer = null;
-
- public TransactionAborter(
- int subtaskId,
- int parallelism,
- Function<String, FlinkKafkaInternalProducer<byte[], byte[]>> producerFactory,
- Consumer<FlinkKafkaInternalProducer<byte[], byte[]>> closeAction) {
- this.subtaskId = subtaskId;
- this.parallelism = parallelism;
- this.producerFactory = checkNotNull(producerFactory);
- this.closeAction = closeAction;
- }
-
- void abortLingeringTransactions(List<String> prefixesToAbort, long startCheckpointId) {
- for (String prefix : prefixesToAbort) {
- abortTransactionsWithPrefix(prefix, startCheckpointId);
- }
- }
-
- /**
- * Aborts all transactions that have been created by this subtask in a previous run.
- *
- * <p>It also aborts transactions from subtasks that may have been removed because of
- * downscaling.
- *
- * <p>When Flink downscales X subtasks to Y subtasks, then subtask i is responsible for cleaning
- * all subtasks j in [0; X), where j % Y = i. For example, if we downscale to 2, then subtask 0
- * is responsible for all even and subtask 1 for all odd subtasks.
- */
- private void abortTransactionsWithPrefix(String prefix, long startCheckpointId) {
- for (int subtaskId = this.subtaskId; ; subtaskId += parallelism) {
- if (abortTransactionOfSubtask(prefix, startCheckpointId, subtaskId) == 0) {
- // If Flink didn't abort any transaction for current subtask, then we assume that no
- // such subtask existed and no subtask with a higher number as well.
- break;
- }
- }
- }
-
- /**
- * Aborts all transactions that have been created by a subtask in a previous run after the given
- * checkpoint id.
- *
- * <p>We assume that transaction ids are consecutively used and thus Flink can stop aborting as
- * soon as Flink notices that a particular transaction id was unused.
- */
- private int abortTransactionOfSubtask(String prefix, long startCheckpointId, int subtaskId) {
- int numTransactionAborted = 0;
- for (long checkpointId = startCheckpointId; ; checkpointId++, numTransactionAborted++) {
- // initTransactions fences all old transactions with the same id by bumping the epoch
- String transactionalId =
- TransactionalIdFactory.buildTransactionalId(prefix, subtaskId, checkpointId);
- if (producer == null) {
- producer = producerFactory.apply(transactionalId);
- } else {
- producer.initTransactionId(transactionalId);
- }
- producer.flush();
- // An epoch of 0 indicates that the id was unused before
- if (producer.getEpoch() == 0) {
- // Note that the check works beyond transaction log timeouts and just depends on the
- // retention of the transaction topic (typically 7d). Any transaction that is not in
- // the that topic anymore is also not lingering (i.e., it will not block downstream
- // from reading)
- // This method will only cease to work if transaction log timeout = topic retention
- // and a user didn't restart the application for that period of time. Then the first
- // transactions would vanish from the topic while later transactions are still
- // lingering until they are cleaned up by Kafka. Then the user has to wait until the
- // other transactions are timed out (which shouldn't take too long).
- break;
- }
- }
- return numTransactionAborted;
- }
-
- public void close() {
- if (producer != null) {
- closeAction.accept(producer);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java
deleted file mode 100644
index eda4c01..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.sink;
-
-class TransactionalIdFactory {
- private static final String TRANSACTIONAL_ID_DELIMITER = "-";
-
- /**
- * Constructs a transactionalId with the following format {@code
- * transactionalIdPrefix-subtaskId-checkpointOffset}.
- *
- * @param transactionalIdPrefix prefix for the id
- * @param subtaskId describing the subtask which is opening the transaction
- * @param checkpointOffset an always incrementing number usually capturing the number of
- * checkpoints taken by the subtask
- * @return transactionalId
- */
- public static String buildTransactionalId(
- String transactionalIdPrefix, int subtaskId, long checkpointOffset) {
- return transactionalIdPrefix
- + TRANSACTIONAL_ID_DELIMITER
- + subtaskId
- + TRANSACTIONAL_ID_DELIMITER
- + checkpointOffset;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java
deleted file mode 100644
index 2d79b35..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.sink;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-class TransactionsToAbortChecker {
-
- private static final int MINIMUM_CHECKPOINT_OFFSET = 1;
-
- private final int numberOfParallelSubtasks;
- private final Map<Integer, Long> subtaskIdCheckpointOffsetMapping;
- private final int subtaskId;
-
- TransactionsToAbortChecker(
- int numberOfParallelSubtasks,
- Map<Integer, Long> subtaskIdCheckpointOffsetMapping,
- int subtaskId) {
- this.subtaskId = subtaskId;
- this.numberOfParallelSubtasks = numberOfParallelSubtasks;
- this.subtaskIdCheckpointOffsetMapping = subtaskIdCheckpointOffsetMapping;
- }
-
- /**
- * Iterates through all open transactions and filters for the following attributes.
- *
- * <ol>
- * <li>If the minimum checkpointOffset for the subtask is {@link #MINIMUM_CHECKPOINT_OFFSET}
- * and [openSubtaskId % {@link #numberOfParallelSubtasks} == {@link #subtaskId}] return
- * all transactions from this subtask
- * <li>If the subtaskId is part of the recovered states {@link
- * #subtaskIdCheckpointOffsetMapping} and the checkpointOffset >= the recovered offSet
- * also return this transactionalId
- * </ol>
- *
- * @param openTransactions Mapping of {subtaskId: {checkpointOffset: transactionalId}}
- * @return transactionalIds which must be aborted
- */
- public List<String> getTransactionsToAbort(Map<Integer, Map<Long, String>> openTransactions) {
- final List<String> transactionalIdsToAbort = new ArrayList<>();
- for (final Map.Entry<Integer, Map<Long, String>> subtaskOffsetMapping :
- openTransactions.entrySet()) {
- final Map<Long, String> checkpointOffsetTransactionalIdMapping =
- subtaskOffsetMapping.getValue();
- // All transactions from this subtask have been closed
- if (checkpointOffsetTransactionalIdMapping.isEmpty()) {
- continue;
- }
- // Abort all open transactions if checkpointOffset 0 is open implying that no checkpoint
- // finished.
- // Cut the transactions in ranges to speed up abort process
- if (Collections.min(checkpointOffsetTransactionalIdMapping.keySet())
- == MINIMUM_CHECKPOINT_OFFSET
- && subtaskOffsetMapping.getKey() % numberOfParallelSubtasks == subtaskId) {
- transactionalIdsToAbort.addAll(checkpointOffsetTransactionalIdMapping.values());
- } else {
- // Check all open transactions against recovered ones and close if the open
- // transaction is equal or higher to the offset
- for (final Map.Entry<Long, String> offsetTransactionId :
- checkpointOffsetTransactionalIdMapping.entrySet()) {
- if (!hasSameSubtaskWithHigherCheckpoint(
- subtaskOffsetMapping.getKey(), offsetTransactionId.getKey())) {
- continue;
- }
- transactionalIdsToAbort.add(offsetTransactionId.getValue());
- }
- }
- }
- return transactionalIdsToAbort;
- }
-
- private boolean hasSameSubtaskWithHigherCheckpoint(
- int openSubtaskIndex, long openCheckpointOffset) {
- return subtaskIdCheckpointOffsetMapping.containsKey(openSubtaskIndex)
- && subtaskIdCheckpointOffsetMapping.get(openSubtaskIndex) <= openCheckpointOffset;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java
deleted file mode 100644
index 7a17b1f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * 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 org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-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.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.source.enumerator.KafkaSourceEnumState;
-import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer;
-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.metrics.KafkaSourceReaderMetrics;
-import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader;
-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.connector.kafka.source.split.KafkaPartitionSplitSerializer;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.util.UserCodeClassLoader;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Properties;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-/**
- * The Source implementation of Kafka. Please use a {@link KafkaSourceBuilder} to construct a {@link
- * KafkaSource}. The following example shows how to create a KafkaSource emitting records of <code>
- * String</code> type.
- *
- * <pre>{@code
- * KafkaSource<String> source = KafkaSource
- * .<String>builder()
- * .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- * .setGroupId("MyGroup")
- * .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- * .setDeserializer(new TestingKafkaRecordDeserializationSchema())
- * .setStartingOffsets(OffsetsInitializer.earliest())
- * .build();
- * }</pre>
- *
- * <p>{@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator} only supports
- * adding new splits and not removing splits in split discovery.
- *
- * <p>See {@link KafkaSourceBuilder} for more details on how to configure this source.
- *
- * @param <OUT> the output type of the source.
- */
-@PublicEvolving
-public class KafkaSource<OUT>
- implements Source<OUT, KafkaPartitionSplit, KafkaSourceEnumState>,
- ResultTypeQueryable<OUT> {
- private static final long serialVersionUID = -8755372893283732098L;
- // Users can choose only one of the following ways to specify the topics to consume from.
- private final KafkaSubscriber subscriber;
- // Users can specify the starting / stopping offset initializer.
- private final OffsetsInitializer startingOffsetsInitializer;
- private final OffsetsInitializer stoppingOffsetsInitializer;
- // Boundedness
- private final Boundedness boundedness;
- private final KafkaRecordDeserializationSchema<OUT> deserializationSchema;
- // The configurations.
- private final Properties props;
-
- KafkaSource(
- KafkaSubscriber subscriber,
- OffsetsInitializer startingOffsetsInitializer,
- @Nullable OffsetsInitializer stoppingOffsetsInitializer,
- Boundedness boundedness,
- KafkaRecordDeserializationSchema<OUT> deserializationSchema,
- Properties props) {
- this.subscriber = subscriber;
- this.startingOffsetsInitializer = startingOffsetsInitializer;
- this.stoppingOffsetsInitializer = stoppingOffsetsInitializer;
- this.boundedness = boundedness;
- this.deserializationSchema = deserializationSchema;
- this.props = props;
- }
-
- /**
- * Get a kafkaSourceBuilder to build a {@link KafkaSource}.
- *
- * @return a Kafka source builder.
- */
- public static <OUT> KafkaSourceBuilder<OUT> builder() {
- return new KafkaSourceBuilder<>();
- }
-
- @Override
- public Boundedness getBoundedness() {
- return this.boundedness;
- }
-
- @Internal
- @Override
- public SourceReader<OUT, KafkaPartitionSplit> createReader(SourceReaderContext readerContext)
- throws Exception {
- return createReader(readerContext, (ignore) -> {});
- }
-
- @VisibleForTesting
- SourceReader<OUT, KafkaPartitionSplit> createReader(
- SourceReaderContext readerContext, Consumer<Collection<String>> splitFinishedHook)
- throws Exception {
- FutureCompletingBlockingQueue<RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>>>
- elementsQueue = new FutureCompletingBlockingQueue<>();
- deserializationSchema.open(
- new DeserializationSchema.InitializationContext() {
- @Override
- public MetricGroup getMetricGroup() {
- return readerContext.metricGroup().addGroup("deserializer");
- }
-
- @Override
- public UserCodeClassLoader getUserCodeClassLoader() {
- return readerContext.getUserCodeClassLoader();
- }
- });
- final KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(readerContext.metricGroup());
-
- Supplier<KafkaPartitionSplitReader> splitReaderSupplier =
- () -> new KafkaPartitionSplitReader(props, readerContext, kafkaSourceReaderMetrics);
- KafkaRecordEmitter<OUT> recordEmitter = new KafkaRecordEmitter<>(deserializationSchema);
-
- return new KafkaSourceReader<>(
- elementsQueue,
- new KafkaSourceFetcherManager(
- elementsQueue, splitReaderSupplier::get, splitFinishedHook),
- recordEmitter,
- toConfiguration(props),
- readerContext,
- kafkaSourceReaderMetrics);
- }
-
- @Internal
- @Override
- public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> createEnumerator(
- SplitEnumeratorContext<KafkaPartitionSplit> enumContext) {
- return new KafkaSourceEnumerator(
- subscriber,
- startingOffsetsInitializer,
- stoppingOffsetsInitializer,
- props,
- enumContext,
- boundedness);
- }
-
- @Internal
- @Override
- public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> restoreEnumerator(
- SplitEnumeratorContext<KafkaPartitionSplit> enumContext,
- KafkaSourceEnumState checkpoint)
- throws IOException {
- return new KafkaSourceEnumerator(
- subscriber,
- startingOffsetsInitializer,
- stoppingOffsetsInitializer,
- props,
- enumContext,
- boundedness,
- checkpoint.assignedPartitions());
- }
-
- @Internal
- @Override
- public SimpleVersionedSerializer<KafkaPartitionSplit> getSplitSerializer() {
- return new KafkaPartitionSplitSerializer();
- }
-
- @Internal
- @Override
- public SimpleVersionedSerializer<KafkaSourceEnumState> getEnumeratorCheckpointSerializer() {
- return new KafkaSourceEnumStateSerializer();
- }
-
- @Override
- public TypeInformation<OUT> getProducedType() {
- return deserializationSchema.getProducedType();
- }
-
- // ----------- private helper methods ---------------
-
- private Configuration toConfiguration(Properties props) {
- Configuration config = new Configuration();
- props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key)));
- return config;
- }
-
- @VisibleForTesting
- Configuration getConfiguration() {
- return toConfiguration(props);
- }
-
- @VisibleForTesting
- KafkaSubscriber getKafkaSubscriber() {
- return subscriber;
- }
-
- @VisibleForTesting
- OffsetsInitializer getStoppingOffsetsInitializer() {
- return stoppingOffsetsInitializer;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java
deleted file mode 100644
index afaa72d..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java
+++ /dev/null
@@ -1,534 +0,0 @@
-/*
- * 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.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.connector.source.Boundedness;
-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.enumerator.initializer.OffsetsInitializerValidator;
-import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber;
-import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * The builder class for {@link KafkaSource} to make it easier for the users to construct a {@link
- * KafkaSource}.
- *
- * <p>The following example shows the minimum setup to create a KafkaSource that reads the String
- * values from a Kafka topic.
- *
- * <pre>{@code
- * KafkaSource<String> source = KafkaSource
- * .<String>builder()
- * .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
- * .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- * .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
- * .build();
- * }</pre>
- *
- * <p>The bootstrap servers, topics/partitions to consume, and the record deserializer are required
- * fields that must be set.
- *
- * <p>To specify the starting offsets of the KafkaSource, one can call {@link
- * #setStartingOffsets(OffsetsInitializer)}.
- *
- * <p>By default the KafkaSource runs in an {@link Boundedness#CONTINUOUS_UNBOUNDED} mode and never
- * stops until the Flink job is canceled or fails. To let the KafkaSource run as {@link
- * Boundedness#CONTINUOUS_UNBOUNDED} yet stop at some given offsets, one can call {@link
- * #setUnbounded(OffsetsInitializer)}. For example the following KafkaSource stops after it consumes
- * up to the latest partition offsets at the point when the Flink job started.
- *
- * <pre>{@code
- * KafkaSource<String> source = KafkaSource
- * .<String>builder()
- * .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
- * .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- * .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
- * .setUnbounded(OffsetsInitializer.latest())
- * .build();
- * }</pre>
- *
- * <p>Check the Java docs of each individual methods to learn more about the settings to build a
- * KafkaSource.
- */
-@PublicEvolving
-public class KafkaSourceBuilder<OUT> {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceBuilder.class);
- private static final String[] REQUIRED_CONFIGS = {ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG};
- // The subscriber specifies the partitions to subscribe to.
- private KafkaSubscriber subscriber;
- // Users can specify the starting / stopping offset initializer.
- private OffsetsInitializer startingOffsetsInitializer;
- private OffsetsInitializer stoppingOffsetsInitializer;
- // Boundedness
- private Boundedness boundedness;
- private KafkaRecordDeserializationSchema<OUT> deserializationSchema;
- // The configurations.
- protected Properties props;
-
- KafkaSourceBuilder() {
- this.subscriber = null;
- this.startingOffsetsInitializer = OffsetsInitializer.earliest();
- this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer();
- this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED;
- this.deserializationSchema = null;
- this.props = new Properties();
- }
-
- /**
- * Sets the bootstrap servers for the KafkaConsumer of the KafkaSource.
- *
- * @param bootstrapServers the bootstrap servers of the Kafka cluster.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setBootstrapServers(String bootstrapServers) {
- return setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- }
-
- /**
- * Sets the consumer group id of the KafkaSource.
- *
- * @param groupId the group id of the KafkaSource.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setGroupId(String groupId) {
- return setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId);
- }
-
- /**
- * Set a list of topics the KafkaSource should consume from. All the topics in the list should
- * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of
- * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead.
- *
- * @param topics the list of topics to consume from.
- * @return this KafkaSourceBuilder.
- * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection)
- */
- public KafkaSourceBuilder<OUT> setTopics(List<String> topics) {
- ensureSubscriberIsNull("topics");
- subscriber = KafkaSubscriber.getTopicListSubscriber(topics);
- return this;
- }
-
- /**
- * Set a list of topics the KafkaSource should consume from. All the topics in the list should
- * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of
- * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead.
- *
- * @param topics the list of topics to consume from.
- * @return this KafkaSourceBuilder.
- * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection)
- */
- public KafkaSourceBuilder<OUT> setTopics(String... topics) {
- return setTopics(Arrays.asList(topics));
- }
-
- /**
- * Set a topic pattern to consume from use the java {@link Pattern}.
- *
- * @param topicPattern the pattern of the topic name to consume from.
- * @return this KafkaSourceBuilder.
- * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Pattern)
- */
- public KafkaSourceBuilder<OUT> setTopicPattern(Pattern topicPattern) {
- ensureSubscriberIsNull("topic pattern");
- subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern);
- return this;
- }
-
- /**
- * Set a set of partitions to consume from.
- *
- * @param partitions the set of partitions to consume from.
- * @return this KafkaSourceBuilder.
- * @see org.apache.kafka.clients.consumer.KafkaConsumer#assign(Collection)
- */
- public KafkaSourceBuilder<OUT> setPartitions(Set<TopicPartition> partitions) {
- ensureSubscriberIsNull("partitions");
- subscriber = KafkaSubscriber.getPartitionSetSubscriber(partitions);
- return this;
- }
-
- /**
- * Set a custom Kafka subscriber to use to discover new splits.
- *
- * @param kafkaSubscriber the {@link KafkaSubscriber} to use for split discovery.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setKafkaSubscriber(KafkaSubscriber kafkaSubscriber) {
- ensureSubscriberIsNull("custom");
- this.subscriber = checkNotNull(kafkaSubscriber);
- return this;
- }
-
- /**
- * Specify from which offsets the KafkaSource should start consuming from by providing an {@link
- * OffsetsInitializer}.
- *
- * <p>The following {@link OffsetsInitializer}s are commonly used and provided out of the box.
- * Users can also implement their own {@link OffsetsInitializer} for custom behaviors.
- *
- * <ul>
- * <li>{@link OffsetsInitializer#earliest()} - starting from the earliest offsets. This is
- * also the default {@link OffsetsInitializer} of the KafkaSource for starting offsets.
- * <li>{@link OffsetsInitializer#latest()} - starting from the latest offsets.
- * <li>{@link OffsetsInitializer#committedOffsets()} - starting from the committed offsets of
- * the consumer group.
- * <li>{@link
- * OffsetsInitializer#committedOffsets(org.apache.kafka.clients.consumer.OffsetResetStrategy)}
- * - starting from the committed offsets of the consumer group. If there is no committed
- * offsets, starting from the offsets specified by the {@link
- * org.apache.kafka.clients.consumer.OffsetResetStrategy OffsetResetStrategy}.
- * <li>{@link OffsetsInitializer#offsets(Map)} - starting from the specified offsets for each
- * partition.
- * <li>{@link OffsetsInitializer#timestamp(long)} - starting from the specified timestamp for
- * each partition. Note that the guarantee here is that all the records in Kafka whose
- * {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater than
- * the given starting timestamp will be consumed. However, it is possible that some
- * consumer records whose timestamp is smaller than the given starting timestamp are also
- * consumed.
- * </ul>
- *
- * @param startingOffsetsInitializer the {@link OffsetsInitializer} setting the starting offsets
- * for the Source.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setStartingOffsets(
- OffsetsInitializer startingOffsetsInitializer) {
- this.startingOffsetsInitializer = startingOffsetsInitializer;
- return this;
- }
-
- /**
- * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus
- * never stops until the Flink job fails or is canceled. To let the KafkaSource run as a
- * streaming source but still stop at some point, one can set an {@link OffsetsInitializer} to
- * specify the stopping offsets for each partition. When all the partitions have reached their
- * stopping offsets, the KafkaSource will then exit.
- *
- * <p>This method is different from {@link #setBounded(OffsetsInitializer)} in that after
- * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will
- * still return {@link Boundedness#CONTINUOUS_UNBOUNDED} even though it will stop at the
- * stopping offsets specified by the stopping offsets {@link OffsetsInitializer}.
- *
- * <p>The following {@link OffsetsInitializer} are commonly used and provided out of the box.
- * Users can also implement their own {@link OffsetsInitializer} for custom behaviors.
- *
- * <ul>
- * <li>{@link OffsetsInitializer#latest()} - stop at the latest offsets of the partitions when
- * the KafkaSource starts to run.
- * <li>{@link OffsetsInitializer#committedOffsets()} - stops at the committed offsets of the
- * consumer group.
- * <li>{@link OffsetsInitializer#offsets(Map)} - stops at the specified offsets for each
- * partition.
- * <li>{@link OffsetsInitializer#timestamp(long)} - stops at the specified timestamp for each
- * partition. The guarantee of setting the stopping timestamp is that no Kafka records
- * whose {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater
- * than the given stopping timestamp will be consumed. However, it is possible that some
- * records whose timestamp is smaller than the specified stopping timestamp are not
- * consumed.
- * </ul>
- *
- * @param stoppingOffsetsInitializer The {@link OffsetsInitializer} to specify the stopping
- * offset.
- * @return this KafkaSourceBuilder.
- * @see #setBounded(OffsetsInitializer)
- */
- public KafkaSourceBuilder<OUT> setUnbounded(OffsetsInitializer stoppingOffsetsInitializer) {
- this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED;
- this.stoppingOffsetsInitializer = stoppingOffsetsInitializer;
- return this;
- }
-
- /**
- * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus
- * never stops until the Flink job fails or is canceled. To let the KafkaSource run as {@link
- * Boundedness#BOUNDED} and stop at some point, one can set an {@link OffsetsInitializer} to
- * specify the stopping offsets for each partition. When all the partitions have reached their
- * stopping offsets, the KafkaSource will then exit.
- *
- * <p>This method is different from {@link #setUnbounded(OffsetsInitializer)} in that after
- * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will
- * return {@link Boundedness#BOUNDED} instead of {@link Boundedness#CONTINUOUS_UNBOUNDED}.
- *
- * <p>The following {@link OffsetsInitializer} are commonly used and provided out of the box.
- * Users can also implement their own {@link OffsetsInitializer} for custom behaviors.
- *
- * <ul>
- * <li>{@link OffsetsInitializer#latest()} - stop at the latest offsets of the partitions when
- * the KafkaSource starts to run.
- * <li>{@link OffsetsInitializer#committedOffsets()} - stops at the committed offsets of the
- * consumer group.
- * <li>{@link OffsetsInitializer#offsets(Map)} - stops at the specified offsets for each
- * partition.
- * <li>{@link OffsetsInitializer#timestamp(long)} - stops at the specified timestamp for each
- * partition. The guarantee of setting the stopping timestamp is that no Kafka records
- * whose {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater
- * than the given stopping timestamp will be consumed. However, it is possible that some
- * records whose timestamp is smaller than the specified stopping timestamp are not
- * consumed.
- * </ul>
- *
- * @param stoppingOffsetsInitializer the {@link OffsetsInitializer} to specify the stopping
- * offsets.
- * @return this KafkaSourceBuilder.
- * @see #setUnbounded(OffsetsInitializer)
- */
- public KafkaSourceBuilder<OUT> setBounded(OffsetsInitializer stoppingOffsetsInitializer) {
- this.boundedness = Boundedness.BOUNDED;
- this.stoppingOffsetsInitializer = stoppingOffsetsInitializer;
- return this;
- }
-
- /**
- * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link
- * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource.
- *
- * @param recordDeserializer the deserializer for Kafka {@link
- * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord}.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setDeserializer(
- KafkaRecordDeserializationSchema<OUT> recordDeserializer) {
- this.deserializationSchema = recordDeserializer;
- return this;
- }
-
- /**
- * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link
- * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource. The given
- * {@link DeserializationSchema} will be used to deserialize the value of ConsumerRecord. The
- * other information (e.g. key) in a ConsumerRecord will be ignored.
- *
- * @param deserializationSchema the {@link DeserializationSchema} to use for deserialization.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setValueOnlyDeserializer(
- DeserializationSchema<OUT> deserializationSchema) {
- this.deserializationSchema =
- KafkaRecordDeserializationSchema.valueOnly(deserializationSchema);
- return this;
- }
-
- /**
- * Sets the client id prefix of this KafkaSource.
- *
- * @param prefix the client id prefix to use for this KafkaSource.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setClientIdPrefix(String prefix) {
- return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix);
- }
-
- /**
- * Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found
- * in {@link ConsumerConfig} and {@link KafkaSourceOptions}.
- *
- * <p>Note that the following keys will be overridden by the builder when the KafkaSource is
- * created.
- *
- * <ul>
- * <li><code>key.deserializer</code> is always set to {@link ByteArrayDeserializer}.
- * <li><code>value.deserializer</code> is always set to {@link ByteArrayDeserializer}.
- * <li><code>auto.offset.reset.strategy</code> is overridden by {@link
- * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by
- * default {@link OffsetsInitializer#earliest()}.
- * <li><code>partition.discovery.interval.ms</code> is overridden to -1 when {@link
- * #setBounded(OffsetsInitializer)} has been invoked.
- * </ul>
- *
- * @param key the key of the property.
- * @param value the value of the property.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setProperty(String key, String value) {
- props.setProperty(key, value);
- return this;
- }
-
- /**
- * Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found
- * in {@link ConsumerConfig} and {@link KafkaSourceOptions}.
- *
- * <p>Note that the following keys will be overridden by the builder when the KafkaSource is
- * created.
- *
- * <ul>
- * <li><code>key.deserializer</code> is always set to {@link ByteArrayDeserializer}.
- * <li><code>value.deserializer</code> is always set to {@link ByteArrayDeserializer}.
- * <li><code>auto.offset.reset.strategy</code> is overridden by {@link
- * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by
- * default {@link OffsetsInitializer#earliest()}.
- * <li><code>partition.discovery.interval.ms</code> is overridden to -1 when {@link
- * #setBounded(OffsetsInitializer)} has been invoked.
- * <li><code>client.id</code> is overridden to the "client.id.prefix-RANDOM_LONG", or
- * "group.id-RANDOM_LONG" if the client id prefix is not set.
- * </ul>
- *
- * @param props the properties to set for the KafkaSource.
- * @return this KafkaSourceBuilder.
- */
- public KafkaSourceBuilder<OUT> setProperties(Properties props) {
- this.props.putAll(props);
- return this;
- }
-
- /**
- * Build the {@link KafkaSource}.
- *
- * @return a KafkaSource with the settings made for this builder.
- */
- public KafkaSource<OUT> build() {
- sanityCheck();
- parseAndSetRequiredProperties();
- return new KafkaSource<>(
- subscriber,
- startingOffsetsInitializer,
- stoppingOffsetsInitializer,
- boundedness,
- deserializationSchema,
- props);
- }
-
- // ------------- private helpers --------------
-
- private void ensureSubscriberIsNull(String attemptingSubscribeMode) {
- if (subscriber != null) {
- throw new IllegalStateException(
- String.format(
- "Cannot use %s for consumption because a %s is already set for consumption.",
- attemptingSubscribeMode, subscriber.getClass().getSimpleName()));
- }
- }
-
- private void parseAndSetRequiredProperties() {
- 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)) {
- LOG.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 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)
- : "KafkaSource-" + new Random().nextLong(),
- false);
- }
-
- private boolean maybeOverride(String key, String value, boolean override) {
- boolean overridden = false;
- String userValue = props.getProperty(key);
- if (userValue != null) {
- if (override) {
- LOG.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() {
- // Check required configs.
- for (String requiredConfig : REQUIRED_CONFIGS) {
- checkNotNull(
- props.getProperty(requiredConfig),
- String.format("Property %s is required but not provided", requiredConfig));
- }
- // Check required settings.
- checkNotNull(
- subscriber,
- "No subscribe mode is specified, "
- + "should be one of topics, topic pattern and partition set.");
- checkNotNull(deserializationSchema, "Deserialization schema is required but not provided.");
- // Check consumer group ID
- checkState(
- props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(),
- String.format(
- "Property %s is required when offset commit is enabled",
- ConsumerConfig.GROUP_ID_CONFIG));
- // Check offsets initializers
- if (startingOffsetsInitializer instanceof OffsetsInitializerValidator) {
- ((OffsetsInitializerValidator) startingOffsetsInitializer).validate(props);
- }
- if (stoppingOffsetsInitializer instanceof OffsetsInitializerValidator) {
- ((OffsetsInitializerValidator) stoppingOffsetsInitializer).validate(props);
- }
- }
-
- 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;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java
deleted file mode 100644
index 248014c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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 org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-
-import java.util.Properties;
-import java.util.function.Function;
-
-/** Configurations for KafkaSource. */
-@Internal
-public class KafkaSourceOptions {
-
- public static final ConfigOption<String> CLIENT_ID_PREFIX =
- ConfigOptions.key("client.id.prefix")
- .stringType()
- .noDefaultValue()
- .withDescription("The prefix to use for the Kafka consumers.");
-
- public static final ConfigOption<Long> PARTITION_DISCOVERY_INTERVAL_MS =
- ConfigOptions.key("partition.discovery.interval.ms")
- .longType()
- .noDefaultValue()
- .withDescription(
- "The interval in milliseconds for the Kafka source to discover "
- + "the new partitions. A non-positive value disables the partition discovery.");
-
- public static final ConfigOption<Boolean> REGISTER_KAFKA_CONSUMER_METRICS =
- ConfigOptions.key("register.consumer.metrics")
- .booleanType()
- .defaultValue(true)
- .withDescription(
- "Whether to register metrics of KafkaConsumer into Flink metric group");
-
- public static final ConfigOption<Boolean> COMMIT_OFFSETS_ON_CHECKPOINT =
- ConfigOptions.key("commit.offsets.on.checkpoint")
- .booleanType()
- .defaultValue(true)
- .withDescription("Whether to commit consuming offset on checkpoint.");
-
- @SuppressWarnings("unchecked")
- public static <T> T getOption(
- Properties props, ConfigOption<?> configOption, Function<String, T> parser) {
- String value = props.getProperty(configOption.key());
- return (T) (value == null ? configOption.defaultValue() : parser.apply(value));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java
deleted file mode 100644
index 04604ab..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.enumerator;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Set;
-
-/** The state of Kafka source enumerator. */
-@Internal
-public class KafkaSourceEnumState {
- private final Set<TopicPartition> assignedPartitions;
-
- KafkaSourceEnumState(Set<TopicPartition> assignedPartitions) {
- this.assignedPartitions = assignedPartitions;
- }
-
- public Set<TopicPartition> assignedPartitions() {
- return assignedPartitions;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java
deleted file mode 100644
index 7428f42..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.enumerator;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.base.source.utils.SerdeUtils;
-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 org.apache.kafka.common.TopicPartition;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * The {@link org.apache.flink.core.io.SimpleVersionedSerializer Serializer} for the enumerator
- * state of Kafka source.
- */
-@Internal
-public class KafkaSourceEnumStateSerializer
- implements SimpleVersionedSerializer<KafkaSourceEnumState> {
-
- private static final int VERSION_0 = 0;
- private static final int VERSION_1 = 1;
-
- private static final int CURRENT_VERSION = VERSION_1;
-
- @Override
- public int getVersion() {
- return CURRENT_VERSION;
- }
-
- @Override
- public byte[] serialize(KafkaSourceEnumState enumState) throws IOException {
- return serializeTopicPartitions(enumState.assignedPartitions());
- }
-
- @Override
- public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
- if (version == CURRENT_VERSION) {
- final Set<TopicPartition> assignedPartitions = deserializeTopicPartitions(serialized);
- return new KafkaSourceEnumState(assignedPartitions);
- }
-
- // Backward compatibility
- if (version == VERSION_0) {
- Map<Integer, Set<KafkaPartitionSplit>> currentPartitionAssignment =
- SerdeUtils.deserializeSplitAssignments(
- serialized, new KafkaPartitionSplitSerializer(), HashSet::new);
- Set<TopicPartition> currentAssignedSplits = new HashSet<>();
- currentPartitionAssignment.forEach(
- (reader, splits) ->
- splits.forEach(
- split -> currentAssignedSplits.add(split.getTopicPartition())));
- return new KafkaSourceEnumState(currentAssignedSplits);
- }
-
- throw new IOException(
- String.format(
- "The bytes are serialized with version %d, "
- + "while this deserializer only supports version up to %d",
- version, CURRENT_VERSION));
- }
-
- private static byte[] serializeTopicPartitions(Collection<TopicPartition> topicPartitions)
- throws IOException {
- try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(baos)) {
-
- out.writeInt(topicPartitions.size());
- for (TopicPartition tp : topicPartitions) {
- out.writeUTF(tp.topic());
- out.writeInt(tp.partition());
- }
- out.flush();
-
- return baos.toByteArray();
- }
- }
-
- private static Set<TopicPartition> deserializeTopicPartitions(byte[] serializedTopicPartitions)
- throws IOException {
- try (ByteArrayInputStream bais = new ByteArrayInputStream(serializedTopicPartitions);
- DataInputStream in = new DataInputStream(bais)) {
-
- final int numPartitions = in.readInt();
- Set<TopicPartition> topicPartitions = new HashSet<>(numPartitions);
- for (int i = 0; i < numPartitions; i++) {
- final String topic = in.readUTF();
- final int partition = in.readInt();
- topicPartitions.add(new TopicPartition(topic, partition));
- }
- if (in.available() > 0) {
- throw new IOException("Unexpected trailing bytes in serialized topic partitions");
- }
-
- return topicPartitions;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java
deleted file mode 100644
index 9cf233c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java
+++ /dev/null
@@ -1,634 +0,0 @@
-/*
- * 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.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.SplitEnumerator;
-import org.apache.flink.api.connector.source.SplitEnumeratorContext;
-import org.apache.flink.api.connector.source.SplitsAssignment;
-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.enumerator.subscriber.KafkaSubscriber;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.KafkaAdminClient;
-import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
-import org.apache.kafka.clients.admin.ListOffsetsResult;
-import org.apache.kafka.clients.admin.OffsetSpec;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.time.Duration;
-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.ExecutionException;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-/** The enumerator class for Kafka source. */
-@Internal
-public class KafkaSourceEnumerator
- implements SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceEnumerator.class);
- private final KafkaSubscriber subscriber;
- private final OffsetsInitializer startingOffsetInitializer;
- private final OffsetsInitializer stoppingOffsetInitializer;
- private final Properties properties;
- private final long partitionDiscoveryIntervalMs;
- private final SplitEnumeratorContext<KafkaPartitionSplit> context;
- private final Boundedness boundedness;
-
- /** Partitions that have been assigned to readers. */
- private final Set<TopicPartition> assignedPartitions;
-
- /**
- * The discovered and initialized partition splits that are waiting for owner reader to be
- * ready.
- */
- private final Map<Integer, Set<KafkaPartitionSplit>> pendingPartitionSplitAssignment;
-
- /** The consumer group id used for this KafkaSource. */
- private final String consumerGroupId;
-
- // Lazily instantiated or mutable fields.
- private AdminClient adminClient;
-
- // This flag will be marked as true if periodically partition discovery is disabled AND the
- // initializing partition discovery has finished.
- private boolean noMoreNewPartitionSplits = false;
-
- public KafkaSourceEnumerator(
- KafkaSubscriber subscriber,
- OffsetsInitializer startingOffsetInitializer,
- OffsetsInitializer stoppingOffsetInitializer,
- Properties properties,
- SplitEnumeratorContext<KafkaPartitionSplit> context,
- Boundedness boundedness) {
- this(
- subscriber,
- startingOffsetInitializer,
- stoppingOffsetInitializer,
- properties,
- context,
- boundedness,
- Collections.emptySet());
- }
-
- public KafkaSourceEnumerator(
- KafkaSubscriber subscriber,
- OffsetsInitializer startingOffsetInitializer,
- OffsetsInitializer stoppingOffsetInitializer,
- Properties properties,
- SplitEnumeratorContext<KafkaPartitionSplit> context,
- Boundedness boundedness,
- Set<TopicPartition> assignedPartitions) {
- this.subscriber = subscriber;
- this.startingOffsetInitializer = startingOffsetInitializer;
- this.stoppingOffsetInitializer = stoppingOffsetInitializer;
- this.properties = properties;
- this.context = context;
- this.boundedness = boundedness;
-
- this.assignedPartitions = new HashSet<>(assignedPartitions);
- this.pendingPartitionSplitAssignment = new HashMap<>();
- this.partitionDiscoveryIntervalMs =
- KafkaSourceOptions.getOption(
- properties,
- KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS,
- Long::parseLong);
- this.consumerGroupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
- }
-
- /**
- * Start the enumerator.
- *
- * <p>Depending on {@link #partitionDiscoveryIntervalMs}, the enumerator will trigger a one-time
- * partition discovery, or schedule a callable for discover partitions periodically.
- *
- * <p>The invoking chain of partition discovery would be:
- *
- * <ol>
- * <li>{@link #getSubscribedTopicPartitions} in worker thread
- * <li>{@link #checkPartitionChanges} in coordinator thread
- * <li>{@link #initializePartitionSplits} in worker thread
- * <li>{@link #handlePartitionSplitChanges} in coordinator thread
- * </ol>
- */
- @Override
- public void start() {
- adminClient = getKafkaAdminClient();
- if (partitionDiscoveryIntervalMs > 0) {
- LOG.info(
- "Starting the KafkaSourceEnumerator for consumer group {} "
- + "with partition discovery interval of {} ms.",
- consumerGroupId,
- partitionDiscoveryIntervalMs);
- context.callAsync(
- this::getSubscribedTopicPartitions,
- this::checkPartitionChanges,
- 0,
- partitionDiscoveryIntervalMs);
- } else {
- LOG.info(
- "Starting the KafkaSourceEnumerator for consumer group {} "
- + "without periodic partition discovery.",
- consumerGroupId);
- context.callAsync(this::getSubscribedTopicPartitions, this::checkPartitionChanges);
- }
- }
-
- @Override
- public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
- // the kafka source pushes splits eagerly, rather than act upon split requests
- }
-
- @Override
- public void addSplitsBack(List<KafkaPartitionSplit> splits, int subtaskId) {
- addPartitionSplitChangeToPendingAssignments(splits);
-
- // If the failed subtask has already restarted, we need to assign pending splits to it
- if (context.registeredReaders().containsKey(subtaskId)) {
- assignPendingPartitionSplits(Collections.singleton(subtaskId));
- }
- }
-
- @Override
- public void addReader(int subtaskId) {
- LOG.debug(
- "Adding reader {} to KafkaSourceEnumerator for consumer group {}.",
- subtaskId,
- consumerGroupId);
- assignPendingPartitionSplits(Collections.singleton(subtaskId));
- }
-
- @Override
- public KafkaSourceEnumState snapshotState(long checkpointId) throws Exception {
- return new KafkaSourceEnumState(assignedPartitions);
- }
-
- @Override
- public void close() {
- if (adminClient != null) {
- adminClient.close();
- }
- }
-
- // ----------------- private methods -------------------
-
- /**
- * List subscribed topic partitions on Kafka brokers.
- *
- * <p>NOTE: This method should only be invoked in the worker executor thread, because it
- * requires network I/O with Kafka brokers.
- *
- * @return Set of subscribed {@link TopicPartition}s
- */
- private Set<TopicPartition> getSubscribedTopicPartitions() {
- return subscriber.getSubscribedTopicPartitions(adminClient);
- }
-
- /**
- * Check if there's any partition changes within subscribed topic partitions fetched by worker
- * thread, and invoke {@link KafkaSourceEnumerator#initializePartitionSplits(PartitionChange)}
- * in worker thread to initialize splits for new partitions.
- *
- * <p>NOTE: This method should only be invoked in the coordinator executor thread.
- *
- * @param fetchedPartitions Map from topic name to its description
- * @param t Exception in worker thread
- */
- private void checkPartitionChanges(Set<TopicPartition> fetchedPartitions, Throwable t) {
- if (t != null) {
- throw new FlinkRuntimeException(
- "Failed to list subscribed topic partitions due to ", t);
- }
- final PartitionChange partitionChange = getPartitionChange(fetchedPartitions);
- if (partitionChange.isEmpty()) {
- return;
- }
- context.callAsync(
- () -> initializePartitionSplits(partitionChange),
- this::handlePartitionSplitChanges);
- }
-
- /**
- * Initialize splits for newly discovered partitions.
- *
- * <p>Enumerator will be responsible for fetching offsets when initializing splits if:
- *
- * <ul>
- * <li>using timestamp for initializing offset
- * <li>or using specified offset, but the offset is not provided for the newly discovered
- * partitions
- * </ul>
- *
- * <p>Otherwise offsets will be initialized by readers.
- *
- * <p>NOTE: This method should only be invoked in the worker executor thread, because it
- * potentially requires network I/O with Kafka brokers for fetching offsets.
- *
- * @param partitionChange Newly discovered and removed partitions
- * @return {@link KafkaPartitionSplit} of new partitions and {@link TopicPartition} of removed
- * partitions
- */
- private PartitionSplitChange initializePartitionSplits(PartitionChange partitionChange) {
- Set<TopicPartition> newPartitions =
- Collections.unmodifiableSet(partitionChange.getNewPartitions());
- OffsetsInitializer.PartitionOffsetsRetriever offsetsRetriever = getOffsetsRetriever();
-
- Map<TopicPartition, Long> startingOffsets =
- startingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever);
- Map<TopicPartition, Long> stoppingOffsets =
- stoppingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever);
-
- Set<KafkaPartitionSplit> partitionSplits = new HashSet<>(newPartitions.size());
- for (TopicPartition tp : newPartitions) {
- Long startingOffset = startingOffsets.get(tp);
- long stoppingOffset =
- stoppingOffsets.getOrDefault(tp, KafkaPartitionSplit.NO_STOPPING_OFFSET);
- partitionSplits.add(new KafkaPartitionSplit(tp, startingOffset, stoppingOffset));
- }
- return new PartitionSplitChange(partitionSplits, partitionChange.getRemovedPartitions());
- }
-
- /**
- * Mark partition splits initialized by {@link
- * KafkaSourceEnumerator#initializePartitionSplits(PartitionChange)} as pending and try to
- * assign pending splits to registered readers.
- *
- * <p>NOTE: This method should only be invoked in the coordinator executor thread.
- *
- * @param partitionSplitChange Partition split changes
- * @param t Exception in worker thread
- */
- private void handlePartitionSplitChanges(
- PartitionSplitChange partitionSplitChange, Throwable t) {
- if (t != null) {
- throw new FlinkRuntimeException("Failed to initialize partition splits due to ", t);
- }
- if (partitionDiscoveryIntervalMs < 0) {
- LOG.debug("Partition discovery is disabled.");
- noMoreNewPartitionSplits = true;
- }
- // TODO: Handle removed partitions.
- addPartitionSplitChangeToPendingAssignments(partitionSplitChange.newPartitionSplits);
- assignPendingPartitionSplits(context.registeredReaders().keySet());
- }
-
- // This method should only be invoked in the coordinator executor thread.
- private void addPartitionSplitChangeToPendingAssignments(
- Collection<KafkaPartitionSplit> newPartitionSplits) {
- int numReaders = context.currentParallelism();
- for (KafkaPartitionSplit split : newPartitionSplits) {
- int ownerReader = getSplitOwner(split.getTopicPartition(), numReaders);
- pendingPartitionSplitAssignment
- .computeIfAbsent(ownerReader, r -> new HashSet<>())
- .add(split);
- }
- LOG.debug(
- "Assigned {} to {} readers of consumer group {}.",
- newPartitionSplits,
- numReaders,
- consumerGroupId);
- }
-
- // This method should only be invoked in the coordinator executor thread.
- private void assignPendingPartitionSplits(Set<Integer> pendingReaders) {
- Map<Integer, List<KafkaPartitionSplit>> incrementalAssignment = new HashMap<>();
-
- // Check if there's any pending splits for given readers
- for (int pendingReader : pendingReaders) {
- checkReaderRegistered(pendingReader);
-
- // Remove pending assignment for the reader
- final Set<KafkaPartitionSplit> pendingAssignmentForReader =
- pendingPartitionSplitAssignment.remove(pendingReader);
-
- if (pendingAssignmentForReader != null && !pendingAssignmentForReader.isEmpty()) {
- // Put pending assignment into incremental assignment
- incrementalAssignment
- .computeIfAbsent(pendingReader, (ignored) -> new ArrayList<>())
- .addAll(pendingAssignmentForReader);
-
- // Mark pending partitions as already assigned
- pendingAssignmentForReader.forEach(
- split -> assignedPartitions.add(split.getTopicPartition()));
- }
- }
-
- // Assign pending splits to readers
- if (!incrementalAssignment.isEmpty()) {
- LOG.info("Assigning splits to readers {}", incrementalAssignment);
- context.assignSplits(new SplitsAssignment<>(incrementalAssignment));
- }
-
- // If periodically partition discovery is disabled and the initializing discovery has done,
- // signal NoMoreSplitsEvent to pending readers
- if (noMoreNewPartitionSplits && boundedness == Boundedness.BOUNDED) {
- LOG.debug(
- "No more KafkaPartitionSplits to assign. Sending NoMoreSplitsEvent to reader {}"
- + " in consumer group {}.",
- pendingReaders,
- consumerGroupId);
- pendingReaders.forEach(context::signalNoMoreSplits);
- }
- }
-
- private void checkReaderRegistered(int readerId) {
- if (!context.registeredReaders().containsKey(readerId)) {
- throw new IllegalStateException(
- String.format("Reader %d is not registered to source coordinator", readerId));
- }
- }
-
- @VisibleForTesting
- PartitionChange getPartitionChange(Set<TopicPartition> fetchedPartitions) {
- final Set<TopicPartition> removedPartitions = new HashSet<>();
- Consumer<TopicPartition> dedupOrMarkAsRemoved =
- (tp) -> {
- if (!fetchedPartitions.remove(tp)) {
- removedPartitions.add(tp);
- }
- };
-
- assignedPartitions.forEach(dedupOrMarkAsRemoved);
- pendingPartitionSplitAssignment.forEach(
- (reader, splits) ->
- splits.forEach(
- split -> dedupOrMarkAsRemoved.accept(split.getTopicPartition())));
-
- if (!fetchedPartitions.isEmpty()) {
- LOG.info("Discovered new partitions: {}", fetchedPartitions);
- }
- if (!removedPartitions.isEmpty()) {
- LOG.info("Discovered removed partitions: {}", removedPartitions);
- }
-
- return new PartitionChange(fetchedPartitions, removedPartitions);
- }
-
- private AdminClient getKafkaAdminClient() {
- Properties adminClientProps = new Properties();
- deepCopyProperties(properties, adminClientProps);
- // set client id prefix
- String clientIdPrefix =
- adminClientProps.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key());
- adminClientProps.setProperty(
- ConsumerConfig.CLIENT_ID_CONFIG, clientIdPrefix + "-enumerator-admin-client");
- return AdminClient.create(adminClientProps);
- }
-
- private OffsetsInitializer.PartitionOffsetsRetriever getOffsetsRetriever() {
- String groupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
- return new PartitionOffsetsRetrieverImpl(adminClient, groupId);
- }
-
- /**
- * Returns the index of the target subtask that a specific Kafka partition should be assigned
- * to.
- *
- * <p>The resulting distribution of partitions of a single topic has the following contract:
- *
- * <ul>
- * <li>1. Uniformly distributed across subtasks
- * <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
- * indices) by using the partition id as the offset from a starting index (i.e., the index
- * of the subtask which partition 0 of the topic will be assigned to, determined using the
- * topic name).
- * </ul>
- *
- * @param tp the Kafka partition to assign.
- * @param numReaders the total number of readers.
- * @return the id of the subtask that owns the split.
- */
- @VisibleForTesting
- static int getSplitOwner(TopicPartition tp, int numReaders) {
- int startIndex = ((tp.topic().hashCode() * 31) & 0x7FFFFFFF) % numReaders;
-
- // here, the assumption is that the id of Kafka partitions are always ascending
- // starting from 0, and therefore can be used directly as the offset clockwise from the
- // start index
- return (startIndex + tp.partition()) % numReaders;
- }
-
- @VisibleForTesting
- static void deepCopyProperties(Properties from, Properties to) {
- for (String key : from.stringPropertyNames()) {
- to.setProperty(key, from.getProperty(key));
- }
- }
-
- // --------------- private class ---------------
-
- /** A container class to hold the newly added partitions and removed partitions. */
- @VisibleForTesting
- static class PartitionChange {
- private final Set<TopicPartition> newPartitions;
- private final Set<TopicPartition> removedPartitions;
-
- PartitionChange(Set<TopicPartition> newPartitions, Set<TopicPartition> removedPartitions) {
- this.newPartitions = newPartitions;
- this.removedPartitions = removedPartitions;
- }
-
- public Set<TopicPartition> getNewPartitions() {
- return newPartitions;
- }
-
- public Set<TopicPartition> getRemovedPartitions() {
- return removedPartitions;
- }
-
- public boolean isEmpty() {
- return newPartitions.isEmpty() && removedPartitions.isEmpty();
- }
- }
-
- private static class PartitionSplitChange {
- private final Set<KafkaPartitionSplit> newPartitionSplits;
- private final Set<TopicPartition> removedPartitions;
-
- private PartitionSplitChange(
- Set<KafkaPartitionSplit> newPartitionSplits,
- Set<TopicPartition> removedPartitions) {
- this.newPartitionSplits = Collections.unmodifiableSet(newPartitionSplits);
- this.removedPartitions = Collections.unmodifiableSet(removedPartitions);
- }
- }
-
- /** The implementation for offsets retriever with a consumer and an admin client. */
- @VisibleForTesting
- public static class PartitionOffsetsRetrieverImpl
- implements OffsetsInitializer.PartitionOffsetsRetriever, AutoCloseable {
- private final AdminClient adminClient;
- private final String groupId;
-
- public PartitionOffsetsRetrieverImpl(AdminClient adminClient, String groupId) {
- this.adminClient = adminClient;
- this.groupId = groupId;
- }
-
- @Override
- public Map<TopicPartition, Long> committedOffsets(Collection<TopicPartition> partitions) {
- ListConsumerGroupOffsetsOptions options =
- new ListConsumerGroupOffsetsOptions()
- .topicPartitions(new ArrayList<>(partitions));
- try {
- return adminClient
- .listConsumerGroupOffsets(groupId, options)
- .partitionsToOffsetAndMetadata()
- .thenApply(
- result -> {
- Map<TopicPartition, Long> offsets = new HashMap<>();
- result.forEach(
- (tp, oam) -> {
- if (oam != null) {
- offsets.put(tp, oam.offset());
- }
- });
- return offsets;
- })
- .get();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new FlinkRuntimeException(
- "Interrupted while listing offsets for consumer group " + groupId, e);
- } catch (ExecutionException e) {
- throw new FlinkRuntimeException(
- "Failed to fetch committed offsets for consumer group "
- + groupId
- + " due to",
- e);
- }
- }
-
- /**
- * List offsets for the specified partitions and OffsetSpec. This operation enables to find
- * the beginning offset, end offset as well as the offset matching a timestamp in
- * partitions.
- *
- * @see KafkaAdminClient#listOffsets(Map)
- * @param topicPartitionOffsets The mapping from partition to the OffsetSpec to look up.
- * @return The list offsets result.
- */
- private Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo> listOffsets(
- Map<TopicPartition, OffsetSpec> topicPartitionOffsets) {
- try {
- return adminClient
- .listOffsets(topicPartitionOffsets)
- .all()
- .thenApply(
- result -> {
- Map<TopicPartition, ListOffsetsResult.ListOffsetsResultInfo>
- offsets = new HashMap<>();
- result.forEach(
- (tp, listOffsetsResultInfo) -> {
- if (listOffsetsResultInfo != null) {
- offsets.put(tp, listOffsetsResultInfo);
- }
- });
- return offsets;
- })
- .get();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new FlinkRuntimeException(
- "Interrupted while listing offsets for topic partitions: "
- + topicPartitionOffsets,
- e);
- } catch (ExecutionException e) {
- throw new FlinkRuntimeException(
- "Failed to list offsets for topic partitions: "
- + topicPartitionOffsets
- + " due to",
- e);
- }
- }
-
- private Map<TopicPartition, Long> listOffsets(
- Collection<TopicPartition> partitions, OffsetSpec offsetSpec) {
- return listOffsets(
- partitions.stream()
- .collect(
- Collectors.toMap(
- partition -> partition, __ -> offsetSpec)))
- .entrySet().stream()
- .collect(
- Collectors.toMap(
- Map.Entry::getKey, entry -> entry.getValue().offset()));
- }
-
- @Override
- public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions) {
- return listOffsets(partitions, OffsetSpec.latest());
- }
-
- @Override
- public Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions) {
- return listOffsets(partitions, OffsetSpec.earliest());
- }
-
- @Override
- public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(
- Map<TopicPartition, Long> timestampsToSearch) {
- return listOffsets(
- timestampsToSearch.entrySet().stream()
- .collect(
- Collectors.toMap(
- Map.Entry::getKey,
- entry ->
- OffsetSpec.forTimestamp(
- entry.getValue()))))
- .entrySet().stream()
- // OffsetAndTimestamp cannot be initialized with a negative offset, which is
- // possible if the timestamp does not correspond to an offset and the topic
- // partition is empty
- .filter(entry -> entry.getValue().offset() >= 0)
- .collect(
- Collectors.toMap(
- Map.Entry::getKey,
- entry ->
- new OffsetAndTimestamp(
- entry.getValue().offset(),
- entry.getValue().timestamp(),
- entry.getValue().leaderEpoch())));
- }
-
- @Override
- public void close() throws Exception {
- adminClient.close(Duration.ZERO);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/NoStoppingOffsetsInitializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/NoStoppingOffsetsInitializer.java
deleted file mode 100644
index bb94d16..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/NoStoppingOffsetsInitializer.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * An implementation of {@link OffsetsInitializer} which does not initialize anything.
- *
- * <p>This class is used as the default stopping offsets initializer for unbounded Kafka sources.
- */
-@Internal
-public class NoStoppingOffsetsInitializer implements OffsetsInitializer {
- private static final long serialVersionUID = 4186323669290142732L;
-
- @Override
- public Map<TopicPartition, Long> getPartitionOffsets(
- Collection<TopicPartition> partitions,
- PartitionOffsetsRetriever partitionOffsetsRetriever) {
- return Collections.emptyMap();
- }
-
- @Override
- public OffsetResetStrategy getAutoOffsetResetStrategy() {
- throw new UnsupportedOperationException(
- "The NoStoppingOffsetsInitializer does not have an OffsetResetStrategy. It should only be used "
- + "to end offset.");
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java
deleted file mode 100644
index db682c6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-
-import org.apache.kafka.clients.admin.KafkaAdminClient;
-import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
-import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * An interface for users to specify the starting / stopping offset of a {@link
- * KafkaPartitionSplit}.
- *
- * @see ReaderHandledOffsetsInitializer
- * @see SpecifiedOffsetsInitializer
- * @see TimestampOffsetsInitializer
- */
-@PublicEvolving
-public interface OffsetsInitializer extends Serializable {
-
- /**
- * Get the initial offsets for the given Kafka partitions. These offsets will be used as either
- * starting offsets or stopping offsets of the Kafka partitions.
- *
- * <p>If the implementation returns a starting offset which causes {@code
- * OffsetsOutOfRangeException} from Kafka. The {@link OffsetResetStrategy} provided by the
- * {@link #getAutoOffsetResetStrategy()} will be used to reset the offset.
- *
- * @param partitions the Kafka partitions to get the starting offsets.
- * @param partitionOffsetsRetriever a helper to retrieve information of the Kafka partitions.
- * @return A mapping from Kafka partition to their offsets to start consuming from.
- */
- Map<TopicPartition, Long> getPartitionOffsets(
- Collection<TopicPartition> partitions,
- PartitionOffsetsRetriever partitionOffsetsRetriever);
-
- /**
- * Get the auto offset reset strategy in case the initialized offsets falls out of the range.
- *
- * <p>The OffsetStrategy is only used when the offset initializer is used to initialize the
- * starting offsets and the starting offsets is out of range.
- *
- * @return An {@link OffsetResetStrategy} to use if the initialized offsets are out of the
- * range.
- */
- OffsetResetStrategy getAutoOffsetResetStrategy();
-
- /**
- * An interface that provides necessary information to the {@link OffsetsInitializer} to get the
- * initial offsets of the Kafka partitions.
- */
- interface PartitionOffsetsRetriever {
-
- /**
- * The group id should be the set for {@link KafkaSource KafkaSource} before invoking this
- * method. Otherwise an {@code IllegalStateException} will be thrown.
- *
- * @see KafkaAdminClient#listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)
- * @throws IllegalStateException if the group id is not set for the {@code KafkaSource}.
- */
- Map<TopicPartition, Long> committedOffsets(Collection<TopicPartition> partitions);
-
- /** List end offsets for the specified partitions. */
- Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions);
-
- /** List beginning offsets for the specified partitions. */
- Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions);
-
- /** List offsets matching a timestamp for the specified partitions. */
- Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(
- Map<TopicPartition, Long> timestampsToSearch);
- }
-
- // --------------- factory methods ---------------
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the committed offsets. An
- * exception will be thrown at runtime if there is no committed offsets.
- *
- * @return an offset initializer which initialize the offsets to the committed offsets.
- */
- static OffsetsInitializer committedOffsets() {
- return committedOffsets(OffsetResetStrategy.NONE);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the committed offsets. Use
- * the given {@link OffsetResetStrategy} to initialize the offsets if the committed offsets does
- * not exist.
- *
- * @param offsetResetStrategy the offset reset strategy to use when the committed offsets do not
- * exist.
- * @return an {@link OffsetsInitializer} which initializes the offsets to the committed offsets.
- */
- static OffsetsInitializer committedOffsets(OffsetResetStrategy offsetResetStrategy) {
- return new ReaderHandledOffsetsInitializer(
- KafkaPartitionSplit.COMMITTED_OFFSET, offsetResetStrategy);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets in each partition so that the
- * initialized offset is the offset of the first record whose record timestamp is greater than
- * or equals the given timestamp (milliseconds).
- *
- * @param timestamp the timestamp (milliseconds) to start the consumption.
- * @return an {@link OffsetsInitializer} which initializes the offsets based on the given
- * timestamp.
- * @see KafkaAdminClient#listOffsets(Map)
- */
- static OffsetsInitializer timestamp(long timestamp) {
- return new TimestampOffsetsInitializer(timestamp);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the earliest available
- * offsets of each partition.
- *
- * @return an {@link OffsetsInitializer} which initializes the offsets to the earliest available
- * offsets.
- */
- static OffsetsInitializer earliest() {
- return new ReaderHandledOffsetsInitializer(
- KafkaPartitionSplit.EARLIEST_OFFSET, OffsetResetStrategy.EARLIEST);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the latest offsets of each
- * partition.
- *
- * @return an {@link OffsetsInitializer} which initializes the offsets to the latest offsets.
- */
- static OffsetsInitializer latest() {
- return new ReaderHandledOffsetsInitializer(
- KafkaPartitionSplit.LATEST_OFFSET, OffsetResetStrategy.LATEST);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the specified offsets.
- *
- * @param offsets the specified offsets for each partition.
- * @return an {@link OffsetsInitializer} which initializes the offsets to the specified offsets.
- */
- static OffsetsInitializer offsets(Map<TopicPartition, Long> offsets) {
- return new SpecifiedOffsetsInitializer(offsets, OffsetResetStrategy.EARLIEST);
- }
-
- /**
- * Get an {@link OffsetsInitializer} which initializes the offsets to the specified offsets. Use
- * the given {@link OffsetResetStrategy} to initialize the offsets in case the specified offset
- * is out of range.
- *
- * @param offsets the specified offsets for each partition.
- * @param offsetResetStrategy the {@link OffsetResetStrategy} to use when the specified offset
- * is out of range.
- * @return an {@link OffsetsInitializer} which initializes the offsets to the specified offsets.
- */
- static OffsetsInitializer offsets(
- Map<TopicPartition, Long> offsets, OffsetResetStrategy offsetResetStrategy) {
- return new SpecifiedOffsetsInitializer(offsets, offsetResetStrategy);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerValidator.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerValidator.java
deleted file mode 100644
index c198107..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerValidator.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.annotation.Internal;
-
-import java.util.Properties;
-
-/**
- * Interface for validating {@link OffsetsInitializer} with properties from {@link
- * org.apache.flink.connector.kafka.source.KafkaSource}.
- */
-@Internal
-public interface OffsetsInitializerValidator {
-
- /**
- * Validate offsets initializer with properties of Kafka source.
- *
- * @param kafkaSourceProperties Properties of Kafka source
- * @throws IllegalStateException if validation fails
- */
- void validate(Properties kafkaSourceProperties) throws IllegalStateException;
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java
deleted file mode 100644
index 026320d..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * A initializer that initialize the partitions to the earliest / latest / last-committed offsets.
- * The offsets initialization are taken care of by the {@code KafkaPartitionSplitReader} instead of
- * by the {@code KafkaSourceEnumerator}.
- *
- * <p>Package private and should be instantiated via {@link OffsetsInitializer}.
- */
-class ReaderHandledOffsetsInitializer implements OffsetsInitializer, OffsetsInitializerValidator {
- private static final long serialVersionUID = 172938052008787981L;
- private final long startingOffset;
- private final OffsetResetStrategy offsetResetStrategy;
-
- /**
- * The only valid value for startingOffset is following. {@link
- * KafkaPartitionSplit#EARLIEST_OFFSET EARLIEST_OFFSET}, {@link
- * KafkaPartitionSplit#LATEST_OFFSET LATEST_OFFSET}, {@link KafkaPartitionSplit#COMMITTED_OFFSET
- * COMMITTED_OFFSET}
- */
- ReaderHandledOffsetsInitializer(long startingOffset, OffsetResetStrategy offsetResetStrategy) {
- this.startingOffset = startingOffset;
- this.offsetResetStrategy = offsetResetStrategy;
- }
-
- @Override
- public Map<TopicPartition, Long> getPartitionOffsets(
- Collection<TopicPartition> partitions,
- PartitionOffsetsRetriever partitionOffsetsRetriever) {
- Map<TopicPartition, Long> initialOffsets = new HashMap<>();
- for (TopicPartition tp : partitions) {
- initialOffsets.put(tp, startingOffset);
- }
- return initialOffsets;
- }
-
- @Override
- public OffsetResetStrategy getAutoOffsetResetStrategy() {
- return offsetResetStrategy;
- }
-
- @Override
- public void validate(Properties kafkaSourceProperties) {
- if (startingOffset == KafkaPartitionSplit.COMMITTED_OFFSET) {
- checkState(
- kafkaSourceProperties.containsKey(ConsumerConfig.GROUP_ID_CONFIG),
- String.format(
- "Property %s is required when using committed offset for offsets initializer",
- ConsumerConfig.GROUP_ID_CONFIG));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java
deleted file mode 100644
index 5766a5f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * An implementation of {@link OffsetsInitializer} which initializes the offsets of the partition
- * according to the user specified offsets.
- *
- * <p>Package private and should be instantiated via {@link OffsetsInitializer}.
- */
-class SpecifiedOffsetsInitializer implements OffsetsInitializer, OffsetsInitializerValidator {
- private static final long serialVersionUID = 1649702397250402877L;
- private final Map<TopicPartition, Long> initialOffsets;
- private final OffsetResetStrategy offsetResetStrategy;
-
- SpecifiedOffsetsInitializer(
- Map<TopicPartition, Long> initialOffsets, OffsetResetStrategy offsetResetStrategy) {
- this.initialOffsets = Collections.unmodifiableMap(initialOffsets);
- this.offsetResetStrategy = offsetResetStrategy;
- }
-
- @Override
- public Map<TopicPartition, Long> getPartitionOffsets(
- Collection<TopicPartition> partitions,
- PartitionOffsetsRetriever partitionOffsetsRetriever) {
- Map<TopicPartition, Long> offsets = new HashMap<>();
- List<TopicPartition> toLookup = new ArrayList<>();
- for (TopicPartition tp : partitions) {
- Long offset = initialOffsets.get(tp);
- if (offset == null) {
- toLookup.add(tp);
- } else {
- offsets.put(tp, offset);
- }
- }
- if (!toLookup.isEmpty()) {
- // First check the committed offsets.
- Map<TopicPartition, Long> committedOffsets =
- partitionOffsetsRetriever.committedOffsets(toLookup);
- offsets.putAll(committedOffsets);
- toLookup.removeAll(committedOffsets.keySet());
-
- switch (offsetResetStrategy) {
- case EARLIEST:
- offsets.putAll(partitionOffsetsRetriever.beginningOffsets(toLookup));
- break;
- case LATEST:
- offsets.putAll(partitionOffsetsRetriever.endOffsets(toLookup));
- break;
- default:
- throw new IllegalStateException(
- "Cannot find initial offsets for partitions: " + toLookup);
- }
- }
- return offsets;
- }
-
- @Override
- public OffsetResetStrategy getAutoOffsetResetStrategy() {
- return offsetResetStrategy;
- }
-
- @Override
- public void validate(Properties kafkaSourceProperties) {
- initialOffsets.forEach(
- (tp, offset) -> {
- if (offset == KafkaPartitionSplit.COMMITTED_OFFSET) {
- checkState(
- kafkaSourceProperties.containsKey(ConsumerConfig.GROUP_ID_CONFIG),
- String.format(
- "Property %s is required because partition %s is initialized with committed offset",
- ConsumerConfig.GROUP_ID_CONFIG, tp));
- }
- });
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java
deleted file mode 100644
index f411307..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * An implementation of {@link OffsetsInitializer} to initialize the offsets based on a timestamp.
- *
- * <p>Package private and should be instantiated via {@link OffsetsInitializer}.
- */
-class TimestampOffsetsInitializer implements OffsetsInitializer {
- private static final long serialVersionUID = 2932230571773627233L;
- private final long startingTimestamp;
-
- TimestampOffsetsInitializer(long startingTimestamp) {
- this.startingTimestamp = startingTimestamp;
- }
-
- @Override
- public Map<TopicPartition, Long> getPartitionOffsets(
- Collection<TopicPartition> partitions,
- PartitionOffsetsRetriever partitionOffsetsRetriever) {
- Map<TopicPartition, Long> startingTimestamps = new HashMap<>();
- Map<TopicPartition, Long> initialOffsets = new HashMap<>();
-
- // First get the current end offsets of the partitions. This is going to be used
- // in case we cannot find a suitable offsets based on the timestamp, i.e. the message
- // meeting the requirement of the timestamp have not been produced to Kafka yet, in
- // this case, we just use the latest offset.
- // We need to get the latest offsets before querying offsets by time to ensure that
- // no message is going to be missed.
- Map<TopicPartition, Long> endOffsets = partitionOffsetsRetriever.endOffsets(partitions);
- partitions.forEach(tp -> startingTimestamps.put(tp, startingTimestamp));
- Map<TopicPartition, OffsetAndTimestamp> topicPartitionOffsetAndTimestampMap =
- partitionOffsetsRetriever.offsetsForTimes(startingTimestamps);
-
- for (TopicPartition tp : partitions) {
- // offset may not have been resolved
- if (topicPartitionOffsetAndTimestampMap.containsKey(tp)) {
- initialOffsets.put(tp, topicPartitionOffsetAndTimestampMap.get(tp).offset());
- } else {
- initialOffsets.put(tp, endOffsets.get(tp));
- }
- }
-
- return initialOffsets;
- }
-
- @Override
- public OffsetResetStrategy getAutoOffsetResetStrategy() {
- return OffsetResetStrategy.LATEST;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java
deleted file mode 100644
index 1b819fb..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-/**
- * Kafka consumer allows a few different ways to consume from the topics, including:
- *
- * <ol>
- * <li>Subscribe from a collection of topics.
- * <li>Subscribe to a topic pattern using Java {@code Regex}.
- * <li>Assign specific partitions.
- * </ol>
- *
- * <p>The KafkaSubscriber provides a unified interface for the Kafka source to support all these
- * three types of subscribing mode.
- */
-@PublicEvolving
-public interface KafkaSubscriber extends Serializable {
-
- /**
- * Get a set of subscribed {@link TopicPartition}s.
- *
- * @param adminClient The admin client used to retrieve subscribed topic partitions.
- * @return A set of subscribed {@link TopicPartition}s
- */
- Set<TopicPartition> getSubscribedTopicPartitions(AdminClient adminClient);
-
- // ----------------- factory methods --------------
-
- static KafkaSubscriber getTopicListSubscriber(List<String> topics) {
- return new TopicListSubscriber(topics);
- }
-
- static KafkaSubscriber getTopicPatternSubscriber(Pattern topicPattern) {
- return new TopicPatternSubscriber(topicPattern);
- }
-
- static KafkaSubscriber getPartitionSetSubscriber(Set<TopicPartition> partitions) {
- return new PartitionSetSubscriber(partitions);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java
deleted file mode 100644
index 404ffae..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.TopicDescription;
-
-import java.util.Map;
-import java.util.Set;
-
-/** The base implementations of {@link KafkaSubscriber}. */
-class KafkaSubscriberUtils {
-
- private KafkaSubscriberUtils() {}
-
- static Map<String, TopicDescription> getAllTopicMetadata(AdminClient adminClient) {
- try {
- Set<String> allTopicNames = adminClient.listTopics().names().get();
- return getTopicMetadata(adminClient, allTopicNames);
- } catch (Exception e) {
- throw new RuntimeException("Failed to get metadata for all topics.", e);
- }
- }
-
- static Map<String, TopicDescription> getTopicMetadata(
- AdminClient adminClient, Set<String> topicNames) {
- try {
- return adminClient.describeTopics(topicNames).allTopicNames().get();
- } catch (Exception e) {
- throw new RuntimeException(
- String.format("Failed to get metadata for topics %s.", topicNames), e);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java
deleted file mode 100644
index 3423b0f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata;
-
-/** A subscriber for a partition set. */
-class PartitionSetSubscriber implements KafkaSubscriber {
- private static final long serialVersionUID = 390970375272146036L;
- private static final Logger LOG = LoggerFactory.getLogger(PartitionSetSubscriber.class);
- private final Set<TopicPartition> subscribedPartitions;
-
- PartitionSetSubscriber(Set<TopicPartition> partitions) {
- this.subscribedPartitions = partitions;
- }
-
- @Override
- public Set<TopicPartition> getSubscribedTopicPartitions(AdminClient adminClient) {
- final Set<String> topicNames =
- subscribedPartitions.stream()
- .map(TopicPartition::topic)
- .collect(Collectors.toSet());
-
- LOG.debug("Fetching descriptions for topics: {}", topicNames);
- final Map<String, TopicDescription> topicMetadata =
- getTopicMetadata(adminClient, topicNames);
-
- Set<TopicPartition> existingSubscribedPartitions = new HashSet<>();
-
- for (TopicPartition subscribedPartition : this.subscribedPartitions) {
- if (topicMetadata.containsKey(subscribedPartition.topic())
- && partitionExistsInTopic(
- subscribedPartition, topicMetadata.get(subscribedPartition.topic()))) {
- existingSubscribedPartitions.add(subscribedPartition);
- } else {
- throw new RuntimeException(
- String.format(
- "Partition '%s' does not exist on Kafka brokers",
- subscribedPartition));
- }
- }
-
- return existingSubscribedPartitions;
- }
-
- private boolean partitionExistsInTopic(TopicPartition partition, TopicDescription topic) {
- return topic.partitions().size() > partition.partition();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java
deleted file mode 100644
index b2ad844..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.TopicPartitionInfo;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata;
-
-/**
- * A subscriber to a fixed list of topics. The subscribed topics must have existed in the Kafka
- * cluster, otherwise an exception will be thrown.
- */
-class TopicListSubscriber implements KafkaSubscriber {
- private static final long serialVersionUID = -6917603843104947866L;
- private static final Logger LOG = LoggerFactory.getLogger(TopicListSubscriber.class);
- private final List<String> topics;
-
- TopicListSubscriber(List<String> topics) {
- this.topics = topics;
- }
-
- @Override
- public Set<TopicPartition> getSubscribedTopicPartitions(AdminClient adminClient) {
- LOG.debug("Fetching descriptions for topics: {}", topics);
- final Map<String, TopicDescription> topicMetadata =
- getTopicMetadata(adminClient, new HashSet<>(topics));
-
- Set<TopicPartition> subscribedPartitions = new HashSet<>();
- for (TopicDescription topic : topicMetadata.values()) {
- for (TopicPartitionInfo partition : topic.partitions()) {
- subscribedPartitions.add(new TopicPartition(topic.name(), partition.partition()));
- }
- }
-
- return subscribedPartitions;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java
deleted file mode 100644
index 2a9a753..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.TopicPartitionInfo;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getAllTopicMetadata;
-
-/** A subscriber to a topic pattern. */
-class TopicPatternSubscriber implements KafkaSubscriber {
- private static final long serialVersionUID = -7471048577725467797L;
- private static final Logger LOG = LoggerFactory.getLogger(TopicPatternSubscriber.class);
- private final Pattern topicPattern;
-
- TopicPatternSubscriber(Pattern topicPattern) {
- this.topicPattern = topicPattern;
- }
-
- @Override
- public Set<TopicPartition> getSubscribedTopicPartitions(AdminClient adminClient) {
- LOG.debug("Fetching descriptions for all topics on Kafka cluster");
- final Map<String, TopicDescription> allTopicMetadata = getAllTopicMetadata(adminClient);
-
- Set<TopicPartition> subscribedTopicPartitions = new HashSet<>();
-
- allTopicMetadata.forEach(
- (topicName, topicDescription) -> {
- if (topicPattern.matcher(topicName).matches()) {
- for (TopicPartitionInfo partition : topicDescription.partitions()) {
- subscribedTopicPartitions.add(
- new TopicPartition(
- topicDescription.name(), partition.partition()));
- }
- }
- });
-
- return subscribedTopicPartitions;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
deleted file mode 100644
index f94243e..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * 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.metrics;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.connector.kafka.MetricUtil;
-import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader;
-import org.apache.flink.metrics.Counter;
-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 org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.function.Predicate;
-
-/**
- * A collection class for handling metrics in {@link KafkaSourceReader}.
- *
- * <p>All metrics of Kafka source reader are registered under group "KafkaSourceReader", which is a
- * child group of {@link org.apache.flink.metrics.groups.OperatorMetricGroup}. Metrics related to a
- * specific topic partition will be registered in the group
- * "KafkaSourceReader.topic.{topic_name}.partition.{partition_id}".
- *
- * <p>For example, current consuming offset of topic "my-topic" and partition 1 will be reported in
- * metric:
- * "{some_parent_groups}.operator.KafkaSourceReader.topic.my-topic.partition.1.currentOffset"
- *
- * <p>and number of successful commits will be reported in metric:
- * "{some_parent_groups}.operator.KafkaSourceReader.commitsSucceeded"
- *
- * <p>All metrics of Kafka consumer are also registered under group
- * "KafkaSourceReader.KafkaConsumer". For example, Kafka consumer metric "records-consumed-total"
- * can be found at:
- * {some_parent_groups}.operator.KafkaSourceReader.KafkaConsumer.records-consumed-total"
- */
-@PublicEvolving
-public class KafkaSourceReaderMetrics {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceReaderMetrics.class);
-
- // Constants
- public static final String KAFKA_SOURCE_READER_METRIC_GROUP = "KafkaSourceReader";
- public static final String TOPIC_GROUP = "topic";
- public static final String PARTITION_GROUP = "partition";
- public static final String CURRENT_OFFSET_METRIC_GAUGE = "currentOffset";
- public static final String COMMITTED_OFFSET_METRIC_GAUGE = "committedOffset";
- public static final String COMMITS_SUCCEEDED_METRIC_COUNTER = "commitsSucceeded";
- public static final String COMMITS_FAILED_METRIC_COUNTER = "commitsFailed";
- public static final String KAFKA_CONSUMER_METRIC_GROUP = "KafkaConsumer";
-
- // Kafka raw metric names and group names
- public static final String CONSUMER_FETCH_MANAGER_GROUP = "consumer-fetch-manager-metrics";
- public static final String BYTES_CONSUMED_TOTAL = "bytes-consumed-total";
- public static final String RECORDS_LAG = "records-lag";
-
- public static final long INITIAL_OFFSET = -1;
-
- // Source reader metric group
- private final SourceReaderMetricGroup sourceReaderMetricGroup;
-
- // Metric group for registering Kafka specific metrics
- private final MetricGroup kafkaSourceReaderMetricGroup;
-
- // Successful / Failed commits counters
- private final Counter commitsSucceeded;
- private final Counter commitsFailed;
-
- // Map for tracking current consuming / committing offsets
- private final Map<TopicPartition, Offset> offsets = new HashMap<>();
-
- // Map for tracking records lag of topic partitions
- @Nullable private ConcurrentMap<TopicPartition, Metric> recordsLagMetrics;
-
- // Kafka raw metric for bytes consumed total
- @Nullable private Metric bytesConsumedTotalMetric;
-
- /** Number of bytes consumed total at the latest {@link #updateNumBytesInCounter()}. */
- private long latestBytesConsumedTotal;
-
- public KafkaSourceReaderMetrics(SourceReaderMetricGroup sourceReaderMetricGroup) {
- this.sourceReaderMetricGroup = sourceReaderMetricGroup;
- this.kafkaSourceReaderMetricGroup =
- sourceReaderMetricGroup.addGroup(KAFKA_SOURCE_READER_METRIC_GROUP);
- this.commitsSucceeded =
- this.kafkaSourceReaderMetricGroup.counter(COMMITS_SUCCEEDED_METRIC_COUNTER);
- this.commitsFailed =
- this.kafkaSourceReaderMetricGroup.counter(COMMITS_FAILED_METRIC_COUNTER);
- }
-
- /**
- * Register metrics of KafkaConsumer in Kafka metric group.
- *
- * @param kafkaConsumer Kafka consumer used by partition split reader.
- */
- @SuppressWarnings("Convert2MethodRef")
- public void registerKafkaConsumerMetrics(KafkaConsumer<?, ?> kafkaConsumer) {
- final Map<MetricName, ? extends Metric> kafkaConsumerMetrics = kafkaConsumer.metrics();
- if (kafkaConsumerMetrics == null) {
- LOG.warn("Consumer implementation does not support metrics");
- return;
- }
-
- final MetricGroup kafkaConsumerMetricGroup =
- kafkaSourceReaderMetricGroup.addGroup(KAFKA_CONSUMER_METRIC_GROUP);
-
- kafkaConsumerMetrics.forEach(
- (name, metric) ->
- kafkaConsumerMetricGroup.gauge(name.name(), () -> metric.metricValue()));
- }
-
- /**
- * Register metric groups for the given {@link TopicPartition}.
- *
- * @param tp Registering topic partition
- */
- public void registerTopicPartition(TopicPartition tp) {
- offsets.put(tp, new Offset(INITIAL_OFFSET, INITIAL_OFFSET));
- registerOffsetMetricsForTopicPartition(tp);
- }
-
- /**
- * Update current consuming offset of the given {@link TopicPartition}.
- *
- * @param tp Updating topic partition
- * @param offset Current consuming offset
- */
- public void recordCurrentOffset(TopicPartition tp, long offset) {
- checkTopicPartitionTracked(tp);
- offsets.get(tp).currentOffset = offset;
- }
-
- /**
- * Update the latest committed offset of the given {@link TopicPartition}.
- *
- * @param tp Updating topic partition
- * @param offset Committing offset
- */
- public void recordCommittedOffset(TopicPartition tp, long offset) {
- checkTopicPartitionTracked(tp);
- offsets.get(tp).committedOffset = offset;
- }
-
- /** Mark a successful commit. */
- public void recordSucceededCommit() {
- commitsSucceeded.inc();
- }
-
- /** Mark a failure commit. */
- public void recordFailedCommit() {
- commitsFailed.inc();
- }
-
- /**
- * Register {@link MetricNames#IO_NUM_BYTES_IN}.
- *
- * @param consumer Kafka consumer
- */
- public void registerNumBytesIn(KafkaConsumer<?, ?> consumer) {
- try {
- Predicate<Map.Entry<MetricName, ? extends Metric>> filter =
- (entry) ->
- entry.getKey().group().equals(CONSUMER_FETCH_MANAGER_GROUP)
- && entry.getKey().name().equals(BYTES_CONSUMED_TOTAL)
- && !entry.getKey().tags().containsKey("topic");
- this.bytesConsumedTotalMetric = MetricUtil.getKafkaMetric(consumer.metrics(), filter);
- } catch (IllegalStateException e) {
- LOG.warn(
- String.format(
- "Error when getting Kafka consumer metric \"%s\". "
- + "I/O metric \"%s\" will not be reported. ",
- BYTES_CONSUMED_TOTAL, MetricNames.IO_NUM_BYTES_IN),
- e);
- }
- }
-
- /**
- * Add a partition's records-lag metric to tracking list if this partition never appears before.
- *
- * <p>This method also lazily register {@link
- * org.apache.flink.runtime.metrics.MetricNames#PENDING_RECORDS} in {@link
- * SourceReaderMetricGroup}
- *
- * @param consumer Kafka consumer
- * @param tp Topic partition
- */
- public void maybeAddRecordsLagMetric(KafkaConsumer<?, ?> consumer, TopicPartition tp) {
- // Lazily register pendingRecords
- if (recordsLagMetrics == null) {
- this.recordsLagMetrics = new ConcurrentHashMap<>();
- this.sourceReaderMetricGroup.setPendingRecordsGauge(
- () -> {
- long pendingRecordsTotal = 0;
- for (Metric recordsLagMetric : this.recordsLagMetrics.values()) {
- pendingRecordsTotal +=
- ((Double) recordsLagMetric.metricValue()).longValue();
- }
- return pendingRecordsTotal;
- });
- }
- recordsLagMetrics.computeIfAbsent(
- tp, (ignored) -> getRecordsLagMetric(consumer.metrics(), tp));
- }
-
- /**
- * Remove a partition's records-lag metric from tracking list.
- *
- * @param tp Unassigned topic partition
- */
- public void removeRecordsLagMetric(TopicPartition tp) {
- if (recordsLagMetrics != null) {
- recordsLagMetrics.remove(tp);
- }
- }
-
- /**
- * Update {@link org.apache.flink.runtime.metrics.MetricNames#IO_NUM_BYTES_IN}.
- *
- * <p>Instead of simply setting {@link OperatorIOMetricGroup#getNumBytesInCounter()} to the same
- * value as bytes-consumed-total from Kafka consumer, which will screw {@link
- * TaskIOMetricGroup#getNumBytesInCounter()} if chained sources exist, we track the increment of
- * bytes-consumed-total and count it towards the counter.
- */
- public void updateNumBytesInCounter() {
- if (this.bytesConsumedTotalMetric != null) {
- long bytesConsumedUntilNow =
- ((Number) this.bytesConsumedTotalMetric.metricValue()).longValue();
- long bytesConsumedSinceLastUpdate = bytesConsumedUntilNow - latestBytesConsumedTotal;
- this.sourceReaderMetricGroup
- .getIOMetricGroup()
- .getNumBytesInCounter()
- .inc(bytesConsumedSinceLastUpdate);
- latestBytesConsumedTotal = bytesConsumedUntilNow;
- }
- }
-
- // -------- Helper functions --------
- private void registerOffsetMetricsForTopicPartition(TopicPartition tp) {
- final MetricGroup topicPartitionGroup =
- this.kafkaSourceReaderMetricGroup
- .addGroup(TOPIC_GROUP, tp.topic())
- .addGroup(PARTITION_GROUP, String.valueOf(tp.partition()));
- topicPartitionGroup.gauge(
- CURRENT_OFFSET_METRIC_GAUGE,
- () ->
- offsets.getOrDefault(tp, new Offset(INITIAL_OFFSET, INITIAL_OFFSET))
- .currentOffset);
- topicPartitionGroup.gauge(
- COMMITTED_OFFSET_METRIC_GAUGE,
- () ->
- offsets.getOrDefault(tp, new Offset(INITIAL_OFFSET, INITIAL_OFFSET))
- .committedOffset);
- }
-
- private void checkTopicPartitionTracked(TopicPartition tp) {
- if (!offsets.containsKey(tp)) {
- throw new IllegalArgumentException(
- String.format("TopicPartition %s is not tracked", tp));
- }
- }
-
- private @Nullable Metric getRecordsLagMetric(
- Map<MetricName, ? extends Metric> metrics, TopicPartition tp) {
- try {
- final String resolvedTopic = tp.topic().replace('.', '_');
- final String resolvedPartition = String.valueOf(tp.partition());
- Predicate<Map.Entry<MetricName, ? extends Metric>> filter =
- entry -> {
- final MetricName metricName = entry.getKey();
- final Map<String, String> tags = metricName.tags();
-
- return metricName.group().equals(CONSUMER_FETCH_MANAGER_GROUP)
- && metricName.name().equals(RECORDS_LAG)
- && tags.containsKey("topic")
- && tags.get("topic").equals(resolvedTopic)
- && tags.containsKey("partition")
- && tags.get("partition").equals(resolvedPartition);
- };
- return MetricUtil.getKafkaMetric(metrics, filter);
- } catch (IllegalStateException e) {
- LOG.warn(
- String.format(
- "Error when getting Kafka consumer metric \"%s\" "
- + "for partition \"%s\". "
- + "Metric \"%s\" may not be reported correctly. ",
- RECORDS_LAG, tp, MetricNames.PENDING_RECORDS),
- e);
- return null;
- }
- }
-
- private static class Offset {
- long currentOffset;
- long committedOffset;
-
- Offset(long currentOffset, long committedOffset) {
- this.currentOffset = currentOffset;
- this.committedOffset = committedOffset;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java
deleted file mode 100644
index c440fc2..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java
+++ /dev/null
@@ -1,532 +0,0 @@
-/*
- * 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.reader;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.connector.source.SourceReaderContext;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
-import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
-import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.WakeupException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.StringJoiner;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-/** A {@link SplitReader} implementation that reads records from Kafka partitions. */
-@Internal
-public class KafkaPartitionSplitReader
- implements SplitReader<ConsumerRecord<byte[], byte[]>, KafkaPartitionSplit> {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaPartitionSplitReader.class);
- private static final long POLL_TIMEOUT = 10000L;
-
- private final KafkaConsumer<byte[], byte[]> consumer;
- private final Map<TopicPartition, Long> stoppingOffsets;
- private final String groupId;
- private final int subtaskId;
-
- private final KafkaSourceReaderMetrics kafkaSourceReaderMetrics;
-
- // Tracking empty splits that has not been added to finished splits in fetch()
- private final Set<String> emptySplits = new HashSet<>();
-
- public KafkaPartitionSplitReader(
- Properties props,
- SourceReaderContext context,
- KafkaSourceReaderMetrics kafkaSourceReaderMetrics) {
- this.subtaskId = context.getIndexOfSubtask();
- this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics;
- Properties consumerProps = new Properties();
- consumerProps.putAll(props);
- consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props));
- this.consumer = new KafkaConsumer<>(consumerProps);
- this.stoppingOffsets = new HashMap<>();
- this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
-
- // Metric registration
- maybeRegisterKafkaConsumerMetrics(props, kafkaSourceReaderMetrics, consumer);
- this.kafkaSourceReaderMetrics.registerNumBytesIn(consumer);
- }
-
- @Override
- public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> fetch() throws IOException {
- ConsumerRecords<byte[], byte[]> consumerRecords;
- try {
- consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT));
- } catch (WakeupException | IllegalStateException e) {
- // IllegalStateException will be thrown if the consumer is not assigned any partitions.
- // This happens if all assigned partitions are invalid or empty (starting offset >=
- // stopping offset). We just mark empty partitions as finished and return an empty
- // record container, and this consumer will be closed by SplitFetcherManager.
- KafkaPartitionSplitRecords recordsBySplits =
- new KafkaPartitionSplitRecords(
- ConsumerRecords.empty(), kafkaSourceReaderMetrics);
- markEmptySplitsAsFinished(recordsBySplits);
- return recordsBySplits;
- }
- KafkaPartitionSplitRecords recordsBySplits =
- new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics);
- List<TopicPartition> finishedPartitions = new ArrayList<>();
- for (TopicPartition tp : consumerRecords.partitions()) {
- long stoppingOffset = getStoppingOffset(tp);
- final List<ConsumerRecord<byte[], byte[]>> recordsFromPartition =
- consumerRecords.records(tp);
-
- if (recordsFromPartition.size() > 0) {
- final ConsumerRecord<byte[], byte[]> lastRecord =
- recordsFromPartition.get(recordsFromPartition.size() - 1);
-
- // After processing a record with offset of "stoppingOffset - 1", the split reader
- // should not continue fetching because the record with stoppingOffset may not
- // exist. Keep polling will just block forever.
- if (lastRecord.offset() >= stoppingOffset - 1) {
- recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset);
- finishSplitAtRecord(
- tp,
- stoppingOffset,
- lastRecord.offset(),
- finishedPartitions,
- recordsBySplits);
- }
- }
- // Track this partition's record lag if it never appears before
- kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp);
- }
-
- markEmptySplitsAsFinished(recordsBySplits);
-
- // Unassign the partitions that has finished.
- if (!finishedPartitions.isEmpty()) {
- finishedPartitions.forEach(kafkaSourceReaderMetrics::removeRecordsLagMetric);
- unassignPartitions(finishedPartitions);
- }
-
- // Update numBytesIn
- kafkaSourceReaderMetrics.updateNumBytesInCounter();
-
- return recordsBySplits;
- }
-
- private void markEmptySplitsAsFinished(KafkaPartitionSplitRecords recordsBySplits) {
- // Some splits are discovered as empty when handling split additions. These splits should be
- // added to finished splits to clean up states in split fetcher and source reader.
- if (!emptySplits.isEmpty()) {
- recordsBySplits.finishedSplits.addAll(emptySplits);
- emptySplits.clear();
- }
- }
-
- @Override
- public void handleSplitsChanges(SplitsChange<KafkaPartitionSplit> splitsChange) {
- // Get all the partition assignments and stopping offsets.
- if (!(splitsChange instanceof SplitsAddition)) {
- throw new UnsupportedOperationException(
- String.format(
- "The SplitChange type of %s is not supported.",
- splitsChange.getClass()));
- }
-
- // Assignment.
- List<TopicPartition> newPartitionAssignments = new ArrayList<>();
- // Starting offsets.
- Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets = new HashMap<>();
- List<TopicPartition> partitionsStartingFromEarliest = new ArrayList<>();
- List<TopicPartition> partitionsStartingFromLatest = new ArrayList<>();
- // Stopping offsets.
- List<TopicPartition> partitionsStoppingAtLatest = new ArrayList<>();
- Set<TopicPartition> partitionsStoppingAtCommitted = new HashSet<>();
-
- // Parse the starting and stopping offsets.
- splitsChange
- .splits()
- .forEach(
- s -> {
- newPartitionAssignments.add(s.getTopicPartition());
- parseStartingOffsets(
- s,
- partitionsStartingFromEarliest,
- partitionsStartingFromLatest,
- partitionsStartingFromSpecifiedOffsets);
- parseStoppingOffsets(
- s, partitionsStoppingAtLatest, partitionsStoppingAtCommitted);
- // Track the new topic partition in metrics
- kafkaSourceReaderMetrics.registerTopicPartition(s.getTopicPartition());
- });
-
- // Assign new partitions.
- newPartitionAssignments.addAll(consumer.assignment());
- consumer.assign(newPartitionAssignments);
-
- // Seek on the newly assigned partitions to their stating offsets.
- seekToStartingOffsets(
- partitionsStartingFromEarliest,
- partitionsStartingFromLatest,
- partitionsStartingFromSpecifiedOffsets);
- // Setup the stopping offsets.
- acquireAndSetStoppingOffsets(partitionsStoppingAtLatest, partitionsStoppingAtCommitted);
-
- // After acquiring the starting and stopping offsets, remove the empty splits if necessary.
- removeEmptySplits();
-
- maybeLogSplitChangesHandlingResult(splitsChange);
- }
-
- @Override
- public void wakeUp() {
- consumer.wakeup();
- }
-
- @Override
- public void close() throws Exception {
- consumer.close();
- }
-
- @Override
- public void pauseOrResumeSplits(
- Collection<KafkaPartitionSplit> splitsToPause,
- Collection<KafkaPartitionSplit> splitsToResume) {
- consumer.resume(
- splitsToResume.stream()
- .map(KafkaPartitionSplit::getTopicPartition)
- .collect(Collectors.toList()));
- consumer.pause(
- splitsToPause.stream()
- .map(KafkaPartitionSplit::getTopicPartition)
- .collect(Collectors.toList()));
- }
-
- // ---------------
-
- public void notifyCheckpointComplete(
- Map<TopicPartition, OffsetAndMetadata> offsetsToCommit,
- OffsetCommitCallback offsetCommitCallback) {
- consumer.commitAsync(offsetsToCommit, offsetCommitCallback);
- }
-
- @VisibleForTesting
- KafkaConsumer<byte[], byte[]> consumer() {
- return consumer;
- }
-
- // --------------- private helper method ----------------------
-
- private void parseStartingOffsets(
- KafkaPartitionSplit split,
- List<TopicPartition> partitionsStartingFromEarliest,
- List<TopicPartition> partitionsStartingFromLatest,
- Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets) {
- TopicPartition tp = split.getTopicPartition();
- // Parse starting offsets.
- if (split.getStartingOffset() == KafkaPartitionSplit.EARLIEST_OFFSET) {
- partitionsStartingFromEarliest.add(tp);
- } else if (split.getStartingOffset() == KafkaPartitionSplit.LATEST_OFFSET) {
- partitionsStartingFromLatest.add(tp);
- } else if (split.getStartingOffset() == KafkaPartitionSplit.COMMITTED_OFFSET) {
- // Do nothing here, the consumer will first try to get the committed offsets of
- // these partitions by default.
- } else {
- partitionsStartingFromSpecifiedOffsets.put(tp, split.getStartingOffset());
- }
- }
-
- private void parseStoppingOffsets(
- KafkaPartitionSplit split,
- List<TopicPartition> partitionsStoppingAtLatest,
- Set<TopicPartition> partitionsStoppingAtCommitted) {
- TopicPartition tp = split.getTopicPartition();
- split.getStoppingOffset()
- .ifPresent(
- stoppingOffset -> {
- if (stoppingOffset >= 0) {
- stoppingOffsets.put(tp, stoppingOffset);
- } else if (stoppingOffset == KafkaPartitionSplit.LATEST_OFFSET) {
- partitionsStoppingAtLatest.add(tp);
- } else if (stoppingOffset == KafkaPartitionSplit.COMMITTED_OFFSET) {
- partitionsStoppingAtCommitted.add(tp);
- } else {
- // This should not happen.
- throw new FlinkRuntimeException(
- String.format(
- "Invalid stopping offset %d for partition %s",
- stoppingOffset, tp));
- }
- });
- }
-
- private void seekToStartingOffsets(
- List<TopicPartition> partitionsStartingFromEarliest,
- List<TopicPartition> partitionsStartingFromLatest,
- Map<TopicPartition, Long> partitionsStartingFromSpecifiedOffsets) {
-
- if (!partitionsStartingFromEarliest.isEmpty()) {
- LOG.trace("Seeking starting offsets to beginning: {}", partitionsStartingFromEarliest);
- consumer.seekToBeginning(partitionsStartingFromEarliest);
- }
-
- if (!partitionsStartingFromLatest.isEmpty()) {
- LOG.trace("Seeking starting offsets to end: {}", partitionsStartingFromLatest);
- consumer.seekToEnd(partitionsStartingFromLatest);
- }
-
- if (!partitionsStartingFromSpecifiedOffsets.isEmpty()) {
- LOG.trace(
- "Seeking starting offsets to specified offsets: {}",
- partitionsStartingFromSpecifiedOffsets);
- partitionsStartingFromSpecifiedOffsets.forEach(consumer::seek);
- }
- }
-
- private void acquireAndSetStoppingOffsets(
- List<TopicPartition> partitionsStoppingAtLatest,
- Set<TopicPartition> partitionsStoppingAtCommitted) {
- Map<TopicPartition, Long> endOffset = consumer.endOffsets(partitionsStoppingAtLatest);
- stoppingOffsets.putAll(endOffset);
- if (!partitionsStoppingAtCommitted.isEmpty()) {
- retryOnWakeup(
- () -> consumer.committed(partitionsStoppingAtCommitted),
- "getting committed offset as stopping offsets")
- .forEach(
- (tp, offsetAndMetadata) -> {
- Preconditions.checkNotNull(
- offsetAndMetadata,
- String.format(
- "Partition %s should stop at committed offset. "
- + "But there is no committed offset of this partition for group %s",
- tp, groupId));
- stoppingOffsets.put(tp, offsetAndMetadata.offset());
- });
- }
- }
-
- private void removeEmptySplits() {
- List<TopicPartition> emptyPartitions = new ArrayList<>();
- // If none of the partitions have any records,
- for (TopicPartition tp : consumer.assignment()) {
- if (retryOnWakeup(
- () -> consumer.position(tp),
- "getting starting offset to check if split is empty")
- >= getStoppingOffset(tp)) {
- emptyPartitions.add(tp);
- }
- }
- if (!emptyPartitions.isEmpty()) {
- LOG.debug(
- "These assigning splits are empty and will be marked as finished in later fetch: {}",
- emptyPartitions);
- // Add empty partitions to empty split set for later cleanup in fetch()
- emptySplits.addAll(
- emptyPartitions.stream()
- .map(KafkaPartitionSplit::toSplitId)
- .collect(Collectors.toSet()));
- // Un-assign partitions from Kafka consumer
- unassignPartitions(emptyPartitions);
- }
- }
-
- private void maybeLogSplitChangesHandlingResult(
- SplitsChange<KafkaPartitionSplit> splitsChange) {
- if (LOG.isDebugEnabled()) {
- StringJoiner splitsInfo = new StringJoiner(",");
- for (KafkaPartitionSplit split : splitsChange.splits()) {
- long startingOffset =
- retryOnWakeup(
- () -> consumer.position(split.getTopicPartition()),
- "logging starting position");
- long stoppingOffset = getStoppingOffset(split.getTopicPartition());
- splitsInfo.add(
- String.format(
- "[%s, start:%d, stop: %d]",
- split.getTopicPartition(), startingOffset, stoppingOffset));
- }
- LOG.debug("SplitsChange handling result: {}", splitsInfo);
- }
- }
-
- private void unassignPartitions(Collection<TopicPartition> partitionsToUnassign) {
- Collection<TopicPartition> newAssignment = new HashSet<>(consumer.assignment());
- newAssignment.removeAll(partitionsToUnassign);
- consumer.assign(newAssignment);
- }
-
- private String createConsumerClientId(Properties props) {
- String prefix = props.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key());
- return prefix + "-" + subtaskId;
- }
-
- private void finishSplitAtRecord(
- TopicPartition tp,
- long stoppingOffset,
- long currentOffset,
- List<TopicPartition> finishedPartitions,
- KafkaPartitionSplitRecords recordsBySplits) {
- LOG.debug(
- "{} has reached stopping offset {}, current offset is {}",
- tp,
- stoppingOffset,
- currentOffset);
- finishedPartitions.add(tp);
- recordsBySplits.addFinishedSplit(KafkaPartitionSplit.toSplitId(tp));
- }
-
- private long getStoppingOffset(TopicPartition tp) {
- return stoppingOffsets.getOrDefault(tp, Long.MAX_VALUE);
- }
-
- private void maybeRegisterKafkaConsumerMetrics(
- Properties props,
- KafkaSourceReaderMetrics kafkaSourceReaderMetrics,
- KafkaConsumer<?, ?> consumer) {
- final Boolean needToRegister =
- KafkaSourceOptions.getOption(
- props,
- KafkaSourceOptions.REGISTER_KAFKA_CONSUMER_METRICS,
- Boolean::parseBoolean);
- if (needToRegister) {
- kafkaSourceReaderMetrics.registerKafkaConsumerMetrics(consumer);
- }
- }
-
- /**
- * Catch {@link WakeupException} in Kafka consumer call and retry the invocation on exception.
- *
- * <p>This helper function handles a race condition as below:
- *
- * <ol>
- * <li>Fetcher thread finishes a {@link KafkaConsumer#poll(Duration)} call
- * <li>Task thread assigns new splits so invokes {@link #wakeUp()}, then the wakeup is
- * recorded and held by the consumer
- * <li>Later fetcher thread invokes {@link #handleSplitsChanges(SplitsChange)}, and
- * interactions with consumer will throw {@link WakeupException} because of the previously
- * held wakeup in the consumer
- * </ol>
- *
- * <p>Under this case we need to catch the {@link WakeupException} and retry the operation.
- */
- private <V> V retryOnWakeup(Supplier<V> consumerCall, String description) {
- try {
- return consumerCall.get();
- } catch (WakeupException we) {
- LOG.info(
- "Caught WakeupException while executing Kafka consumer call for {}. Will retry the consumer call.",
- description);
- return consumerCall.get();
- }
- }
-
- // ---------------- private helper class ------------------------
-
- private static class KafkaPartitionSplitRecords
- implements RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> {
-
- private final Set<String> finishedSplits = new HashSet<>();
- private final Map<TopicPartition, Long> stoppingOffsets = new HashMap<>();
- private final ConsumerRecords<byte[], byte[]> consumerRecords;
- private final KafkaSourceReaderMetrics metrics;
- private final Iterator<TopicPartition> splitIterator;
- private Iterator<ConsumerRecord<byte[], byte[]>> recordIterator;
- private TopicPartition currentTopicPartition;
- private Long currentSplitStoppingOffset;
-
- private KafkaPartitionSplitRecords(
- ConsumerRecords<byte[], byte[]> consumerRecords, KafkaSourceReaderMetrics metrics) {
- this.consumerRecords = consumerRecords;
- this.splitIterator = consumerRecords.partitions().iterator();
- this.metrics = metrics;
- }
-
- private void setPartitionStoppingOffset(
- TopicPartition topicPartition, long stoppingOffset) {
- stoppingOffsets.put(topicPartition, stoppingOffset);
- }
-
- private void addFinishedSplit(String splitId) {
- finishedSplits.add(splitId);
- }
-
- @Nullable
- @Override
- public String nextSplit() {
- if (splitIterator.hasNext()) {
- currentTopicPartition = splitIterator.next();
- recordIterator = consumerRecords.records(currentTopicPartition).iterator();
- currentSplitStoppingOffset =
- stoppingOffsets.getOrDefault(currentTopicPartition, Long.MAX_VALUE);
- return currentTopicPartition.toString();
- } else {
- currentTopicPartition = null;
- recordIterator = null;
- currentSplitStoppingOffset = null;
- return null;
- }
- }
-
- @Nullable
- @Override
- public ConsumerRecord<byte[], byte[]> nextRecordFromSplit() {
- Preconditions.checkNotNull(
- currentTopicPartition,
- "Make sure nextSplit() did not return null before "
- + "iterate over the records split.");
- if (recordIterator.hasNext()) {
- final ConsumerRecord<byte[], byte[]> record = recordIterator.next();
- // Only emit records before stopping offset
- if (record.offset() < currentSplitStoppingOffset) {
- metrics.recordCurrentOffset(currentTopicPartition, record.offset());
- return record;
- }
- }
- return null;
- }
-
- @Override
- public Set<String> finishedSplits() {
- return finishedSplits;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaRecordEmitter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaRecordEmitter.java
deleted file mode 100644
index 3a1320a..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaRecordEmitter.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.reader;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.connector.source.SourceOutput;
-import org.apache.flink.connector.base.source.reader.RecordEmitter;
-import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitState;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.IOException;
-
-/** The {@link RecordEmitter} implementation for {@link KafkaSourceReader}. */
-@Internal
-public class KafkaRecordEmitter<T>
- implements RecordEmitter<ConsumerRecord<byte[], byte[]>, T, KafkaPartitionSplitState> {
-
- private final KafkaRecordDeserializationSchema<T> deserializationSchema;
- private final SourceOutputWrapper<T> sourceOutputWrapper = new SourceOutputWrapper<>();
-
- public KafkaRecordEmitter(KafkaRecordDeserializationSchema<T> deserializationSchema) {
- this.deserializationSchema = deserializationSchema;
- }
-
- @Override
- public void emitRecord(
- ConsumerRecord<byte[], byte[]> consumerRecord,
- SourceOutput<T> output,
- KafkaPartitionSplitState splitState)
- throws Exception {
- try {
- sourceOutputWrapper.setSourceOutput(output);
- sourceOutputWrapper.setTimestamp(consumerRecord.timestamp());
- deserializationSchema.deserialize(consumerRecord, sourceOutputWrapper);
- splitState.setCurrentOffset(consumerRecord.offset() + 1);
- } catch (Exception e) {
- throw new IOException("Failed to deserialize consumer record due to", e);
- }
- }
-
- private static class SourceOutputWrapper<T> implements Collector<T> {
-
- private SourceOutput<T> sourceOutput;
- private long timestamp;
-
- @Override
- public void collect(T record) {
- sourceOutput.collect(record, timestamp);
- }
-
- @Override
- public void close() {}
-
- private void setSourceOutput(SourceOutput<T> sourceOutput) {
- this.sourceOutput = sourceOutput;
- }
-
- private void setTimestamp(long timestamp) {
- this.timestamp = timestamp;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java
deleted file mode 100644
index a3aa555..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * 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.reader;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.connector.source.SourceReaderContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.RecordEmitter;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
-import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
-import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
-import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics;
-import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitState;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/** The source reader for Kafka partitions. */
-@Internal
-public class KafkaSourceReader<T>
- extends SingleThreadMultiplexSourceReaderBase<
- ConsumerRecord<byte[], byte[]>, T, KafkaPartitionSplit, KafkaPartitionSplitState> {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceReader.class);
- // These maps need to be concurrent because it will be accessed by both the main thread
- // and the split fetcher thread in the callback.
- private final SortedMap<Long, Map<TopicPartition, OffsetAndMetadata>> offsetsToCommit;
- private final ConcurrentMap<TopicPartition, OffsetAndMetadata> offsetsOfFinishedSplits;
- private final KafkaSourceReaderMetrics kafkaSourceReaderMetrics;
- private final boolean commitOffsetsOnCheckpoint;
-
- public KafkaSourceReader(
- FutureCompletingBlockingQueue<RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>>>
- elementsQueue,
- KafkaSourceFetcherManager kafkaSourceFetcherManager,
- RecordEmitter<ConsumerRecord<byte[], byte[]>, T, KafkaPartitionSplitState>
- recordEmitter,
- Configuration config,
- SourceReaderContext context,
- KafkaSourceReaderMetrics kafkaSourceReaderMetrics) {
- super(elementsQueue, kafkaSourceFetcherManager, recordEmitter, config, context);
- this.offsetsToCommit = Collections.synchronizedSortedMap(new TreeMap<>());
- this.offsetsOfFinishedSplits = new ConcurrentHashMap<>();
- this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics;
- this.commitOffsetsOnCheckpoint =
- config.get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT);
- if (!commitOffsetsOnCheckpoint) {
- LOG.warn(
- "Offset commit on checkpoint is disabled. "
- + "Consuming offset will not be reported back to Kafka cluster.");
- }
- }
-
- @Override
- protected void onSplitFinished(Map<String, KafkaPartitionSplitState> finishedSplitIds) {
- finishedSplitIds.forEach(
- (ignored, splitState) -> {
- if (splitState.getCurrentOffset() >= 0) {
- offsetsOfFinishedSplits.put(
- splitState.getTopicPartition(),
- new OffsetAndMetadata(splitState.getCurrentOffset()));
- }
- });
- }
-
- @Override
- public List<KafkaPartitionSplit> snapshotState(long checkpointId) {
- List<KafkaPartitionSplit> splits = super.snapshotState(checkpointId);
- if (!commitOffsetsOnCheckpoint) {
- return splits;
- }
-
- if (splits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) {
- offsetsToCommit.put(checkpointId, Collections.emptyMap());
- } else {
- Map<TopicPartition, OffsetAndMetadata> offsetsMap =
- offsetsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>());
- // Put the offsets of the active splits.
- for (KafkaPartitionSplit split : splits) {
- // If the checkpoint is triggered before the partition starting offsets
- // is retrieved, do not commit the offsets for those partitions.
- if (split.getStartingOffset() >= 0) {
- offsetsMap.put(
- split.getTopicPartition(),
- new OffsetAndMetadata(split.getStartingOffset()));
- }
- }
- // Put offsets of all the finished splits.
- offsetsMap.putAll(offsetsOfFinishedSplits);
- }
- return splits;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- LOG.debug("Committing offsets for checkpoint {}", checkpointId);
- if (!commitOffsetsOnCheckpoint) {
- return;
- }
-
- Map<TopicPartition, OffsetAndMetadata> committedPartitions =
- offsetsToCommit.get(checkpointId);
- if (committedPartitions == null) {
- LOG.debug(
- "Offsets for checkpoint {} either do not exist or have already been committed.",
- checkpointId);
- return;
- }
-
- ((KafkaSourceFetcherManager) splitFetcherManager)
- .commitOffsets(
- committedPartitions,
- (ignored, e) -> {
- // The offset commit here is needed by the external monitoring. It won't
- // break Flink job's correctness if we fail to commit the offset here.
- if (e != null) {
- kafkaSourceReaderMetrics.recordFailedCommit();
- LOG.warn(
- "Failed to commit consumer offsets for checkpoint {}",
- checkpointId,
- e);
- } else {
- LOG.debug(
- "Successfully committed offsets for checkpoint {}",
- checkpointId);
- kafkaSourceReaderMetrics.recordSucceededCommit();
- // If the finished topic partition has been committed, we remove it
- // from the offsets of the finished splits map.
- committedPartitions.forEach(
- (tp, offset) ->
- kafkaSourceReaderMetrics.recordCommittedOffset(
- tp, offset.offset()));
- offsetsOfFinishedSplits
- .entrySet()
- .removeIf(
- entry ->
- committedPartitions.containsKey(
- entry.getKey()));
- while (!offsetsToCommit.isEmpty()
- && offsetsToCommit.firstKey() <= checkpointId) {
- offsetsToCommit.remove(offsetsToCommit.firstKey());
- }
- }
- });
- }
-
- @Override
- protected KafkaPartitionSplitState initializedState(KafkaPartitionSplit split) {
- return new KafkaPartitionSplitState(split);
- }
-
- @Override
- protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitState splitState) {
- return splitState.toKafkaPartitionSplit();
- }
-
- // ------------------------
-
- @VisibleForTesting
- SortedMap<Long, Map<TopicPartition, OffsetAndMetadata>> getOffsetsToCommit() {
- return offsetsToCommit;
- }
-
- @VisibleForTesting
- int getNumAliveFetchers() {
- return splitFetcherManager.getNumAliveFetchers();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java
deleted file mode 100644
index 94197e3..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.reader.deserializer;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.IOException;
-
-/**
- * A wrapper class that wraps a {@link
- * org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema} to deserialize {@link
- * ConsumerRecord ConsumerRecords}.
- *
- * @param <T> the type of the deserialized records.
- */
-class KafkaDeserializationSchemaWrapper<T> implements KafkaRecordDeserializationSchema<T> {
- private static final long serialVersionUID = 1L;
- private final KafkaDeserializationSchema<T> kafkaDeserializationSchema;
-
- KafkaDeserializationSchemaWrapper(KafkaDeserializationSchema<T> kafkaDeserializationSchema) {
- this.kafkaDeserializationSchema = kafkaDeserializationSchema;
- }
-
- @Override
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- kafkaDeserializationSchema.open(context);
- }
-
- @Override
- public void deserialize(ConsumerRecord<byte[], byte[]> message, Collector<T> out)
- throws IOException {
- try {
- kafkaDeserializationSchema.deserialize(message, out);
- } catch (Exception exception) {
- throw new IOException(
- String.format("Failed to deserialize consumer record %s.", message), exception);
- }
- }
-
- @Override
- public TypeInformation<T> getProducedType() {
- return kafkaDeserializationSchema.getProducedType();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java
deleted file mode 100644
index 6ad6607..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.reader.deserializer;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.serialization.Deserializer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.Map;
-
-/** An interface for the deserialization of Kafka records. */
-@PublicEvolving
-public interface KafkaRecordDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
-
- /**
- * Initialization method for the schema. It is called before the actual working methods {@link
- * #deserialize} and thus suitable for one time setup work.
- *
- * <p>The provided {@link DeserializationSchema.InitializationContext} can be used to access
- * additional features such as e.g. registering user metrics.
- *
- * @param context Contextual information that can be used during initialization.
- */
- default void open(DeserializationSchema.InitializationContext context) throws Exception {}
-
- /**
- * Deserializes the byte message.
- *
- * <p>Can output multiple records through the {@link Collector}. Note that number and size of
- * the produced records should be relatively small. Depending on the source implementation
- * records can be buffered in memory or collecting records might delay emitting checkpoint
- * barrier.
- *
- * @param record The ConsumerRecord to deserialize.
- * @param out The collector to put the resulting messages.
- */
- void deserialize(ConsumerRecord<byte[], byte[]> record, Collector<T> out) throws IOException;
-
- /**
- * Wraps a legacy {@link KafkaDeserializationSchema} as the deserializer of the {@link
- * ConsumerRecord ConsumerRecords}.
- *
- * <p>Note that the {@link KafkaDeserializationSchema#isEndOfStream(Object)} method will no
- * longer be used to determine the end of the stream.
- *
- * @param kafkaDeserializationSchema the legacy {@link KafkaDeserializationSchema} to use.
- * @param <V> the return type of the deserialized record.
- * @return A {@link KafkaRecordDeserializationSchema} that uses the given {@link
- * KafkaDeserializationSchema} to deserialize the {@link ConsumerRecord ConsumerRecords}.
- */
- static <V> KafkaRecordDeserializationSchema<V> of(
- KafkaDeserializationSchema<V> kafkaDeserializationSchema) {
- return new KafkaDeserializationSchemaWrapper<>(kafkaDeserializationSchema);
- }
-
- /**
- * Wraps a {@link DeserializationSchema} as the value deserialization schema of the {@link
- * ConsumerRecord ConsumerRecords}. The other fields such as key, headers, timestamp are
- * ignored.
- *
- * <p>Note that the {@link DeserializationSchema#isEndOfStream(Object)} method will no longer be
- * used to determine the end of the stream.
- *
- * @param valueDeserializationSchema the {@link DeserializationSchema} used to deserialized the
- * value of a {@link ConsumerRecord}.
- * @param <V> the type of the deserialized record.
- * @return A {@link KafkaRecordDeserializationSchema} that uses the given {@link
- * DeserializationSchema} to deserialize a {@link ConsumerRecord} from its value.
- */
- static <V> KafkaRecordDeserializationSchema<V> valueOnly(
- DeserializationSchema<V> valueDeserializationSchema) {
- return new KafkaValueOnlyDeserializationSchemaWrapper<>(valueDeserializationSchema);
- }
-
- /**
- * Wraps a Kafka {@link Deserializer} to a {@link KafkaRecordDeserializationSchema}.
- *
- * @param valueDeserializerClass the deserializer class used to deserialize the value.
- * @param <V> the value type.
- * @return A {@link KafkaRecordDeserializationSchema} that deserialize the value with the given
- * deserializer.
- */
- static <V> KafkaRecordDeserializationSchema<V> valueOnly(
- Class<? extends Deserializer<V>> valueDeserializerClass) {
- return valueOnly(valueDeserializerClass, Collections.emptyMap());
- }
-
- /**
- * Wraps a Kafka {@link Deserializer} to a {@link KafkaRecordDeserializationSchema}.
- *
- * @param valueDeserializerClass the deserializer class used to deserialize the value.
- * @param config the configuration of the value deserializer. If the deserializer is an
- * implementation of {@code Configurable}, the configuring logic will be handled by {@link
- * org.apache.kafka.common.Configurable#configure(Map)} with the given {@link config},
- * otherwise {@link Deserializer#configure(Map, boolean)} will be invoked.
- * @param <V> the value type.
- * @param <D> the type of the deserializer.
- * @return A {@link KafkaRecordDeserializationSchema} that deserialize the value with the given
- * deserializer.
- */
- static <V, D extends Deserializer<V>> KafkaRecordDeserializationSchema<V> valueOnly(
- Class<D> valueDeserializerClass, Map<String, String> config) {
- return new KafkaValueOnlyDeserializerWrapper<>(valueDeserializerClass, config);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializationSchemaWrapper.java
deleted file mode 100644
index 209f5e1..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializationSchemaWrapper.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.reader.deserializer;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.IOException;
-
-/**
- * A class that wraps a {@link DeserializationSchema} as the value deserializer for a {@link
- * ConsumerRecord}.
- *
- * @param <T> the return type of the deserialization.
- */
-class KafkaValueOnlyDeserializationSchemaWrapper<T> implements KafkaRecordDeserializationSchema<T> {
- private static final long serialVersionUID = 1L;
- private final DeserializationSchema<T> deserializationSchema;
-
- KafkaValueOnlyDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
- this.deserializationSchema = deserializationSchema;
- }
-
- @Override
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- deserializationSchema.open(context);
- }
-
- @Override
- public void deserialize(ConsumerRecord<byte[], byte[]> message, Collector<T> out)
- throws IOException {
- deserializationSchema.deserialize(message.value(), out);
- }
-
- @Override
- public TypeInformation<T> getProducedType() {
- return deserializationSchema.getProducedType();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java
deleted file mode 100644
index 8c8095b..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.reader.deserializer;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.TemporaryClassLoaderContext;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.Configurable;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Map;
-
-/** A package private class to wrap {@link Deserializer}. */
-class KafkaValueOnlyDeserializerWrapper<T> implements KafkaRecordDeserializationSchema<T> {
-
- private static final long serialVersionUID = 5409547407386004054L;
-
- private static final Logger LOG =
- LoggerFactory.getLogger(KafkaValueOnlyDeserializerWrapper.class);
-
- private final Class<? extends Deserializer<T>> deserializerClass;
-
- private final Map<String, String> config;
-
- private transient Deserializer<T> deserializer;
-
- KafkaValueOnlyDeserializerWrapper(
- Class<? extends Deserializer<T>> deserializerClass, Map<String, String> config) {
- this.deserializerClass = deserializerClass;
- this.config = config;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- ClassLoader userCodeClassLoader = context.getUserCodeClassLoader().asClassLoader();
- try (TemporaryClassLoaderContext ignored =
- TemporaryClassLoaderContext.of(userCodeClassLoader)) {
- deserializer =
- (Deserializer<T>)
- InstantiationUtil.instantiate(
- deserializerClass.getName(),
- Deserializer.class,
- getClass().getClassLoader());
-
- if (deserializer instanceof Configurable) {
- ((Configurable) deserializer).configure(config);
- } else {
- // Always be false since this Deserializer is only used for value.
- deserializer.configure(config, false);
- }
- } catch (Exception e) {
- throw new IOException(
- "Failed to instantiate the deserializer of class " + deserializerClass, e);
- }
- }
-
- @Override
- public void deserialize(ConsumerRecord<byte[], byte[]> record, Collector<T> collector)
- throws IOException {
- if (deserializer == null) {
- throw new IllegalStateException(
- "The deserializer has not been created. Make sure the open() method has been "
- + "invoked.");
- }
-
- T value = deserializer.deserialize(record.topic(), record.value());
- LOG.trace(
- "Deserialized [partition: {}-{}, offset: {}, timestamp: {}, value: {}]",
- record.topic(),
- record.partition(),
- record.offset(),
- record.timestamp(),
- value);
- collector.collect(value);
- }
-
- @Override
- public TypeInformation<T> getProducedType() {
- return TypeExtractor.createTypeInfo(Deserializer.class, deserializerClass, 0, null, null);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/fetcher/KafkaSourceFetcherManager.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/fetcher/KafkaSourceFetcherManager.java
deleted file mode 100644
index 2113fb9..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/fetcher/KafkaSourceFetcherManager.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.reader.fetcher;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.SourceReaderBase;
-import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
-import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher;
-import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
-import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
-import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-/**
- * The SplitFetcherManager for Kafka source. This class is needed to help commit the offsets to
- * Kafka using the KafkaConsumer inside the {@link
- * org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader}.
- */
-@Internal
-public class KafkaSourceFetcherManager
- extends SingleThreadFetcherManager<ConsumerRecord<byte[], byte[]>, KafkaPartitionSplit> {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceFetcherManager.class);
-
- /**
- * Creates a new SplitFetcherManager with a single I/O threads.
- *
- * @param elementsQueue The queue that is used to hand over data from the I/O thread (the
- * fetchers) to the reader (which emits the records and book-keeps the state. This must be
- * the same queue instance that is also passed to the {@link SourceReaderBase}.
- * @param splitReaderSupplier The factory for the split reader that connects to the source
- * system.
- * @param splitFinishedHook Hook for handling finished splits in split fetchers.
- */
- public KafkaSourceFetcherManager(
- FutureCompletingBlockingQueue<RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>>>
- elementsQueue,
- Supplier<SplitReader<ConsumerRecord<byte[], byte[]>, KafkaPartitionSplit>>
- splitReaderSupplier,
- Consumer<Collection<String>> splitFinishedHook) {
- super(elementsQueue, splitReaderSupplier, new Configuration(), splitFinishedHook);
- }
-
- public void commitOffsets(
- Map<TopicPartition, OffsetAndMetadata> offsetsToCommit, OffsetCommitCallback callback) {
- LOG.debug("Committing offsets {}", offsetsToCommit);
- if (offsetsToCommit.isEmpty()) {
- return;
- }
- SplitFetcher<ConsumerRecord<byte[], byte[]>, KafkaPartitionSplit> splitFetcher =
- fetchers.get(0);
- if (splitFetcher != null) {
- // The fetcher thread is still running. This should be the majority of the cases.
- enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback);
- } else {
- splitFetcher = createSplitFetcher();
- enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback);
- startFetcher(splitFetcher);
- }
- }
-
- private void enqueueOffsetsCommitTask(
- SplitFetcher<ConsumerRecord<byte[], byte[]>, KafkaPartitionSplit> splitFetcher,
- Map<TopicPartition, OffsetAndMetadata> offsetsToCommit,
- OffsetCommitCallback callback) {
- KafkaPartitionSplitReader kafkaReader =
- (KafkaPartitionSplitReader) splitFetcher.getSplitReader();
-
- splitFetcher.enqueueTask(
- new SplitFetcherTask() {
- @Override
- public boolean run() throws IOException {
- kafkaReader.notifyCheckpointComplete(offsetsToCommit, callback);
- return true;
- }
-
- @Override
- public void wakeUp() {}
- });
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java
deleted file mode 100644
index 8c2a1fd..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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.split;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.connector.source.SourceSplit;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import org.apache.kafka.common.TopicPartition;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Set;
-
-/** A {@link SourceSplit} for a Kafka partition. */
-@Internal
-public class KafkaPartitionSplit implements SourceSplit {
- public static final long NO_STOPPING_OFFSET = Long.MIN_VALUE;
- // Indicating the split should consume from the latest.
- public static final long LATEST_OFFSET = -1;
- // Indicating the split should consume from the earliest.
- public static final long EARLIEST_OFFSET = -2;
- // Indicating the split should consume from the last committed offset.
- public static final long COMMITTED_OFFSET = -3;
-
- // Valid special starting offsets
- public static final Set<Long> VALID_STARTING_OFFSET_MARKERS =
- new HashSet<>(Arrays.asList(EARLIEST_OFFSET, LATEST_OFFSET, COMMITTED_OFFSET));
- public static final Set<Long> VALID_STOPPING_OFFSET_MARKERS =
- new HashSet<>(Arrays.asList(LATEST_OFFSET, COMMITTED_OFFSET, NO_STOPPING_OFFSET));
-
- private final TopicPartition tp;
- private final long startingOffset;
- private final long stoppingOffset;
-
- public KafkaPartitionSplit(TopicPartition tp, long startingOffset) {
- this(tp, startingOffset, NO_STOPPING_OFFSET);
- }
-
- public KafkaPartitionSplit(TopicPartition tp, long startingOffset, long stoppingOffset) {
- verifyInitialOffset(tp, startingOffset, stoppingOffset);
- this.tp = tp;
- this.startingOffset = startingOffset;
- this.stoppingOffset = stoppingOffset;
- }
-
- public String getTopic() {
- return tp.topic();
- }
-
- public int getPartition() {
- return tp.partition();
- }
-
- public TopicPartition getTopicPartition() {
- return tp;
- }
-
- public long getStartingOffset() {
- return startingOffset;
- }
-
- public Optional<Long> getStoppingOffset() {
- return stoppingOffset >= 0
- || stoppingOffset == LATEST_OFFSET
- || stoppingOffset == COMMITTED_OFFSET
- ? Optional.of(stoppingOffset)
- : Optional.empty();
- }
-
- @Override
- public String splitId() {
- return toSplitId(tp);
- }
-
- @Override
- public String toString() {
- return String.format(
- "[Partition: %s, StartingOffset: %d, StoppingOffset: %d]",
- tp, startingOffset, stoppingOffset);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(tp, startingOffset, stoppingOffset);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof KafkaPartitionSplit)) {
- return false;
- }
- KafkaPartitionSplit other = (KafkaPartitionSplit) obj;
- return tp.equals(other.tp)
- && startingOffset == other.startingOffset
- && stoppingOffset == other.stoppingOffset;
- }
-
- public static String toSplitId(TopicPartition tp) {
- return tp.toString();
- }
-
- // ------------ private methods ---------------
-
- private static void verifyInitialOffset(
- TopicPartition tp, Long startingOffset, long stoppingOffset) {
- if (startingOffset == null) {
- throw new FlinkRuntimeException(
- "Cannot initialize starting offset for partition " + tp);
- }
-
- if (startingOffset < 0 && !VALID_STARTING_OFFSET_MARKERS.contains(startingOffset)) {
- throw new FlinkRuntimeException(
- String.format(
- "Invalid starting offset %d is specified for partition %s. "
- + "It should either be non-negative or be one of the "
- + "[%d(earliest), %d(latest), %d(committed)].",
- startingOffset, tp, LATEST_OFFSET, EARLIEST_OFFSET, COMMITTED_OFFSET));
- }
-
- if (stoppingOffset < 0 && !VALID_STOPPING_OFFSET_MARKERS.contains(stoppingOffset)) {
- throw new FlinkRuntimeException(
- String.format(
- "Illegal stopping offset %d is specified for partition %s. "
- + "It should either be non-negative or be one of the "
- + "[%d(latest), %d(committed), %d(Long.MIN_VALUE, no_stopping_offset)].",
- stoppingOffset,
- tp,
- LATEST_OFFSET,
- COMMITTED_OFFSET,
- NO_STOPPING_OFFSET));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializer.java
deleted file mode 100644
index d906a0b..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializer.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.split;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-/**
- * The {@link org.apache.flink.core.io.SimpleVersionedSerializer serializer} for {@link
- * KafkaPartitionSplit}.
- */
-@Internal
-public class KafkaPartitionSplitSerializer
- implements SimpleVersionedSerializer<KafkaPartitionSplit> {
-
- private static final int CURRENT_VERSION = 0;
-
- @Override
- public int getVersion() {
- return CURRENT_VERSION;
- }
-
- @Override
- public byte[] serialize(KafkaPartitionSplit split) throws IOException {
- try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(baos)) {
- out.writeUTF(split.getTopic());
- out.writeInt(split.getPartition());
- out.writeLong(split.getStartingOffset());
- out.writeLong(split.getStoppingOffset().orElse(KafkaPartitionSplit.NO_STOPPING_OFFSET));
- out.flush();
- return baos.toByteArray();
- }
- }
-
- @Override
- public KafkaPartitionSplit deserialize(int version, byte[] serialized) throws IOException {
- try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
- DataInputStream in = new DataInputStream(bais)) {
- String topic = in.readUTF();
- int partition = in.readInt();
- long offset = in.readLong();
- long stoppingOffset = in.readLong();
- return new KafkaPartitionSplit(
- new TopicPartition(topic, partition), offset, stoppingOffset);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitState.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitState.java
deleted file mode 100644
index 18a9e65..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitState.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.split;
-
-import org.apache.flink.annotation.Internal;
-
-/** This class extends KafkaPartitionSplit to track a mutable current offset. */
-@Internal
-public class KafkaPartitionSplitState extends KafkaPartitionSplit {
-
- private long currentOffset;
-
- public KafkaPartitionSplitState(KafkaPartitionSplit partitionSplit) {
- super(
- partitionSplit.getTopicPartition(),
- partitionSplit.getStartingOffset(),
- partitionSplit.getStoppingOffset().orElse(NO_STOPPING_OFFSET));
- this.currentOffset = partitionSplit.getStartingOffset();
- }
-
- public long getCurrentOffset() {
- return currentOffset;
- }
-
- public void setCurrentOffset(long currentOffset) {
- this.currentOffset = currentOffset;
- }
-
- /**
- * Use the current offset as the starting offset to create a new KafkaPartitionSplit.
- *
- * @return a new KafkaPartitionSplit which uses the current offset as its starting offset.
- */
- public KafkaPartitionSplit toKafkaPartitionSplit() {
- return new KafkaPartitionSplit(
- getTopicPartition(),
- getCurrentOffset(),
- getStoppingOffset().orElse(NO_STOPPING_OFFSET));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
deleted file mode 100644
index 1e506a8..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
+++ /dev/null
@@ -1,342 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.util.PropertiesUtil;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.PropertiesUtil.getBoolean;
-import static org.apache.flink.util.PropertiesUtil.getLong;
-
-/**
- * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from Apache
- * Kafka. The consumer can run in multiple parallel instances, each of which will pull data from one
- * or more Kafka partitions.
- *
- * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
- * during a failure, and that the computation processes elements "exactly once". (Note: These
- * guarantees naturally assume that Kafka itself does not lose any data.)
- *
- * <p>Please note that Flink snapshots the offsets internally as part of its distributed
- * checkpoints. The offsets committed to Kafka are only to bring the outside view of progress in
- * sync with Flink's view of the progress. That way, monitoring and other jobs can get a view of how
- * far the Flink Kafka consumer has consumed a topic.
- *
- * <p>Please refer to Kafka's documentation for the available configuration properties:
- * http://kafka.apache.org/documentation.html#newconsumerconfigs
- */
-@PublicEvolving
-@Deprecated
-public class FlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-
- private static final long serialVersionUID = 1L;
-
- /** Configuration key to change the polling timeout. * */
- public static final String KEY_POLL_TIMEOUT = "flink.poll-timeout";
-
- /**
- * From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
- * available. If 0, returns immediately with any records that are available now.
- */
- public static final long DEFAULT_POLL_TIMEOUT = 100L;
-
- // ------------------------------------------------------------------------
-
- /** User-supplied properties for Kafka. * */
- protected final Properties properties;
-
- /**
- * From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
- * available. If 0, returns immediately with any records that are available now
- */
- protected final long pollTimeout;
-
- // ------------------------------------------------------------------------
-
- /**
- * Creates a new Kafka streaming source consumer.
- *
- * @param topic The name of the topic that should be consumed.
- * @param valueDeserializer The de-/serializer used to convert between Kafka's byte messages and
- * Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
- this(Collections.singletonList(topic), valueDeserializer, props);
- }
-
- /**
- * Creates a new Kafka streaming source consumer.
- *
- * <p>This constructor allows passing a {@see KafkaDeserializationSchema} for reading key/value
- * pairs, offsets, and topic names from Kafka.
- *
- * @param topic The name of the topic that should be consumed.
- * @param deserializer The keyed de-/serializer used to convert between Kafka's byte messages
- * and Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- String topic, KafkaDeserializationSchema<T> deserializer, Properties props) {
- this(Collections.singletonList(topic), deserializer, props);
- }
-
- /**
- * Creates a new Kafka streaming source consumer.
- *
- * <p>This constructor allows passing multiple topics to the consumer.
- *
- * @param topics The Kafka topics to read from.
- * @param deserializer The de-/serializer used to convert between Kafka's byte messages and
- * Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- List<String> topics, DeserializationSchema<T> deserializer, Properties props) {
- this(topics, new KafkaDeserializationSchemaWrapper<>(deserializer), props);
- }
-
- /**
- * Creates a new Kafka streaming source consumer.
- *
- * <p>This constructor allows passing multiple topics and a key/value deserialization schema.
- *
- * @param topics The Kafka topics to read from.
- * @param deserializer The keyed de-/serializer used to convert between Kafka's byte messages
- * and Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- List<String> topics, KafkaDeserializationSchema<T> deserializer, Properties props) {
- this(topics, null, deserializer, props);
- }
-
- /**
- * Creates a new Kafka streaming source consumer. Use this constructor to subscribe to multiple
- * topics based on a regular expression pattern.
- *
- * <p>If partition discovery is enabled (by setting a non-negative value for {@link
- * FlinkKafkaConsumer#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics with
- * names matching the pattern will also be subscribed to as they are created on the fly.
- *
- * @param subscriptionPattern The regular expression for a pattern of topic names to subscribe
- * to.
- * @param valueDeserializer The de-/serializer used to convert between Kafka's byte messages and
- * Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- Pattern subscriptionPattern,
- DeserializationSchema<T> valueDeserializer,
- Properties props) {
- this(
- null,
- subscriptionPattern,
- new KafkaDeserializationSchemaWrapper<>(valueDeserializer),
- props);
- }
-
- /**
- * Creates a new Kafka streaming source consumer. Use this constructor to subscribe to multiple
- * topics based on a regular expression pattern.
- *
- * <p>If partition discovery is enabled (by setting a non-negative value for {@link
- * FlinkKafkaConsumer#KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS} in the properties), topics with
- * names matching the pattern will also be subscribed to as they are created on the fly.
- *
- * <p>This constructor allows passing a {@see KafkaDeserializationSchema} for reading key/value
- * pairs, offsets, and topic names from Kafka.
- *
- * @param subscriptionPattern The regular expression for a pattern of topic names to subscribe
- * to.
- * @param deserializer The keyed de-/serializer used to convert between Kafka's byte messages
- * and Flink's objects.
- * @param props
- */
- public FlinkKafkaConsumer(
- Pattern subscriptionPattern,
- KafkaDeserializationSchema<T> deserializer,
- Properties props) {
- this(null, subscriptionPattern, deserializer, props);
- }
-
- private FlinkKafkaConsumer(
- List<String> topics,
- Pattern subscriptionPattern,
- KafkaDeserializationSchema<T> deserializer,
- Properties props) {
-
- super(
- topics,
- subscriptionPattern,
- deserializer,
- getLong(
- checkNotNull(props, "props"),
- KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS,
- PARTITION_DISCOVERY_DISABLED),
- !getBoolean(props, KEY_DISABLE_METRICS, false));
-
- this.properties = props;
- setDeserializer(this.properties);
-
- // configure the polling timeout
- try {
- if (properties.containsKey(KEY_POLL_TIMEOUT)) {
- this.pollTimeout = Long.parseLong(properties.getProperty(KEY_POLL_TIMEOUT));
- } else {
- this.pollTimeout = DEFAULT_POLL_TIMEOUT;
- }
- } catch (Exception e) {
- throw new IllegalArgumentException(
- "Cannot parse poll timeout for '" + KEY_POLL_TIMEOUT + '\'', e);
- }
- }
-
- @Override
- protected AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
-
- // make sure that auto commit is disabled when our offset commit mode is ON_CHECKPOINTS;
- // this overwrites whatever setting the user configured in the properties
- adjustAutoCommitConfig(properties, offsetCommitMode);
-
- return new KafkaFetcher<>(
- sourceContext,
- assignedPartitionsWithInitialOffsets,
- watermarkStrategy,
- runtimeContext.getProcessingTimeService(),
- runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
- runtimeContext.getUserCodeClassLoader(),
- runtimeContext.getTaskNameWithSubtasks(),
- deserializer,
- properties,
- pollTimeout,
- runtimeContext.getMetricGroup(),
- consumerMetricGroup,
- useMetrics);
- }
-
- @Override
- protected AbstractPartitionDiscoverer createPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks) {
-
- return new KafkaPartitionDiscoverer(
- topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, properties);
- }
-
- @Override
- protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
- Collection<KafkaTopicPartition> partitions, long timestamp) {
-
- Map<TopicPartition, Long> partitionOffsetsRequest = new HashMap<>(partitions.size());
- for (KafkaTopicPartition partition : partitions) {
- partitionOffsetsRequest.put(
- new TopicPartition(partition.getTopic(), partition.getPartition()), timestamp);
- }
-
- final Map<KafkaTopicPartition, Long> result = new HashMap<>(partitions.size());
- // use a short-lived consumer to fetch the offsets;
- // this is ok because this is a one-time operation that happens only on startup
- try (KafkaConsumer<?, ?> consumer = new KafkaConsumer(properties)) {
- for (Map.Entry<TopicPartition, OffsetAndTimestamp> partitionToOffset :
- consumer.offsetsForTimes(partitionOffsetsRequest).entrySet()) {
-
- result.put(
- new KafkaTopicPartition(
- partitionToOffset.getKey().topic(),
- partitionToOffset.getKey().partition()),
- (partitionToOffset.getValue() == null)
- ? null
- : partitionToOffset.getValue().offset());
- }
- }
- return result;
- }
-
- @Override
- protected boolean getIsAutoCommitEnabled() {
- return getBoolean(properties, ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true)
- && PropertiesUtil.getLong(
- properties, ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 5000)
- > 0;
- }
-
- /**
- * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties.
- *
- * @param props The Kafka properties to register the serializer in.
- */
- private static void setDeserializer(Properties props) {
- final String deSerName = ByteArrayDeserializer.class.getName();
-
- Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
- Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
-
- if (keyDeSer != null && !keyDeSer.equals(deSerName)) {
- LOG.warn(
- "Ignoring configured key DeSerializer ({})",
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
- }
- if (valDeSer != null && !valDeSer.equals(deSerName)) {
- LOG.warn(
- "Ignoring configured value DeSerializer ({})",
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
- }
-
- props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName);
- props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
deleted file mode 100644
index f291b05..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ /dev/null
@@ -1,1227 +0,0 @@
-/*
- * 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.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters;
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitModes;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionAssigner;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.runtime.operators.util.AssignerWithPeriodicWatermarksAdapter;
-import org.apache.flink.streaming.runtime.operators.util.AssignerWithPunctuatedWatermarksAdapter;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.COMMITS_FAILED_METRICS_COUNTER;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.COMMITS_SUCCEEDED_METRICS_COUNTER;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.KAFKA_CONSUMER_METRICS_GROUP;
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Base class of all Flink Kafka Consumer data sources. This implements the common behavior across
- * all Kafka versions.
- *
- * <p>The Kafka version specific behavior is defined mainly in the specific subclasses of the {@link
- * AbstractFetcher}.
- *
- * @param <T> The type of records produced by this data source
- */
-@Internal
-public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T>
- implements CheckpointListener, ResultTypeQueryable<T>, CheckpointedFunction {
-
- private static final long serialVersionUID = -6272159445203409112L;
-
- protected static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
-
- /** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks. */
- public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
-
- /**
- * The default interval to execute partition discovery, in milliseconds ({@code Long.MIN_VALUE},
- * i.e. disabled by default).
- */
- public static final long PARTITION_DISCOVERY_DISABLED = Long.MIN_VALUE;
-
- /** Boolean configuration key to disable metrics tracking. * */
- public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
-
- /** Configuration key to define the consumer's partition discovery interval, in milliseconds. */
- public static final String KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS =
- "flink.partition-discovery.interval-millis";
-
- /** State name of the consumer's partition offset states. */
- private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
-
- // ------------------------------------------------------------------------
- // configuration state, set on the client relevant for all subtasks
- // ------------------------------------------------------------------------
-
- /** Describes whether we are discovering partitions for fixed topics or a topic pattern. */
- private final KafkaTopicsDescriptor topicsDescriptor;
-
- /** The schema to convert between Kafka's byte messages, and Flink's objects. */
- protected final KafkaDeserializationSchema<T> deserializer;
-
- /**
- * The set of topic partitions that the source will read, with their initial offsets to start
- * reading from.
- */
- private Map<KafkaTopicPartition, Long> subscribedPartitionsToStartOffsets;
-
- /**
- * Optional watermark strategy that will be run per Kafka partition, to exploit per-partition
- * timestamp characteristics. The watermark strategy is kept in serialized form, to deserialize
- * it into multiple copies.
- */
- private SerializedValue<WatermarkStrategy<T>> watermarkStrategy;
-
- /**
- * User-set flag determining whether or not to commit on checkpoints. Note: this flag does not
- * represent the final offset commit mode.
- */
- private boolean enableCommitOnCheckpoints = true;
-
- /** User-set flag to disable filtering restored partitions with current topics descriptor. */
- private boolean filterRestoredPartitionsWithCurrentTopicsDescriptor = true;
-
- /**
- * The offset commit mode for the consumer. The value of this can only be determined in {@link
- * FlinkKafkaConsumerBase#open(Configuration)} since it depends on whether or not checkpointing
- * is enabled for the job.
- */
- private OffsetCommitMode offsetCommitMode;
-
- /** User configured value for discovery interval, in milliseconds. */
- private final long discoveryIntervalMillis;
-
- /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}). */
- private StartupMode startupMode = StartupMode.GROUP_OFFSETS;
-
- /**
- * Specific startup offsets; only relevant when startup mode is {@link
- * StartupMode#SPECIFIC_OFFSETS}.
- */
- private Map<KafkaTopicPartition, Long> specificStartupOffsets;
-
- /**
- * Timestamp to determine startup offsets; only relevant when startup mode is {@link
- * StartupMode#TIMESTAMP}.
- */
- private Long startupOffsetsTimestamp;
-
- // ------------------------------------------------------------------------
- // runtime state (used individually by each parallel subtask)
- // ------------------------------------------------------------------------
-
- /** Data for pending but uncommitted offsets. */
- private final LinkedMap pendingOffsetsToCommit = new LinkedMap();
-
- /** The fetcher implements the connections to the Kafka brokers. */
- private transient volatile AbstractFetcher<T, ?> kafkaFetcher;
-
- /** The partition discoverer, used to find new partitions. */
- private transient volatile AbstractPartitionDiscoverer partitionDiscoverer;
-
- /**
- * The offsets to restore to, if the consumer restores state from a checkpoint.
- *
- * <p>This map will be populated by the {@link #initializeState(FunctionInitializationContext)}
- * method.
- *
- * <p>Using a sorted map as the ordering is important when using restored state to seed the
- * partition discoverer.
- */
- private transient volatile TreeMap<KafkaTopicPartition, Long> restoredState;
-
- /** Accessor for state in the operator state backend. */
- private transient ListState<Tuple2<KafkaTopicPartition, Long>> unionOffsetStates;
-
- /** Discovery loop, executed in a separate thread. */
- private transient volatile Thread discoveryLoopThread;
-
- /** Flag indicating whether the consumer is still running. */
- private volatile boolean running = true;
-
- // ------------------------------------------------------------------------
- // internal metrics
- // ------------------------------------------------------------------------
-
- /**
- * Flag indicating whether or not metrics should be exposed. If {@code true}, offset metrics
- * (e.g. current offset, committed offset) and Kafka-shipped metrics will be registered.
- */
- private final boolean useMetrics;
-
- /** Counter for successful Kafka offset commits. */
- private transient Counter successfulCommits;
-
- /** Counter for failed Kafka offset commits. */
- private transient Counter failedCommits;
-
- /**
- * Callback interface that will be invoked upon async Kafka commit completion. Please be aware
- * that default callback implementation in base class does not provide any guarantees on
- * thread-safety. This is sufficient for now because current supported Kafka connectors
- * guarantee no more than 1 concurrent async pending offset commit.
- */
- private transient KafkaCommitCallback offsetCommitCallback;
-
- // ------------------------------------------------------------------------
-
- /**
- * Base constructor.
- *
- * @param topics fixed list of topics to subscribe to (null, if using topic pattern)
- * @param topicPattern the topic pattern to subscribe to (null, if using fixed topics)
- * @param deserializer The deserializer to turn raw byte messages into Java/Scala objects.
- * @param discoveryIntervalMillis the topic / partition discovery interval, in milliseconds (0
- * if discovery is disabled).
- */
- public FlinkKafkaConsumerBase(
- List<String> topics,
- Pattern topicPattern,
- KafkaDeserializationSchema<T> deserializer,
- long discoveryIntervalMillis,
- boolean useMetrics) {
- this.topicsDescriptor = new KafkaTopicsDescriptor(topics, topicPattern);
- this.deserializer = checkNotNull(deserializer, "valueDeserializer");
-
- checkArgument(
- discoveryIntervalMillis == PARTITION_DISCOVERY_DISABLED
- || discoveryIntervalMillis >= 0,
- "Cannot define a negative value for the topic / partition discovery interval.");
- this.discoveryIntervalMillis = discoveryIntervalMillis;
-
- this.useMetrics = useMetrics;
- }
-
- /**
- * Make sure that auto commit is disabled when our offset commit mode is ON_CHECKPOINTS. This
- * overwrites whatever setting the user configured in the properties.
- *
- * @param properties - Kafka configuration properties to be adjusted
- * @param offsetCommitMode offset commit mode
- */
- protected static void adjustAutoCommitConfig(
- Properties properties, OffsetCommitMode offsetCommitMode) {
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS
- || offsetCommitMode == OffsetCommitMode.DISABLED) {
- properties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
- }
- }
- // ------------------------------------------------------------------------
- // Configuration
- // ------------------------------------------------------------------------
-
- /**
- * Sets the given {@link WatermarkStrategy} on this consumer. These will be used to assign
- * timestamps to records and generates watermarks to signal event time progress.
- *
- * <p>Running timestamp extractors / watermark generators directly inside the Kafka source
- * (which you can do by using this method), per Kafka partition, allows users to let them
- * exploit the per-partition characteristics.
- *
- * <p>When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, the streams
- * from the partitions are unioned in a "first come first serve" fashion. Per-partition
- * characteristics are usually lost that way. For example, if the timestamps are strictly
- * ascending per Kafka partition, they will not be strictly ascending in the resulting Flink
- * DataStream, if the parallel source subtask reads more than one partition.
- *
- * <p>Common watermark generation patterns can be found as static methods in the {@link
- * org.apache.flink.api.common.eventtime.WatermarkStrategy} class.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> assignTimestampsAndWatermarks(
- WatermarkStrategy<T> watermarkStrategy) {
- checkNotNull(watermarkStrategy);
-
- try {
- ClosureCleaner.clean(
- watermarkStrategy, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- this.watermarkStrategy = new SerializedValue<>(watermarkStrategy);
- } catch (Exception e) {
- throw new IllegalArgumentException(
- "The given WatermarkStrategy is not serializable", e);
- }
-
- return this;
- }
-
- /**
- * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated
- * manner. The watermark extractor will run per Kafka partition, watermarks will be merged
- * across partitions in the same way as in the Flink runtime, when streams are merged.
- *
- * <p>When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, the streams
- * from the partitions are unioned in a "first come first serve" fashion. Per-partition
- * characteristics are usually lost that way. For example, if the timestamps are strictly
- * ascending per Kafka partition, they will not be strictly ascending in the resulting Flink
- * DataStream, if the parallel source subtask reads more than one partition.
- *
- * <p>Running timestamp extractors / watermark generators directly inside the Kafka source, per
- * Kafka partition, allows users to let them exploit the per-partition characteristics.
- *
- * <p>Note: One can use either an {@link AssignerWithPunctuatedWatermarks} or an {@link
- * AssignerWithPeriodicWatermarks}, not both at the same time.
- *
- * <p>This method uses the deprecated watermark generator interfaces. Please switch to {@link
- * #assignTimestampsAndWatermarks(WatermarkStrategy)} to use the new interfaces instead. The new
- * interfaces support watermark idleness and no longer need to differentiate between "periodic"
- * and "punctuated" watermarks.
- *
- * @deprecated Please use {@link #assignTimestampsAndWatermarks(WatermarkStrategy)} instead.
- * @param assigner The timestamp assigner / watermark generator to use.
- * @return The consumer object, to allow function chaining.
- */
- @Deprecated
- public FlinkKafkaConsumerBase<T> assignTimestampsAndWatermarks(
- AssignerWithPunctuatedWatermarks<T> assigner) {
- checkNotNull(assigner);
-
- if (this.watermarkStrategy != null) {
- throw new IllegalStateException("Some watermark strategy has already been set.");
- }
-
- try {
- ClosureCleaner.clean(assigner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- final WatermarkStrategy<T> wms =
- new AssignerWithPunctuatedWatermarksAdapter.Strategy<>(assigner);
-
- return assignTimestampsAndWatermarks(wms);
- } catch (Exception e) {
- throw new IllegalArgumentException("The given assigner is not serializable", e);
- }
- }
-
- /**
- * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated
- * manner. The watermark extractor will run per Kafka partition, watermarks will be merged
- * across partitions in the same way as in the Flink runtime, when streams are merged.
- *
- * <p>When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions, the streams
- * from the partitions are unioned in a "first come first serve" fashion. Per-partition
- * characteristics are usually lost that way. For example, if the timestamps are strictly
- * ascending per Kafka partition, they will not be strictly ascending in the resulting Flink
- * DataStream, if the parallel source subtask reads more that one partition.
- *
- * <p>Running timestamp extractors / watermark generators directly inside the Kafka source, per
- * Kafka partition, allows users to let them exploit the per-partition characteristics.
- *
- * <p>Note: One can use either an {@link AssignerWithPunctuatedWatermarks} or an {@link
- * AssignerWithPeriodicWatermarks}, not both at the same time.
- *
- * <p>This method uses the deprecated watermark generator interfaces. Please switch to {@link
- * #assignTimestampsAndWatermarks(WatermarkStrategy)} to use the new interfaces instead. The new
- * interfaces support watermark idleness and no longer need to differentiate between "periodic"
- * and "punctuated" watermarks.
- *
- * @deprecated Please use {@link #assignTimestampsAndWatermarks(WatermarkStrategy)} instead.
- * @param assigner The timestamp assigner / watermark generator to use.
- * @return The consumer object, to allow function chaining.
- */
- @Deprecated
- public FlinkKafkaConsumerBase<T> assignTimestampsAndWatermarks(
- AssignerWithPeriodicWatermarks<T> assigner) {
- checkNotNull(assigner);
-
- if (this.watermarkStrategy != null) {
- throw new IllegalStateException("Some watermark strategy has already been set.");
- }
-
- try {
- ClosureCleaner.clean(assigner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- final WatermarkStrategy<T> wms =
- new AssignerWithPeriodicWatermarksAdapter.Strategy<>(assigner);
-
- return assignTimestampsAndWatermarks(wms);
- } catch (Exception e) {
- throw new IllegalArgumentException("The given assigner is not serializable", e);
- }
- }
-
- /**
- * Specifies whether or not the consumer should commit offsets back to Kafka on checkpoints.
- *
- * <p>This setting will only have effect if checkpointing is enabled for the job. If
- * checkpointing isn't enabled, only the "auto.commit.enable" (for 0.8) / "enable.auto.commit"
- * (for 0.9+) property settings will be used.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setCommitOffsetsOnCheckpoints(boolean commitOnCheckpoints) {
- this.enableCommitOnCheckpoints = commitOnCheckpoints;
- return this;
- }
-
- /**
- * Specifies the consumer to start reading from the earliest offset for all partitions. This
- * lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers.
- *
- * <p>This method does not affect where partitions are read from when the consumer is restored
- * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint,
- * only the offsets in the restored state will be used.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setStartFromEarliest() {
- this.startupMode = StartupMode.EARLIEST;
- this.startupOffsetsTimestamp = null;
- this.specificStartupOffsets = null;
- return this;
- }
-
- /**
- * Specifies the consumer to start reading from the latest offset for all partitions. This lets
- * the consumer ignore any committed group offsets in Zookeeper / Kafka brokers.
- *
- * <p>This method does not affect where partitions are read from when the consumer is restored
- * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint,
- * only the offsets in the restored state will be used.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setStartFromLatest() {
- this.startupMode = StartupMode.LATEST;
- this.startupOffsetsTimestamp = null;
- this.specificStartupOffsets = null;
- return this;
- }
-
- /**
- * Specifies the consumer to start reading partitions from a specified timestamp. The specified
- * timestamp must be before the current timestamp. This lets the consumer ignore any committed
- * group offsets in Zookeeper / Kafka brokers.
- *
- * <p>The consumer will look up the earliest offset whose timestamp is greater than or equal to
- * the specific timestamp from Kafka. If there's no such offset, the consumer will use the
- * latest offset to read data from kafka.
- *
- * <p>This method does not affect where partitions are read from when the consumer is restored
- * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint,
- * only the offsets in the restored state will be used.
- *
- * @param startupOffsetsTimestamp timestamp for the startup offsets, as milliseconds from epoch.
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setStartFromTimestamp(long startupOffsetsTimestamp) {
- checkArgument(
- startupOffsetsTimestamp >= 0,
- "The provided value for the startup offsets timestamp is invalid.");
-
- long currentTimestamp = System.currentTimeMillis();
- checkArgument(
- startupOffsetsTimestamp <= currentTimestamp,
- "Startup time[%s] must be before current time[%s].",
- startupOffsetsTimestamp,
- currentTimestamp);
-
- this.startupMode = StartupMode.TIMESTAMP;
- this.startupOffsetsTimestamp = startupOffsetsTimestamp;
- this.specificStartupOffsets = null;
- return this;
- }
-
- /**
- * Specifies the consumer to start reading from any committed group offsets found in Zookeeper /
- * Kafka brokers. The "group.id" property must be set in the configuration properties. If no
- * offset can be found for a partition, the behaviour in "auto.offset.reset" set in the
- * configuration properties will be used for the partition.
- *
- * <p>This method does not affect where partitions are read from when the consumer is restored
- * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint,
- * only the offsets in the restored state will be used.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setStartFromGroupOffsets() {
- this.startupMode = StartupMode.GROUP_OFFSETS;
- this.startupOffsetsTimestamp = null;
- this.specificStartupOffsets = null;
- return this;
- }
-
- /**
- * Specifies the consumer to start reading partitions from specific offsets, set independently
- * for each partition. The specified offset should be the offset of the next record that will be
- * read from partitions. This lets the consumer ignore any committed group offsets in Zookeeper
- * / Kafka brokers.
- *
- * <p>If the provided map of offsets contains entries whose {@link KafkaTopicPartition} is not
- * subscribed by the consumer, the entry will be ignored. If the consumer subscribes to a
- * partition that does not exist in the provided map of offsets, the consumer will fallback to
- * the default group offset behaviour (see {@link
- * FlinkKafkaConsumerBase#setStartFromGroupOffsets()}) for that particular partition.
- *
- * <p>If the specified offset for a partition is invalid, or the behaviour for that partition is
- * defaulted to group offsets but still no group offset could be found for it, then the
- * "auto.offset.reset" behaviour set in the configuration properties will be used for the
- * partition
- *
- * <p>This method does not affect where partitions are read from when the consumer is restored
- * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or savepoint,
- * only the offsets in the restored state will be used.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> setStartFromSpecificOffsets(
- Map<KafkaTopicPartition, Long> specificStartupOffsets) {
- this.startupMode = StartupMode.SPECIFIC_OFFSETS;
- this.startupOffsetsTimestamp = null;
- this.specificStartupOffsets = checkNotNull(specificStartupOffsets);
- return this;
- }
-
- /**
- * By default, when restoring from a checkpoint / savepoint, the consumer always ignores
- * restored partitions that are no longer associated with the current specified topics or topic
- * pattern to subscribe to.
- *
- * <p>This method configures the consumer to not filter the restored partitions, therefore
- * always attempting to consume whatever partition was present in the previous execution
- * regardless of the specified topics to subscribe to in the current execution.
- *
- * @return The consumer object, to allow function chaining.
- */
- public FlinkKafkaConsumerBase<T> disableFilterRestoredPartitionsWithSubscribedTopics() {
- this.filterRestoredPartitionsWithCurrentTopicsDescriptor = false;
- return this;
- }
-
- // ------------------------------------------------------------------------
- // Work methods
- // ------------------------------------------------------------------------
-
- @Override
- public void open(Configuration configuration) throws Exception {
- // determine the offset commit mode
- this.offsetCommitMode =
- OffsetCommitModes.fromConfiguration(
- getIsAutoCommitEnabled(),
- enableCommitOnCheckpoints,
- ((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled());
-
- // create the partition discoverer
- this.partitionDiscoverer =
- createPartitionDiscoverer(
- topicsDescriptor,
- getRuntimeContext().getIndexOfThisSubtask(),
- getRuntimeContext().getNumberOfParallelSubtasks());
- this.partitionDiscoverer.open();
-
- subscribedPartitionsToStartOffsets = new HashMap<>();
- final List<KafkaTopicPartition> allPartitions = partitionDiscoverer.discoverPartitions();
- if (restoredState != null) {
- for (KafkaTopicPartition partition : allPartitions) {
- if (!restoredState.containsKey(partition)) {
- restoredState.put(partition, KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET);
- }
- }
-
- for (Map.Entry<KafkaTopicPartition, Long> restoredStateEntry :
- restoredState.entrySet()) {
- // seed the partition discoverer with the union state while filtering out
- // restored partitions that should not be subscribed by this subtask
- if (KafkaTopicPartitionAssigner.assign(
- restoredStateEntry.getKey(),
- getRuntimeContext().getNumberOfParallelSubtasks())
- == getRuntimeContext().getIndexOfThisSubtask()) {
- subscribedPartitionsToStartOffsets.put(
- restoredStateEntry.getKey(), restoredStateEntry.getValue());
- }
- }
-
- if (filterRestoredPartitionsWithCurrentTopicsDescriptor) {
- subscribedPartitionsToStartOffsets
- .entrySet()
- .removeIf(
- entry -> {
- if (!topicsDescriptor.isMatchingTopic(
- entry.getKey().getTopic())) {
- LOG.warn(
- "{} is removed from subscribed partitions since it is no longer associated with topics descriptor of current execution.",
- entry.getKey());
- return true;
- }
- return false;
- });
- }
-
- LOG.info(
- "Consumer subtask {} will start reading {} partitions with offsets in restored state: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- subscribedPartitionsToStartOffsets);
- } else {
- // use the partition discoverer to fetch the initial seed partitions,
- // and set their initial offsets depending on the startup mode.
- // for SPECIFIC_OFFSETS and TIMESTAMP modes, we set the specific offsets now;
- // for other modes (EARLIEST, LATEST, and GROUP_OFFSETS), the offset is lazily
- // determined
- // when the partition is actually read.
- switch (startupMode) {
- case SPECIFIC_OFFSETS:
- if (specificStartupOffsets == null) {
- throw new IllegalStateException(
- "Startup mode for the consumer set to "
- + StartupMode.SPECIFIC_OFFSETS
- + ", but no specific offsets were specified.");
- }
-
- for (KafkaTopicPartition seedPartition : allPartitions) {
- Long specificOffset = specificStartupOffsets.get(seedPartition);
- if (specificOffset != null) {
- // since the specified offsets represent the next record to read, we
- // subtract
- // it by one so that the initial state of the consumer will be correct
- subscribedPartitionsToStartOffsets.put(
- seedPartition, specificOffset - 1);
- } else {
- // default to group offset behaviour if the user-provided specific
- // offsets
- // do not contain a value for this partition
- subscribedPartitionsToStartOffsets.put(
- seedPartition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
- }
- }
-
- break;
- case TIMESTAMP:
- if (startupOffsetsTimestamp == null) {
- throw new IllegalStateException(
- "Startup mode for the consumer set to "
- + StartupMode.TIMESTAMP
- + ", but no startup timestamp was specified.");
- }
-
- for (Map.Entry<KafkaTopicPartition, Long> partitionToOffset :
- fetchOffsetsWithTimestamp(allPartitions, startupOffsetsTimestamp)
- .entrySet()) {
- subscribedPartitionsToStartOffsets.put(
- partitionToOffset.getKey(),
- (partitionToOffset.getValue() == null)
- // if an offset cannot be retrieved for a partition with the
- // given timestamp,
- // we default to using the latest offset for the partition
- ? KafkaTopicPartitionStateSentinel.LATEST_OFFSET
- // since the specified offsets represent the next record to
- // read, we subtract
- // it by one so that the initial state of the consumer will
- // be correct
- : partitionToOffset.getValue() - 1);
- }
-
- break;
- default:
- for (KafkaTopicPartition seedPartition : allPartitions) {
- subscribedPartitionsToStartOffsets.put(
- seedPartition, startupMode.getStateSentinel());
- }
- }
-
- if (!subscribedPartitionsToStartOffsets.isEmpty()) {
- switch (startupMode) {
- case EARLIEST:
- LOG.info(
- "Consumer subtask {} will start reading the following {} partitions from the earliest offsets: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- subscribedPartitionsToStartOffsets.keySet());
- break;
- case LATEST:
- LOG.info(
- "Consumer subtask {} will start reading the following {} partitions from the latest offsets: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- subscribedPartitionsToStartOffsets.keySet());
- break;
- case TIMESTAMP:
- LOG.info(
- "Consumer subtask {} will start reading the following {} partitions from timestamp {}: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- startupOffsetsTimestamp,
- subscribedPartitionsToStartOffsets.keySet());
- break;
- case SPECIFIC_OFFSETS:
- LOG.info(
- "Consumer subtask {} will start reading the following {} partitions from the specified startup offsets {}: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- specificStartupOffsets,
- subscribedPartitionsToStartOffsets.keySet());
-
- List<KafkaTopicPartition> partitionsDefaultedToGroupOffsets =
- new ArrayList<>(subscribedPartitionsToStartOffsets.size());
- for (Map.Entry<KafkaTopicPartition, Long> subscribedPartition :
- subscribedPartitionsToStartOffsets.entrySet()) {
- if (subscribedPartition.getValue()
- == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) {
- partitionsDefaultedToGroupOffsets.add(subscribedPartition.getKey());
- }
- }
-
- if (partitionsDefaultedToGroupOffsets.size() > 0) {
- LOG.warn(
- "Consumer subtask {} cannot find offsets for the following {} partitions in the specified startup offsets: {}"
- + "; their startup offsets will be defaulted to their committed group offsets in Kafka.",
- getRuntimeContext().getIndexOfThisSubtask(),
- partitionsDefaultedToGroupOffsets.size(),
- partitionsDefaultedToGroupOffsets);
- }
- break;
- case GROUP_OFFSETS:
- LOG.info(
- "Consumer subtask {} will start reading the following {} partitions from the committed group offsets in Kafka: {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets.size(),
- subscribedPartitionsToStartOffsets.keySet());
- }
- } else {
- LOG.info(
- "Consumer subtask {} initially has no partitions to read from.",
- getRuntimeContext().getIndexOfThisSubtask());
- }
- }
-
- this.deserializer.open(
- RuntimeContextInitializationContextAdapters.deserializationAdapter(
- getRuntimeContext(), metricGroup -> metricGroup.addGroup("user")));
- }
-
- @Override
- public void run(SourceContext<T> sourceContext) throws Exception {
- if (subscribedPartitionsToStartOffsets == null) {
- throw new Exception("The partitions were not set for the consumer");
- }
-
- // initialize commit metrics and default offset callback method
- this.successfulCommits =
- this.getRuntimeContext()
- .getMetricGroup()
- .counter(COMMITS_SUCCEEDED_METRICS_COUNTER);
- this.failedCommits =
- this.getRuntimeContext().getMetricGroup().counter(COMMITS_FAILED_METRICS_COUNTER);
- final int subtaskIndex = this.getRuntimeContext().getIndexOfThisSubtask();
-
- this.offsetCommitCallback =
- new KafkaCommitCallback() {
- @Override
- public void onSuccess() {
- successfulCommits.inc();
- }
-
- @Override
- public void onException(Throwable cause) {
- LOG.warn(
- String.format(
- "Consumer subtask %d failed async Kafka commit.",
- subtaskIndex),
- cause);
- failedCommits.inc();
- }
- };
-
- // mark the subtask as temporarily idle if there are no initial seed partitions;
- // once this subtask discovers some partitions and starts collecting records, the subtask's
- // status will automatically be triggered back to be active.
- if (subscribedPartitionsToStartOffsets.isEmpty()) {
- sourceContext.markAsTemporarilyIdle();
- }
-
- LOG.info(
- "Consumer subtask {} creating fetcher with offsets {}.",
- getRuntimeContext().getIndexOfThisSubtask(),
- subscribedPartitionsToStartOffsets);
- // from this point forward:
- // - 'snapshotState' will draw offsets from the fetcher,
- // instead of being built from `subscribedPartitionsToStartOffsets`
- // - 'notifyCheckpointComplete' will start to do work (i.e. commit offsets to
- // Kafka through the fetcher, if configured to do so)
- this.kafkaFetcher =
- createFetcher(
- sourceContext,
- subscribedPartitionsToStartOffsets,
- watermarkStrategy,
- (StreamingRuntimeContext) getRuntimeContext(),
- offsetCommitMode,
- getRuntimeContext().getMetricGroup().addGroup(KAFKA_CONSUMER_METRICS_GROUP),
- useMetrics);
-
- if (!running) {
- return;
- }
-
- // depending on whether we were restored with the current state version (1.3),
- // remaining logic branches off into 2 paths:
- // 1) New state - partition discovery loop executed as separate thread, with this
- // thread running the main fetcher loop
- // 2) Old state - partition discovery is disabled and only the main fetcher loop is
- // executed
- if (discoveryIntervalMillis == PARTITION_DISCOVERY_DISABLED) {
- kafkaFetcher.runFetchLoop();
- } else {
- runWithPartitionDiscovery();
- }
- }
-
- private void runWithPartitionDiscovery() throws Exception {
- final AtomicReference<Exception> discoveryLoopErrorRef = new AtomicReference<>();
- createAndStartDiscoveryLoop(discoveryLoopErrorRef);
-
- kafkaFetcher.runFetchLoop();
-
- // make sure that the partition discoverer is waked up so that
- // the discoveryLoopThread exits
- partitionDiscoverer.wakeup();
- joinDiscoveryLoopThread();
-
- // rethrow any fetcher errors
- final Exception discoveryLoopError = discoveryLoopErrorRef.get();
- if (discoveryLoopError != null) {
- throw new RuntimeException(discoveryLoopError);
- }
- }
-
- @VisibleForTesting
- void joinDiscoveryLoopThread() throws InterruptedException {
- if (discoveryLoopThread != null) {
- discoveryLoopThread.join();
- }
- }
-
- private void createAndStartDiscoveryLoop(AtomicReference<Exception> discoveryLoopErrorRef) {
- discoveryLoopThread =
- new Thread(
- () -> {
- try {
- // --------------------- partition discovery loop
- // ---------------------
-
- // throughout the loop, we always eagerly check if we are still
- // running before
- // performing the next operation, so that we can escape the loop as
- // soon as possible
-
- while (running) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Consumer subtask {} is trying to discover new partitions ...",
- getRuntimeContext().getIndexOfThisSubtask());
- }
-
- final List<KafkaTopicPartition> discoveredPartitions;
- try {
- discoveredPartitions =
- partitionDiscoverer.discoverPartitions();
- } catch (AbstractPartitionDiscoverer.WakeupException
- | AbstractPartitionDiscoverer.ClosedException e) {
- // the partition discoverer may have been closed or woken up
- // before or during the discovery;
- // this would only happen if the consumer was canceled;
- // simply escape the loop
- break;
- }
-
- // no need to add the discovered partitions if we were closed
- // during the meantime
- if (running && !discoveredPartitions.isEmpty()) {
- kafkaFetcher.addDiscoveredPartitions(discoveredPartitions);
- }
-
- // do not waste any time sleeping if we're not running anymore
- if (running && discoveryIntervalMillis != 0) {
- try {
- Thread.sleep(discoveryIntervalMillis);
- } catch (InterruptedException iex) {
- // may be interrupted if the consumer was canceled
- // midway; simply escape the loop
- break;
- }
- }
- }
- } catch (Exception e) {
- discoveryLoopErrorRef.set(e);
- } finally {
- // calling cancel will also let the fetcher loop escape
- // (if not running, cancel() was already called)
- if (running) {
- cancel();
- }
- }
- },
- "Kafka Partition Discovery for "
- + getRuntimeContext().getTaskNameWithSubtasks());
-
- discoveryLoopThread.start();
- }
-
- @Override
- public void cancel() {
- // set ourselves as not running;
- // this would let the main discovery loop escape as soon as possible
- running = false;
-
- if (discoveryLoopThread != null) {
-
- if (partitionDiscoverer != null) {
- // we cannot close the discoverer here, as it is error-prone to concurrent access;
- // only wakeup the discoverer, the discovery loop will clean itself up after it
- // escapes
- partitionDiscoverer.wakeup();
- }
-
- // the discovery loop may currently be sleeping in-between
- // consecutive discoveries; interrupt to shutdown faster
- discoveryLoopThread.interrupt();
- }
-
- // abort the fetcher, if there is one
- if (kafkaFetcher != null) {
- kafkaFetcher.cancel();
- }
- }
-
- @Override
- public void close() throws Exception {
- cancel();
-
- joinDiscoveryLoopThread();
-
- Exception exception = null;
- if (partitionDiscoverer != null) {
- try {
- partitionDiscoverer.close();
- } catch (Exception e) {
- exception = e;
- }
- }
-
- try {
- super.close();
- } catch (Exception e) {
- exception = ExceptionUtils.firstOrSuppressed(e, exception);
- }
-
- if (exception != null) {
- throw exception;
- }
- }
-
- // ------------------------------------------------------------------------
- // Checkpoint and restore
- // ------------------------------------------------------------------------
-
- @Override
- public final void initializeState(FunctionInitializationContext context) throws Exception {
-
- OperatorStateStore stateStore = context.getOperatorStateStore();
-
- this.unionOffsetStates =
- stateStore.getUnionListState(
- new ListStateDescriptor<>(
- OFFSETS_STATE_NAME,
- createStateSerializer(getRuntimeContext().getExecutionConfig())));
-
- if (context.isRestored()) {
- restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator());
-
- // populate actual holder for restored state
- for (Tuple2<KafkaTopicPartition, Long> kafkaOffset : unionOffsetStates.get()) {
- restoredState.put(kafkaOffset.f0, kafkaOffset.f1);
- }
-
- LOG.info(
- "Consumer subtask {} restored state: {}.",
- getRuntimeContext().getIndexOfThisSubtask(),
- restoredState);
- } else {
- LOG.info(
- "Consumer subtask {} has no restore state.",
- getRuntimeContext().getIndexOfThisSubtask());
- }
- }
-
- @Override
- public final void snapshotState(FunctionSnapshotContext context) throws Exception {
- if (!running) {
- LOG.debug("snapshotState() called on closed source");
- } else {
- unionOffsetStates.clear();
-
- final AbstractFetcher<?, ?> fetcher = this.kafkaFetcher;
- if (fetcher == null) {
- // the fetcher has not yet been initialized, which means we need to return the
- // originally restored offsets or the assigned partitions
- for (Map.Entry<KafkaTopicPartition, Long> subscribedPartition :
- subscribedPartitionsToStartOffsets.entrySet()) {
- unionOffsetStates.add(
- Tuple2.of(
- subscribedPartition.getKey(), subscribedPartition.getValue()));
- }
-
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS) {
- // the map cannot be asynchronously updated, because only one checkpoint call
- // can happen
- // on this function at a time: either snapshotState() or
- // notifyCheckpointComplete()
- pendingOffsetsToCommit.put(context.getCheckpointId(), restoredState);
- }
- } else {
- HashMap<KafkaTopicPartition, Long> currentOffsets = fetcher.snapshotCurrentState();
-
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS) {
- // the map cannot be asynchronously updated, because only one checkpoint call
- // can happen
- // on this function at a time: either snapshotState() or
- // notifyCheckpointComplete()
- pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
- }
-
- for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry :
- currentOffsets.entrySet()) {
- unionOffsetStates.add(
- Tuple2.of(
- kafkaTopicPartitionLongEntry.getKey(),
- kafkaTopicPartitionLongEntry.getValue()));
- }
- }
-
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS) {
- // truncate the map of pending offsets to commit, to prevent infinite growth
- while (pendingOffsetsToCommit.size() > MAX_NUM_PENDING_CHECKPOINTS) {
- pendingOffsetsToCommit.remove(0);
- }
- }
- }
- }
-
- @Override
- public final void notifyCheckpointComplete(long checkpointId) throws Exception {
- if (!running) {
- LOG.debug("notifyCheckpointComplete() called on closed source");
- return;
- }
-
- final AbstractFetcher<?, ?> fetcher = this.kafkaFetcher;
- if (fetcher == null) {
- LOG.debug("notifyCheckpointComplete() called on uninitialized source");
- return;
- }
-
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS) {
- // only one commit operation must be in progress
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Consumer subtask {} committing offsets to Kafka/ZooKeeper for checkpoint {}.",
- getRuntimeContext().getIndexOfThisSubtask(),
- checkpointId);
- }
-
- try {
- final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
- if (posInMap == -1) {
- LOG.warn(
- "Consumer subtask {} received confirmation for unknown checkpoint id {}",
- getRuntimeContext().getIndexOfThisSubtask(),
- checkpointId);
- return;
- }
-
- @SuppressWarnings("unchecked")
- Map<KafkaTopicPartition, Long> offsets =
- (Map<KafkaTopicPartition, Long>) pendingOffsetsToCommit.remove(posInMap);
-
- // remove older checkpoints in map
- for (int i = 0; i < posInMap; i++) {
- pendingOffsetsToCommit.remove(0);
- }
-
- if (offsets == null || offsets.size() == 0) {
- LOG.debug(
- "Consumer subtask {} has empty checkpoint state.",
- getRuntimeContext().getIndexOfThisSubtask());
- return;
- }
-
- fetcher.commitInternalOffsetsToKafka(offsets, offsetCommitCallback);
- } catch (Exception e) {
- if (running) {
- throw e;
- }
- // else ignore exception if we are no longer running
- }
- }
- }
-
- @Override
- public void notifyCheckpointAborted(long checkpointId) {}
-
- // ------------------------------------------------------------------------
- // Kafka Consumer specific methods
- // ------------------------------------------------------------------------
-
- /**
- * Creates the fetcher that connect to the Kafka brokers, pulls data, deserialized the data, and
- * emits it into the data streams.
- *
- * @param sourceContext The source context to emit data to.
- * @param subscribedPartitionsToStartOffsets The set of partitions that this subtask should
- * handle, with their start offsets.
- * @param watermarkStrategy Optional, a serialized WatermarkStrategy.
- * @param runtimeContext The task's runtime context.
- * @return The instantiated fetcher
- * @throws Exception The method should forward exceptions
- */
- protected abstract AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> subscribedPartitionsToStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup kafkaMetricGroup,
- boolean useMetrics)
- throws Exception;
-
- /**
- * Creates the partition discoverer that is used to find new partitions for this subtask.
- *
- * @param topicsDescriptor Descriptor that describes whether we are discovering partitions for
- * fixed topics or a topic pattern.
- * @param indexOfThisSubtask The index of this consumer subtask.
- * @param numParallelSubtasks The total number of parallel consumer subtasks.
- * @return The instantiated partition discoverer
- */
- protected abstract AbstractPartitionDiscoverer createPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks);
-
- protected abstract boolean getIsAutoCommitEnabled();
-
- protected abstract Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
- Collection<KafkaTopicPartition> partitions, long timestamp);
-
- // ------------------------------------------------------------------------
- // ResultTypeQueryable methods
- // ------------------------------------------------------------------------
-
- @Override
- public TypeInformation<T> getProducedType() {
- return deserializer.getProducedType();
- }
-
- // ------------------------------------------------------------------------
- // Test utilities
- // ------------------------------------------------------------------------
-
- @VisibleForTesting
- Map<KafkaTopicPartition, Long> getSubscribedPartitionsToStartOffsets() {
- return subscribedPartitionsToStartOffsets;
- }
-
- @VisibleForTesting
- TreeMap<KafkaTopicPartition, Long> getRestoredState() {
- return restoredState;
- }
-
- @VisibleForTesting
- OffsetCommitMode getOffsetCommitMode() {
- return offsetCommitMode;
- }
-
- @VisibleForTesting
- LinkedMap getPendingOffsetsToCommit() {
- return pendingOffsetsToCommit;
- }
-
- @VisibleForTesting
- public boolean getEnableCommitOnCheckpoints() {
- return enableCommitOnCheckpoints;
- }
-
- /**
- * Creates state serializer for kafka topic partition to offset tuple. Using of the explicit
- * state serializer with KryoSerializer is needed because otherwise users cannot use
- * 'disableGenericTypes' properties with KafkaConsumer.
- */
- @VisibleForTesting
- static TupleSerializer<Tuple2<KafkaTopicPartition, Long>> createStateSerializer(
- ExecutionConfig executionConfig) {
- // explicit serializer will keep the compatibility with GenericTypeInformation and allow to
- // disableGenericTypes for users
- TypeSerializer<?>[] fieldSerializers =
- new TypeSerializer<?>[] {
- new KryoSerializer<>(KafkaTopicPartition.class, executionConfig),
- LongSerializer.INSTANCE
- };
- @SuppressWarnings("unchecked")
- Class<Tuple2<KafkaTopicPartition, Long>> tupleClass =
- (Class<Tuple2<KafkaTopicPartition, Long>>) (Class<?>) Tuple2.class;
- return new TupleSerializer<>(tupleClass, fieldSerializers);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java
deleted file mode 100644
index c11f1b1..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-
-/** Error codes used in {@link FlinkKafkaException}. */
-@PublicEvolving
-public enum FlinkKafkaErrorCode {
- PRODUCERS_POOL_EMPTY,
- EXTERNAL_ERROR
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java
deleted file mode 100644
index 77d0231..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-import org.apache.flink.util.FlinkException;
-
-/** Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. */
-@PublicEvolving
-public class FlinkKafkaException extends FlinkException {
-
- private static final long serialVersionUID = 920269130311214200L;
-
- private final FlinkKafkaErrorCode errorCode;
-
- public FlinkKafkaException(FlinkKafkaErrorCode errorCode, String message) {
- super(message);
- this.errorCode = errorCode;
- }
-
- public FlinkKafkaException(FlinkKafkaErrorCode errorCode, String message, Throwable cause) {
- super(message, cause);
- this.errorCode = errorCode;
- }
-
- public FlinkKafkaErrorCode getErrorCode() {
- return errorCode;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
deleted file mode 100644
index ca81cc8..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ /dev/null
@@ -1,1950 +0,0 @@
-/*
- * 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.annotation.Internal;
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaSerializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.internals.TransactionalIdsGenerator;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricMutableWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.TemporaryClassLoaderContext;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.errors.InvalidTxnStateException;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.BlockingDeque;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * Flink Sink to produce data into a Kafka topic. By default producer will use {@link
- * FlinkKafkaProducer.Semantic#AT_LEAST_ONCE} semantic. Before using {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE} please refer to Flink's Kafka connector documentation.
- *
- * @deprecated Please use {@link org.apache.flink.connector.kafka.sink.KafkaSink}.
- */
-@Deprecated
-@PublicEvolving
-public class FlinkKafkaProducer<IN>
- extends TwoPhaseCommitSinkFunction<
- IN,
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext> {
-
- /**
- * Semantics that can be chosen.
- * <li>{@link #EXACTLY_ONCE}
- * <li>{@link #AT_LEAST_ONCE}
- * <li>{@link #NONE}
- */
- public enum Semantic {
-
- /**
- * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction
- * that will be committed to Kafka on a checkpoint.
- *
- * <p>In this mode {@link FlinkKafkaProducer} sets up a pool of {@link
- * FlinkKafkaInternalProducer}. Between each checkpoint a Kafka transaction is created,
- * which is committed on {@link FlinkKafkaProducer#notifyCheckpointComplete(long)}. If
- * checkpoint complete notifications are running late, {@link FlinkKafkaProducer} can run
- * out of {@link FlinkKafkaInternalProducer}s in the pool. In that case any subsequent
- * {@link FlinkKafkaProducer#snapshotState(FunctionSnapshotContext)} requests will fail and
- * {@link FlinkKafkaProducer} will keep using the {@link FlinkKafkaInternalProducer} from
- * the previous checkpoint. To decrease the chance of failing checkpoints there are four
- * options:
- * <li>decrease number of max concurrent checkpoints
- * <li>make checkpoints more reliable (so that they complete faster)
- * <li>increase the delay between checkpoints
- * <li>increase the size of {@link FlinkKafkaInternalProducer}s pool
- */
- EXACTLY_ONCE,
-
- /**
- * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the
- * Kafka buffers to be acknowledged by the Kafka producer on a checkpoint.
- */
- AT_LEAST_ONCE,
-
- /**
- * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or
- * duplicated in case of failure.
- */
- NONE
- }
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
-
- private static final long serialVersionUID = 1L;
-
- /**
- * Number of characters to truncate the taskName to for the Kafka transactionalId. The maximum
- * this can possibly be set to is 32,767 - (length of operatorUniqueId).
- */
- private static final short maxTaskNameSize = 1_000;
-
- /**
- * This coefficient determines what is the safe scale down factor.
- *
- * <p>If the Flink application previously failed before first checkpoint completed or we are
- * starting new batch of {@link FlinkKafkaProducer} from scratch without clean shutdown of the
- * previous one, {@link FlinkKafkaProducer} doesn't know what was the set of previously used
- * Kafka's transactionalId's. In that case, it will try to play safe and abort all of the
- * possible transactionalIds from the range of: {@code [0, getNumberOfParallelSubtasks() *
- * kafkaProducersPoolSize * SAFE_SCALE_DOWN_FACTOR) }
- *
- * <p>The range of available to use transactional ids is: {@code [0,
- * getNumberOfParallelSubtasks() * kafkaProducersPoolSize) }
- *
- * <p>This means that if we decrease {@code getNumberOfParallelSubtasks()} by a factor larger
- * than {@code SAFE_SCALE_DOWN_FACTOR} we can have a left some lingering transaction.
- */
- public static final int SAFE_SCALE_DOWN_FACTOR = 5;
-
- /**
- * Default number of KafkaProducers in the pool. See {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}.
- */
- public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
-
- /** Default value for kafka transaction timeout. */
- public static final Time DEFAULT_KAFKA_TRANSACTION_TIMEOUT = Time.hours(1);
-
- /** Configuration key for disabling the metrics reporting. */
- public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
-
- /**
- * Descriptor of the transactional IDs list. Note: This state is serialized by Kryo Serializer
- * and it has compatibility problem that will be removed later. Please use
- * NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2.
- */
- @Deprecated
- private static final ListStateDescriptor<FlinkKafkaProducer.NextTransactionalIdHint>
- NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR =
- new ListStateDescriptor<>(
- "next-transactional-id-hint",
- TypeInformation.of(NextTransactionalIdHint.class));
-
- private static final ListStateDescriptor<FlinkKafkaProducer.NextTransactionalIdHint>
- NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2 =
- new ListStateDescriptor<>(
- "next-transactional-id-hint-v2",
- new NextTransactionalIdHintSerializer());
-
- /** State for nextTransactionalIdHint. */
- private transient ListState<FlinkKafkaProducer.NextTransactionalIdHint>
- nextTransactionalIdHintState;
-
- /** Generator for Transactional IDs. */
- private transient TransactionalIdsGenerator transactionalIdsGenerator;
-
- /** Hint for picking next transactional id. */
- private transient FlinkKafkaProducer.NextTransactionalIdHint nextTransactionalIdHint;
-
- /** User defined properties for the Producer. */
- protected final Properties producerConfig;
-
- /** The name of the default topic this producer is writing data to. */
- protected final String defaultTopicId;
-
- /**
- * (Serializable) SerializationSchema for turning objects used with Flink into. byte[] for
- * Kafka.
- */
- @Nullable private final KeyedSerializationSchema<IN> keyedSchema;
-
- /**
- * (Serializable) serialization schema for serializing records to {@link ProducerRecord
- * ProducerRecords}.
- */
- @Nullable private final KafkaSerializationSchema<IN> kafkaSchema;
-
- /** User-provided partitioner for assigning an object to a Kafka partition for each topic. */
- @Nullable private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
-
- /** Partitions of each topic. */
- protected final Map<String, int[]> topicPartitionsMap;
-
- /**
- * Max number of producers in the pool. If all producers are in use, snapshoting state will
- * throw an exception.
- */
- private final int kafkaProducersPoolSize;
-
- /** Pool of available transactional ids. */
- private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
-
- /** Flag controlling whether we are writing the Flink record's timestamp into Kafka. */
- protected boolean writeTimestampToKafka = false;
-
- /** The transactional.id prefix to be used by the producers when communicating with Kafka. */
- @Nullable private String transactionalIdPrefix = null;
-
- /** Flag indicating whether to accept failures (and log them), or to fail on failures. */
- private boolean logFailuresOnly;
-
- /** Semantic chosen for this instance. */
- protected FlinkKafkaProducer.Semantic semantic;
-
- // -------------------------------- Runtime fields ------------------------------------------
-
- /** The callback than handles error propagation or logging callbacks. */
- @Nullable protected transient Callback callback;
-
- /** Errors encountered in the async producer are stored here. */
- @Nullable protected transient volatile Exception asyncException;
-
- /** Number of unacknowledged records. */
- protected final AtomicLong pendingRecords = new AtomicLong();
-
- /**
- * Cache of metrics to replace already registered metrics instead of overwriting existing ones.
- */
- private final Map<String, KafkaMetricMutableWrapper> previouslyCreatedMetrics = new HashMap<>();
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic.
- *
- * @param brokerList Comma separated addresses of the brokers
- * @param topicId ID of the Kafka topic.
- * @param serializationSchema User defined (keyless) serialization schema.
- */
- public FlinkKafkaProducer(
- String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
- this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList));
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic.
- *
- * <p>Using this constructor, the default {@link FlinkFixedPartitioner} will be used as the
- * partitioner. This default partitioner maps each sink subtask to a single Kafka partition
- * (i.e. all records received by a sink subtask will end up in the same Kafka partition).
- *
- * <p>To use a custom partitioner, please use {@link #FlinkKafkaProducer(String,
- * SerializationSchema, Properties, Optional)} instead.
- *
- * @param topicId ID of the Kafka topic.
- * @param serializationSchema User defined key-less serialization schema.
- * @param producerConfig Properties with the producer configuration.
- */
- public FlinkKafkaProducer(
- String topicId,
- SerializationSchema<IN> serializationSchema,
- Properties producerConfig) {
- this(
- topicId,
- serializationSchema,
- producerConfig,
- Optional.of(new FlinkFixedPartitioner<>()));
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a key-less {@link SerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * <p>Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not
- * have an attached key. Therefore, if a partitioner is also not provided, records will be
- * distributed to Kafka partitions in a round-robin fashion.
- *
- * @param topicId The topic to write data to
- * @param serializationSchema A key-less serializable serialization schema for turning user
- * objects into a kafka-consumable byte[]
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. If a partitioner is not provided, records will be distributed to Kafka
- * partitions in a round-robin fashion.
- */
- public FlinkKafkaProducer(
- String topicId,
- SerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- Optional<FlinkKafkaPartitioner<IN>> customPartitioner) {
- this(
- topicId,
- serializationSchema,
- producerConfig,
- customPartitioner.orElse(null),
- Semantic.AT_LEAST_ONCE,
- DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a key-less {@link SerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * <p>Since a key-less {@link SerializationSchema} is used, all records sent to Kafka will not
- * have an attached key. Therefore, if a partitioner is also not provided, records will be
- * distributed to Kafka partitions in a round-robin fashion.
- *
- * @param topicId The topic to write data to
- * @param serializationSchema A key-less serializable serialization schema for turning user
- * objects into a kafka-consumable byte[]
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. If a partitioner is not provided, records will be distributed to Kafka
- * partitions in a round-robin fashion.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
- */
- public FlinkKafkaProducer(
- String topicId,
- SerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- @Nullable FlinkKafkaPartitioner<IN> customPartitioner,
- FlinkKafkaProducer.Semantic semantic,
- int kafkaProducersPoolSize) {
- this(
- topicId,
- null,
- null,
- new KafkaSerializationSchemaWrapper<>(
- topicId, customPartitioner, false, serializationSchema),
- producerConfig,
- semantic,
- kafkaProducersPoolSize);
- }
-
- // ------------------- Key/Value serialization schema constructors ----------------------
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic.
- *
- * <p>Using this constructor, the default {@link FlinkFixedPartitioner} will be used as the
- * partitioner. This default partitioner maps each sink subtask to a single Kafka partition
- * (i.e. all records received by a sink subtask will end up in the same Kafka partition).
- *
- * <p>To use a custom partitioner, please use {@link #FlinkKafkaProducer(String,
- * KeyedSerializationSchema, Properties, Optional)} instead.
- *
- * @param brokerList Comma separated addresses of the brokers
- * @param topicId ID of the Kafka topic.
- * @param serializationSchema User defined serialization schema supporting key/value messages
- * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
- * FlinkKafkaProducer.Semantic)}
- */
- @Deprecated
- public FlinkKafkaProducer(
- String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
- this(
- topicId,
- serializationSchema,
- getPropertiesFromBrokerList(brokerList),
- Optional.of(new FlinkFixedPartitioner<IN>()));
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic.
- *
- * <p>Using this constructor, the default {@link FlinkFixedPartitioner} will be used as the
- * partitioner. This default partitioner maps each sink subtask to a single Kafka partition
- * (i.e. all records received by a sink subtask will end up in the same Kafka partition).
- *
- * <p>To use a custom partitioner, please use {@link #FlinkKafkaProducer(String,
- * KeyedSerializationSchema, Properties, Optional)} instead.
- *
- * @param topicId ID of the Kafka topic.
- * @param serializationSchema User defined serialization schema supporting key/value messages
- * @param producerConfig Properties with the producer configuration.
- * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
- * FlinkKafkaProducer.Semantic)}
- */
- @Deprecated
- public FlinkKafkaProducer(
- String topicId,
- KeyedSerializationSchema<IN> serializationSchema,
- Properties producerConfig) {
- this(
- topicId,
- serializationSchema,
- producerConfig,
- Optional.of(new FlinkFixedPartitioner<IN>()));
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic.
- *
- * <p>Using this constructor, the default {@link FlinkFixedPartitioner} will be used as the
- * partitioner. This default partitioner maps each sink subtask to a single Kafka partition
- * (i.e. all records received by a sink subtask will end up in the same Kafka partition).
- *
- * @param topicId ID of the Kafka topic.
- * @param serializationSchema User defined serialization schema supporting key/value messages
- * @param producerConfig Properties with the producer configuration.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
- * FlinkKafkaProducer.Semantic)}
- */
- @Deprecated
- public FlinkKafkaProducer(
- String topicId,
- KeyedSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- FlinkKafkaProducer.Semantic semantic) {
- this(
- topicId,
- serializationSchema,
- producerConfig,
- Optional.of(new FlinkFixedPartitioner<IN>()),
- semantic,
- DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * <p>If a partitioner is not provided, written records will be partitioned by the attached key
- * of each record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If
- * written records do not have a key (i.e., {@link
- * KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they will be
- * distributed to Kafka partitions in a round-robin fashion.
- *
- * @param defaultTopicId The default topic to write data to
- * @param serializationSchema A serializable serialization schema for turning user objects into
- * a kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. If a partitioner is not provided, records will be partitioned by the key of
- * each record (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the
- * keys are {@code null}, then records will be distributed to Kafka partitions in a
- * round-robin fashion.
- * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
- * FlinkKafkaProducer.Semantic)}
- */
- @Deprecated
- public FlinkKafkaProducer(
- String defaultTopicId,
- KeyedSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- Optional<FlinkKafkaPartitioner<IN>> customPartitioner) {
- this(
- defaultTopicId,
- serializationSchema,
- producerConfig,
- customPartitioner,
- FlinkKafkaProducer.Semantic.AT_LEAST_ONCE,
- DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a keyed {@link KeyedSerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * <p>If a partitioner is not provided, written records will be partitioned by the attached key
- * of each record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If
- * written records do not have a key (i.e., {@link
- * KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they will be
- * distributed to Kafka partitions in a round-robin fashion.
- *
- * @param defaultTopicId The default topic to write data to
- * @param serializationSchema A serializable serialization schema for turning user objects into
- * a kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. If a partitioner is not provided, records will be partitioned by the key of
- * each record (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the
- * keys are {@code null}, then records will be distributed to Kafka partitions in a
- * round-robin fashion.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
- * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
- * FlinkKafkaProducer.Semantic)}
- */
- @Deprecated
- public FlinkKafkaProducer(
- String defaultTopicId,
- KeyedSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- Optional<FlinkKafkaPartitioner<IN>> customPartitioner,
- FlinkKafkaProducer.Semantic semantic,
- int kafkaProducersPoolSize) {
- this(
- defaultTopicId,
- serializationSchema,
- customPartitioner.orElse(null),
- null, /* kafka serialization schema */
- producerConfig,
- semantic,
- kafkaProducersPoolSize);
- }
-
- /**
- * Creates a {@link FlinkKafkaProducer} for a given topic. The sink produces its input to the
- * topic. It accepts a {@link KafkaSerializationSchema} for serializing records to a {@link
- * ProducerRecord}, including partitioning information.
- *
- * @param defaultTopic The default topic to write data to
- * @param serializationSchema A serializable serialization schema for turning user objects into
- * a kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- */
- public FlinkKafkaProducer(
- String defaultTopic,
- KafkaSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- FlinkKafkaProducer.Semantic semantic) {
- this(
- defaultTopic,
- serializationSchema,
- producerConfig,
- semantic,
- DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a {@link KafkaSerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * @param defaultTopic The default topic to write data to
- * @param serializationSchema A serializable serialization schema for turning user objects into
- * a kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
- */
- public FlinkKafkaProducer(
- String defaultTopic,
- KafkaSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- FlinkKafkaProducer.Semantic semantic,
- int kafkaProducersPoolSize) {
- this(
- defaultTopic,
- null,
- null, /* keyed schema and FlinkKafkaPartitioner */
- serializationSchema,
- producerConfig,
- semantic,
- kafkaProducersPoolSize);
- }
-
- /**
- * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to the topic. It
- * accepts a {@link KafkaSerializationSchema} and possibly a custom {@link
- * FlinkKafkaPartitioner}.
- *
- * <p>If a partitioner is not provided, written records will be partitioned by the attached key
- * of each record (as determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If
- * written records do not have a key (i.e., {@link
- * KeyedSerializationSchema#serializeKey(Object)} returns {@code null}), they will be
- * distributed to Kafka partitions in a round-robin fashion.
- *
- * @param defaultTopic The default topic to write data to
- * @param keyedSchema A serializable serialization schema for turning user objects into a
- * kafka-consumable byte[] supporting key/value messages
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. If a partitioner is not provided, records will be partitioned by the key of
- * each record (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the
- * keys are {@code null}, then records will be distributed to Kafka partitions in a
- * round-robin fashion.
- * @param kafkaSchema A serializable serialization schema for turning user objects into a
- * kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param semantic Defines semantic that will be used by this producer (see {@link
- * FlinkKafkaProducer.Semantic}).
- * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link
- * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}).
- */
- private FlinkKafkaProducer(
- String defaultTopic,
- KeyedSerializationSchema<IN> keyedSchema,
- FlinkKafkaPartitioner<IN> customPartitioner,
- KafkaSerializationSchema<IN> kafkaSchema,
- Properties producerConfig,
- FlinkKafkaProducer.Semantic semantic,
- int kafkaProducersPoolSize) {
- super(
- new FlinkKafkaProducer.TransactionStateSerializer(),
- new FlinkKafkaProducer.ContextStateSerializer());
-
- this.defaultTopicId = checkNotNull(defaultTopic, "defaultTopic is null");
-
- if (kafkaSchema != null) {
- this.keyedSchema = null;
- this.kafkaSchema = kafkaSchema;
- this.flinkKafkaPartitioner = null;
- ClosureCleaner.clean(
- this.kafkaSchema, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
-
- if (customPartitioner != null) {
- throw new IllegalArgumentException(
- "Customer partitioner can only be used when"
- + "using a KeyedSerializationSchema or SerializationSchema.");
- }
- } else if (keyedSchema != null) {
- this.kafkaSchema = null;
- this.keyedSchema = keyedSchema;
- this.flinkKafkaPartitioner = customPartitioner;
- ClosureCleaner.clean(
- this.flinkKafkaPartitioner,
- ExecutionConfig.ClosureCleanerLevel.RECURSIVE,
- true);
- ClosureCleaner.clean(
- this.keyedSchema, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- } else {
- throw new IllegalArgumentException(
- "You must provide either a KafkaSerializationSchema or a"
- + "KeyedSerializationSchema.");
- }
-
- this.producerConfig = checkNotNull(producerConfig, "producerConfig is null");
- this.semantic = checkNotNull(semantic, "semantic is null");
- this.kafkaProducersPoolSize = kafkaProducersPoolSize;
- checkState(kafkaProducersPoolSize > 0, "kafkaProducersPoolSize must be non empty");
-
- // set the producer configuration properties for kafka record key value serializers.
- if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
- this.producerConfig.put(
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
- ByteArraySerializer.class.getName());
- } else {
- LOG.warn(
- "Overwriting the '{}' is not recommended",
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
- }
-
- if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
- this.producerConfig.put(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
- ByteArraySerializer.class.getName());
- } else {
- LOG.warn(
- "Overwriting the '{}' is not recommended",
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
- }
-
- // eagerly ensure that bootstrap servers are set.
- if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
- throw new IllegalArgumentException(
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG
- + " must be supplied in the producer config properties.");
- }
-
- if (!producerConfig.containsKey(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG)) {
- long timeout = DEFAULT_KAFKA_TRANSACTION_TIMEOUT.toMilliseconds();
- checkState(
- timeout < Integer.MAX_VALUE && timeout > 0,
- "timeout does not fit into 32 bit integer");
- this.producerConfig.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, (int) timeout);
- LOG.warn(
- "Property [{}] not specified. Setting it to {}",
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
- DEFAULT_KAFKA_TRANSACTION_TIMEOUT);
- }
-
- // Enable transactionTimeoutWarnings to avoid silent data loss
- // See KAFKA-6119 (affects versions 0.11.0.0 and 0.11.0.1):
- // The KafkaProducer may not throw an exception if the transaction failed to commit
- if (semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) {
- final long transactionTimeout = getTransactionTimeout(producerConfig);
- super.setTransactionTimeout(transactionTimeout);
- super.enableTransactionTimeoutWarnings(0.8);
- }
-
- this.topicPartitionsMap = new HashMap<>();
- }
-
- // ---------------------------------- Properties --------------------------
-
- /**
- * If set to true, Flink will write the (event time) timestamp attached to each record into
- * Kafka. Timestamps must be positive for Kafka to accept them.
- *
- * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to
- * Kafka.
- */
- public void setWriteTimestampToKafka(boolean writeTimestampToKafka) {
- this.writeTimestampToKafka = writeTimestampToKafka;
- if (kafkaSchema instanceof KafkaSerializationSchemaWrapper) {
- ((KafkaSerializationSchemaWrapper<IN>) kafkaSchema)
- .setWriteTimestamp(writeTimestampToKafka);
- }
- }
-
- /**
- * Defines whether the producer should fail on errors, or only log them. If this is set to true,
- * then exceptions will be only logged, if set to false, exceptions will be eventually thrown
- * and cause the streaming program to fail (and enter recovery).
- *
- * @param logFailuresOnly The flag to indicate logging-only on exceptions.
- */
- public void setLogFailuresOnly(boolean logFailuresOnly) {
- this.logFailuresOnly = logFailuresOnly;
- }
-
- /**
- * Specifies the prefix of the transactional.id property to be used by the producers when
- * communicating with Kafka. If not set, the transactional.id will be prefixed with {@code
- * taskName + "-" + operatorUid}.
- *
- * <p>Note that, if we change the prefix when the Flink application previously failed before
- * first checkpoint completed or we are starting new batch of {@link FlinkKafkaProducer} from
- * scratch without clean shutdown of the previous one, since we don't know what was the
- * previously used transactional.id prefix, there will be some lingering transactions left.
- *
- * @param transactionalIdPrefix the transactional.id prefix
- * @throws NullPointerException Thrown, if the transactionalIdPrefix was null.
- */
- public void setTransactionalIdPrefix(String transactionalIdPrefix) {
- this.transactionalIdPrefix = Preconditions.checkNotNull(transactionalIdPrefix);
- }
-
- /**
- * Disables the propagation of exceptions thrown when committing presumably timed out Kafka
- * transactions during recovery of the job. If a Kafka transaction is timed out, a commit will
- * never be successful. Hence, use this feature to avoid recovery loops of the Job. Exceptions
- * will still be logged to inform the user that data loss might have occurred.
- *
- * <p>Note that we use {@link System#currentTimeMillis()} to track the age of a transaction.
- * Moreover, only exceptions thrown during the recovery are caught, i.e., the producer will
- * attempt at least one commit of the transaction before giving up.
- */
- @Override
- public FlinkKafkaProducer<IN> ignoreFailuresAfterTransactionTimeout() {
- super.ignoreFailuresAfterTransactionTimeout();
- return this;
- }
-
- // ----------------------------------- Utilities --------------------------
-
- /** Initializes the connection to Kafka. */
- @Override
- public void open(Configuration configuration) throws Exception {
- if (logFailuresOnly) {
- callback =
- new Callback() {
- @Override
- public void onCompletion(RecordMetadata metadata, Exception e) {
- if (e != null) {
- LOG.error(
- "Error while sending record to Kafka: " + e.getMessage(),
- e);
- }
- acknowledgeMessage();
- }
- };
- } else {
- callback =
- new Callback() {
- @Override
- public void onCompletion(RecordMetadata metadata, Exception exception) {
- if (exception != null && asyncException == null) {
- asyncException = exception;
- }
- acknowledgeMessage();
- }
- };
- }
-
- RuntimeContext ctx = getRuntimeContext();
-
- if (flinkKafkaPartitioner != null) {
- flinkKafkaPartitioner.open(
- ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks());
- }
-
- if (kafkaSchema instanceof KafkaContextAware) {
- KafkaContextAware<IN> contextAwareSchema = (KafkaContextAware<IN>) kafkaSchema;
- contextAwareSchema.setParallelInstanceId(ctx.getIndexOfThisSubtask());
- contextAwareSchema.setNumParallelInstances(ctx.getNumberOfParallelSubtasks());
- }
-
- if (kafkaSchema != null) {
- kafkaSchema.open(
- RuntimeContextInitializationContextAdapters.serializationAdapter(
- getRuntimeContext(), metricGroup -> metricGroup.addGroup("user")));
- }
-
- super.open(configuration);
- }
-
- @Override
- public void invoke(
- FlinkKafkaProducer.KafkaTransactionState transaction, IN next, Context context)
- throws FlinkKafkaException {
- checkErroneous();
-
- ProducerRecord<byte[], byte[]> record;
- if (keyedSchema != null) {
- byte[] serializedKey = keyedSchema.serializeKey(next);
- byte[] serializedValue = keyedSchema.serializeValue(next);
- String targetTopic = keyedSchema.getTargetTopic(next);
- if (targetTopic == null) {
- targetTopic = defaultTopicId;
- }
-
- Long timestamp = null;
- if (this.writeTimestampToKafka) {
- timestamp = context.timestamp();
- }
-
- int[] partitions = topicPartitionsMap.get(targetTopic);
- if (null == partitions) {
- partitions = getPartitionsByTopic(targetTopic, transaction.producer);
- topicPartitionsMap.put(targetTopic, partitions);
- }
- if (flinkKafkaPartitioner != null) {
- record =
- new ProducerRecord<>(
- targetTopic,
- flinkKafkaPartitioner.partition(
- next,
- serializedKey,
- serializedValue,
- targetTopic,
- partitions),
- timestamp,
- serializedKey,
- serializedValue);
- } else {
- record =
- new ProducerRecord<>(
- targetTopic, null, timestamp, serializedKey, serializedValue);
- }
- } else if (kafkaSchema != null) {
- if (kafkaSchema instanceof KafkaContextAware) {
- @SuppressWarnings("unchecked")
- KafkaContextAware<IN> contextAwareSchema = (KafkaContextAware<IN>) kafkaSchema;
-
- String targetTopic = contextAwareSchema.getTargetTopic(next);
- if (targetTopic == null) {
- targetTopic = defaultTopicId;
- }
- int[] partitions = topicPartitionsMap.get(targetTopic);
-
- if (null == partitions) {
- partitions = getPartitionsByTopic(targetTopic, transaction.producer);
- topicPartitionsMap.put(targetTopic, partitions);
- }
-
- contextAwareSchema.setPartitions(partitions);
- }
- record = kafkaSchema.serialize(next, context.timestamp());
- } else {
- throw new RuntimeException(
- "We have neither KafkaSerializationSchema nor KeyedSerializationSchema, this"
- + "is a bug.");
- }
-
- pendingRecords.incrementAndGet();
- transaction.producer.send(record, callback);
- }
-
- @Override
- public void close() throws FlinkKafkaException {
- // First close the producer for current transaction.
- try {
- final KafkaTransactionState currentTransaction = currentTransaction();
- if (currentTransaction != null) {
- // to avoid exceptions on aborting transactions with some pending records
- flush(currentTransaction);
-
- // normal abort for AT_LEAST_ONCE and NONE do not clean up resources because of
- // producer reusing, thus
- // we need to close it manually
- switch (semantic) {
- case EXACTLY_ONCE:
- break;
- case AT_LEAST_ONCE:
- case NONE:
- currentTransaction.producer.flush();
- currentTransaction.producer.close(Duration.ofSeconds(0));
- break;
- }
- }
- super.close();
- } catch (Exception e) {
- asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
- } finally {
- // We may have to close producer of the current transaction in case some exception was
- // thrown before
- // the normal close routine finishes.
- if (currentTransaction() != null) {
- try {
- currentTransaction().producer.close(Duration.ofSeconds(0));
- } catch (Throwable t) {
- LOG.warn("Error closing producer.", t);
- }
- }
- // Make sure all the producers for pending transactions are closed.
- pendingTransactions()
- .forEach(
- transaction -> {
- try {
- transaction.getValue().producer.close(Duration.ofSeconds(0));
- } catch (Throwable t) {
- LOG.warn("Error closing producer.", t);
- }
- });
- // make sure we propagate pending errors
- checkErroneous();
- }
- }
-
- // ------------------- Logic for handling checkpoint flushing -------------------------- //
-
- @Override
- protected FlinkKafkaProducer.KafkaTransactionState beginTransaction()
- throws FlinkKafkaException {
- switch (semantic) {
- case EXACTLY_ONCE:
- FlinkKafkaInternalProducer<byte[], byte[]> producer = createTransactionalProducer();
- producer.beginTransaction();
- return new FlinkKafkaProducer.KafkaTransactionState(
- producer.getTransactionalId(), producer);
- case AT_LEAST_ONCE:
- case NONE:
- // Do not create new producer on each beginTransaction() if it is not necessary
- final FlinkKafkaProducer.KafkaTransactionState currentTransaction =
- currentTransaction();
- if (currentTransaction != null && currentTransaction.producer != null) {
- return new FlinkKafkaProducer.KafkaTransactionState(
- currentTransaction.producer);
- }
- return new FlinkKafkaProducer.KafkaTransactionState(
- initNonTransactionalProducer(true));
- default:
- throw new UnsupportedOperationException("Not implemented semantic");
- }
- }
-
- @Override
- protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction)
- throws FlinkKafkaException {
- switch (semantic) {
- case EXACTLY_ONCE:
- case AT_LEAST_ONCE:
- flush(transaction);
- break;
- case NONE:
- break;
- default:
- throw new UnsupportedOperationException("Not implemented semantic");
- }
- checkErroneous();
- }
-
- @Override
- protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) {
- if (transaction.isTransactional()) {
- try {
- transaction.producer.commitTransaction();
- } finally {
- recycleTransactionalProducer(transaction.producer);
- }
- }
- }
-
- @Override
- protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction) {
- if (transaction.isTransactional()) {
- FlinkKafkaInternalProducer<byte[], byte[]> producer = null;
- try {
- producer = initTransactionalProducer(transaction.transactionalId, false);
- producer.resumeTransaction(transaction.producerId, transaction.epoch);
- producer.commitTransaction();
- } catch (InvalidTxnStateException e) {
- LOG.warn(
- "Unable to commit recovered transaction ({}) because it's in an invalid state. "
- + "Most likely the transaction has been aborted for some reason. Please check the Kafka logs for more details.",
- transaction,
- e);
- } catch (ProducerFencedException e) {
- LOG.warn(
- "Unable to commit recovered transaction ({}) because its producer is already fenced."
- + " This means that you either have a different producer with the same '{}' or"
- + " recovery took longer than '{}' ({}ms). In both cases this most likely signals data loss,"
- + " please consult the Flink documentation for more details.",
- transaction,
- ProducerConfig.TRANSACTIONAL_ID_CONFIG,
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
- getTransactionTimeout(producerConfig),
- e);
- } finally {
- if (producer != null) {
- producer.close(Duration.ofSeconds(0));
- }
- }
- }
- }
-
- @Override
- protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) {
- if (transaction.isTransactional()) {
- transaction.producer.abortTransaction();
- recycleTransactionalProducer(transaction.producer);
- }
- }
-
- @Override
- protected void recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction) {
- if (transaction.isTransactional()) {
- FlinkKafkaInternalProducer<byte[], byte[]> producer = null;
- try {
- producer = initTransactionalProducer(transaction.transactionalId, false);
- producer.initTransactions();
- } finally {
- if (producer != null) {
- producer.close(Duration.ofSeconds(0));
- }
- }
- }
- }
-
- /**
- * <b>ATTENTION to subclass implementors:</b> When overriding this method, please always call
- * {@code super.acknowledgeMessage()} to keep the invariants of the internal bookkeeping of the
- * producer. If not, be sure to know what you are doing.
- */
- protected void acknowledgeMessage() {
- pendingRecords.decrementAndGet();
- }
-
- /**
- * Flush pending records.
- *
- * @param transaction
- */
- private void flush(FlinkKafkaProducer.KafkaTransactionState transaction)
- throws FlinkKafkaException {
- if (transaction.producer != null) {
- transaction.producer.flush();
- }
- long pendingRecordsCount = pendingRecords.get();
- if (pendingRecordsCount != 0) {
- throw new IllegalStateException(
- "Pending record count must be zero at this point: " + pendingRecordsCount);
- }
-
- // if the flushed requests has errors, we should propagate it also and fail the checkpoint
- checkErroneous();
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext context) throws Exception {
- super.snapshotState(context);
-
- nextTransactionalIdHintState.clear();
- // To avoid duplication only first subtask keeps track of next transactional id hint.
- // Otherwise all of the
- // subtasks would write exactly same information.
- if (getRuntimeContext().getIndexOfThisSubtask() == 0
- && semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) {
- checkState(
- nextTransactionalIdHint != null,
- "nextTransactionalIdHint must be set for EXACTLY_ONCE");
- long nextFreeTransactionalId = nextTransactionalIdHint.nextFreeTransactionalId;
-
- // If we scaled up, some (unknown) subtask must have created new transactional ids from
- // scratch. In that
- // case we adjust nextFreeTransactionalId by the range of transactionalIds that could be
- // used for this
- // scaling up.
- if (getRuntimeContext().getNumberOfParallelSubtasks()
- > nextTransactionalIdHint.lastParallelism) {
- nextFreeTransactionalId +=
- getRuntimeContext().getNumberOfParallelSubtasks() * kafkaProducersPoolSize;
- }
-
- nextTransactionalIdHintState.add(
- new FlinkKafkaProducer.NextTransactionalIdHint(
- getRuntimeContext().getNumberOfParallelSubtasks(),
- nextFreeTransactionalId));
- }
- }
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {
- if (semantic != FlinkKafkaProducer.Semantic.NONE
- && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
- LOG.warn(
- "Using {} semantic, but checkpointing is not enabled. Switching to {} semantic.",
- semantic,
- FlinkKafkaProducer.Semantic.NONE);
- semantic = FlinkKafkaProducer.Semantic.NONE;
- }
-
- nextTransactionalIdHintState =
- context.getOperatorStateStore()
- .getUnionListState(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2);
-
- if (context.getOperatorStateStore()
- .getRegisteredStateNames()
- .contains(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR)) {
- migrateNextTransactionalIdHindState(context);
- }
-
- String actualTransactionalIdPrefix;
- if (this.transactionalIdPrefix != null) {
- actualTransactionalIdPrefix = this.transactionalIdPrefix;
- } else {
- String taskName = getRuntimeContext().getTaskName();
- // Kafka transactional IDs are limited in length to be less than the max value of
- // a short, so we truncate here if necessary to a more reasonable length string.
- if (taskName.length() > maxTaskNameSize) {
- taskName = taskName.substring(0, maxTaskNameSize);
- LOG.warn(
- "Truncated task name for Kafka TransactionalId from {} to {}.",
- getRuntimeContext().getTaskName(),
- taskName);
- }
- actualTransactionalIdPrefix =
- taskName
- + "-"
- + ((StreamingRuntimeContext) getRuntimeContext()).getOperatorUniqueID();
- }
- transactionalIdsGenerator =
- new TransactionalIdsGenerator(
- actualTransactionalIdPrefix,
- getRuntimeContext().getIndexOfThisSubtask(),
- getRuntimeContext().getNumberOfParallelSubtasks(),
- kafkaProducersPoolSize,
- SAFE_SCALE_DOWN_FACTOR);
-
- if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) {
- nextTransactionalIdHint = null;
- } else {
- ArrayList<FlinkKafkaProducer.NextTransactionalIdHint> transactionalIdHints =
- Lists.newArrayList(nextTransactionalIdHintState.get());
- if (transactionalIdHints.size() > 1) {
- throw new IllegalStateException(
- "There should be at most one next transactional id hint written by the first subtask");
- } else if (transactionalIdHints.size() == 0) {
- nextTransactionalIdHint = new FlinkKafkaProducer.NextTransactionalIdHint(0, 0);
-
- // this means that this is either:
- // (1) the first execution of this application
- // (2) previous execution has failed before first checkpoint completed
- //
- // in case of (2) we have to abort all previous transactions
- abortTransactions(transactionalIdsGenerator.generateIdsToAbort());
- } else {
- nextTransactionalIdHint = transactionalIdHints.get(0);
- }
- }
-
- super.initializeState(context);
- }
-
- @Override
- protected Optional<FlinkKafkaProducer.KafkaTransactionContext> initializeUserContext() {
- if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) {
- return Optional.empty();
- }
-
- Set<String> transactionalIds = generateNewTransactionalIds();
- resetAvailableTransactionalIdsPool(transactionalIds);
- return Optional.of(new FlinkKafkaProducer.KafkaTransactionContext(transactionalIds));
- }
-
- private Set<String> generateNewTransactionalIds() {
- checkState(
- nextTransactionalIdHint != null,
- "nextTransactionalIdHint must be present for EXACTLY_ONCE");
-
- Set<String> transactionalIds =
- transactionalIdsGenerator.generateIdsToUse(
- nextTransactionalIdHint.nextFreeTransactionalId);
- LOG.info("Generated new transactionalIds {}", transactionalIds);
- return transactionalIds;
- }
-
- @Override
- protected void finishRecoveringContext(
- Collection<FlinkKafkaProducer.KafkaTransactionState> handledTransactions) {
- cleanUpUserContext(handledTransactions);
- resetAvailableTransactionalIdsPool(getUserContext().get().transactionalIds);
- LOG.info("Recovered transactionalIds {}", getUserContext().get().transactionalIds);
- }
-
- protected FlinkKafkaInternalProducer<byte[], byte[]> createProducer() {
- return new FlinkKafkaInternalProducer<>(this.producerConfig);
- }
-
- /**
- * After initialization make sure that all previous transactions from the current user context
- * have been completed.
- *
- * @param handledTransactions transactions which were already committed or aborted and do not
- * need further handling
- */
- private void cleanUpUserContext(
- Collection<FlinkKafkaProducer.KafkaTransactionState> handledTransactions) {
- if (!getUserContext().isPresent()) {
- return;
- }
- HashSet<String> abortTransactions = new HashSet<>(getUserContext().get().transactionalIds);
- handledTransactions.forEach(
- kafkaTransactionState ->
- abortTransactions.remove(kafkaTransactionState.transactionalId));
- abortTransactions(abortTransactions);
- }
-
- private void resetAvailableTransactionalIdsPool(Collection<String> transactionalIds) {
- availableTransactionalIds.clear();
- availableTransactionalIds.addAll(transactionalIds);
- }
-
- // ----------------------------------- Utilities --------------------------
-
- private void abortTransactions(final Set<String> transactionalIds) {
- final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
- transactionalIds
- .parallelStream()
- .forEach(
- transactionalId -> {
- // The parallelStream executes the consumer in a separated thread pool.
- // Because the consumer(e.g. Kafka) uses the context classloader to
- // construct some class
- // we should set the correct classloader for it.
- try (TemporaryClassLoaderContext ignored =
- TemporaryClassLoaderContext.of(classLoader)) {
- // don't mess with the original configuration or any other
- // properties of the
- // original object
- // -> create an internal kafka producer on our own and do not rely
- // on
- // initTransactionalProducer().
- final Properties myConfig = new Properties();
- myConfig.putAll(producerConfig);
- initTransactionalProducerConfig(myConfig, transactionalId);
- FlinkKafkaInternalProducer<byte[], byte[]> kafkaProducer = null;
- try {
- kafkaProducer = new FlinkKafkaInternalProducer<>(myConfig);
- // it suffices to call initTransactions - this will abort any
- // lingering transactions
- kafkaProducer.initTransactions();
- } finally {
- if (kafkaProducer != null) {
- kafkaProducer.close(Duration.ofSeconds(0));
- }
- }
- }
- });
- }
-
- int getTransactionCoordinatorId() {
- final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction();
- if (currentTransaction == null || currentTransaction.producer == null) {
- throw new IllegalArgumentException();
- }
- return currentTransaction.producer.getTransactionCoordinatorId();
- }
-
- @VisibleForTesting
- String getTransactionalId() {
- final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction();
- if (currentTransaction == null || currentTransaction.producer == null) {
- throw new IllegalArgumentException();
- }
- return currentTransaction.producer.getTransactionalId();
- }
-
- /**
- * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions
- * will not clash with transactions created during previous checkpoints ({@code
- * producer.initTransactions()} assures that we obtain new producerId and epoch counters).
- */
- private FlinkKafkaInternalProducer<byte[], byte[]> createTransactionalProducer()
- throws FlinkKafkaException {
- String transactionalId = availableTransactionalIds.poll();
- if (transactionalId == null) {
- throw new FlinkKafkaException(
- FlinkKafkaErrorCode.PRODUCERS_POOL_EMPTY,
- "Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checkpoints.");
- }
- FlinkKafkaInternalProducer<byte[], byte[]> producer =
- initTransactionalProducer(transactionalId, true);
- producer.initTransactions();
- return producer;
- }
-
- private void recycleTransactionalProducer(FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- availableTransactionalIds.add(producer.getTransactionalId());
- producer.flush();
- producer.close(Duration.ofSeconds(0));
- }
-
- private FlinkKafkaInternalProducer<byte[], byte[]> initTransactionalProducer(
- String transactionalId, boolean registerMetrics) {
- initTransactionalProducerConfig(producerConfig, transactionalId);
- return initProducer(registerMetrics);
- }
-
- private static void initTransactionalProducerConfig(
- Properties producerConfig, String transactionalId) {
- producerConfig.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
- }
-
- private FlinkKafkaInternalProducer<byte[], byte[]> initNonTransactionalProducer(
- boolean registerMetrics) {
- producerConfig.remove(ProducerConfig.TRANSACTIONAL_ID_CONFIG);
- return initProducer(registerMetrics);
- }
-
- private FlinkKafkaInternalProducer<byte[], byte[]> initProducer(boolean registerMetrics) {
- FlinkKafkaInternalProducer<byte[], byte[]> producer = createProducer();
-
- LOG.info(
- "Starting FlinkKafkaInternalProducer ({}/{}) to produce into default topic {}",
- getRuntimeContext().getIndexOfThisSubtask() + 1,
- getRuntimeContext().getNumberOfParallelSubtasks(),
- defaultTopicId);
-
- // register Kafka metrics to Flink accumulators
- if (registerMetrics
- && !Boolean.parseBoolean(
- producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
- Map<MetricName, ? extends Metric> metrics = producer.metrics();
-
- if (metrics == null) {
- // MapR's Kafka implementation returns null here.
- LOG.info("Producer implementation does not support metrics");
- } else {
- final MetricGroup kafkaMetricGroup =
- getRuntimeContext().getMetricGroup().addGroup("KafkaProducer");
- for (Map.Entry<MetricName, ? extends Metric> entry : metrics.entrySet()) {
- String name = entry.getKey().name();
- Metric metric = entry.getValue();
-
- KafkaMetricMutableWrapper wrapper = previouslyCreatedMetrics.get(name);
- if (wrapper != null) {
- wrapper.setKafkaMetric(metric);
- } else {
- // TODO: somehow merge metrics from all active producers?
- wrapper = new KafkaMetricMutableWrapper(metric);
- previouslyCreatedMetrics.put(name, wrapper);
- kafkaMetricGroup.gauge(name, wrapper);
- }
- }
- }
- }
- return producer;
- }
-
- protected void checkErroneous() throws FlinkKafkaException {
- Exception e = asyncException;
- if (e != null) {
- // prevent double throwing
- asyncException = null;
- throw new FlinkKafkaException(
- FlinkKafkaErrorCode.EXTERNAL_ERROR,
- "Failed to send data to Kafka: " + e.getMessage(),
- e);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- }
-
- private void migrateNextTransactionalIdHindState(FunctionInitializationContext context)
- throws Exception {
- ListState<NextTransactionalIdHint> oldNextTransactionalIdHintState =
- context.getOperatorStateStore()
- .getUnionListState(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR);
- nextTransactionalIdHintState =
- context.getOperatorStateStore()
- .getUnionListState(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2);
-
- ArrayList<NextTransactionalIdHint> oldTransactionalIdHints =
- Lists.newArrayList(oldNextTransactionalIdHintState.get());
- if (!oldTransactionalIdHints.isEmpty()) {
- nextTransactionalIdHintState.addAll(oldTransactionalIdHints);
- // clear old state
- oldNextTransactionalIdHintState.clear();
- }
- }
-
- private static Properties getPropertiesFromBrokerList(String brokerList) {
- String[] elements = brokerList.split(",");
-
- // validate the broker addresses
- for (String broker : elements) {
- NetUtils.getCorrectHostnamePort(broker);
- }
-
- Properties props = new Properties();
- props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
- return props;
- }
-
- protected static int[] getPartitionsByTopic(String topic, Producer<byte[], byte[]> producer) {
- // the fetched list is immutable, so we're creating a mutable copy in order to sort it
- List<PartitionInfo> partitionsList = new ArrayList<>(producer.partitionsFor(topic));
-
- // sort the partitions by partition id to make sure the fetched partition list is the same
- // across subtasks
- Collections.sort(
- partitionsList,
- new Comparator<PartitionInfo>() {
- @Override
- public int compare(PartitionInfo o1, PartitionInfo o2) {
- return Integer.compare(o1.partition(), o2.partition());
- }
- });
-
- int[] partitions = new int[partitionsList.size()];
- for (int i = 0; i < partitions.length; i++) {
- partitions[i] = partitionsList.get(i).partition();
- }
-
- return partitions;
- }
-
- public static long getTransactionTimeout(Properties producerConfig) {
- final Object object = producerConfig.get(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG);
- if (object instanceof String && StringUtils.isNumeric((String) object)) {
- return Long.parseLong((String) object);
- } else if (object instanceof Number) {
- return ((Number) object).longValue();
- } else {
- throw new IllegalArgumentException(
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG + " must be numeric, was " + object);
- }
- }
-
- /** State for handling transactions. */
- @VisibleForTesting
- @Internal
- public static class KafkaTransactionState {
-
- private final transient FlinkKafkaInternalProducer<byte[], byte[]> producer;
-
- @Nullable final String transactionalId;
-
- final long producerId;
-
- final short epoch;
-
- @VisibleForTesting
- public KafkaTransactionState(
- String transactionalId, FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- this(transactionalId, producer.getProducerId(), producer.getEpoch(), producer);
- }
-
- @VisibleForTesting
- public KafkaTransactionState(FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- this(null, -1, (short) -1, producer);
- }
-
- @VisibleForTesting
- public KafkaTransactionState(
- @Nullable String transactionalId,
- long producerId,
- short epoch,
- FlinkKafkaInternalProducer<byte[], byte[]> producer) {
- this.transactionalId = transactionalId;
- this.producerId = producerId;
- this.epoch = epoch;
- this.producer = producer;
- }
-
- boolean isTransactional() {
- return transactionalId != null;
- }
-
- public FlinkKafkaInternalProducer<byte[], byte[]> getProducer() {
- return producer;
- }
-
- @Override
- public String toString() {
- return String.format(
- "%s [transactionalId=%s, producerId=%s, epoch=%s]",
- this.getClass().getSimpleName(), transactionalId, producerId, epoch);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- FlinkKafkaProducer.KafkaTransactionState that =
- (FlinkKafkaProducer.KafkaTransactionState) o;
-
- if (producerId != that.producerId) {
- return false;
- }
- if (epoch != that.epoch) {
- return false;
- }
- return transactionalId != null
- ? transactionalId.equals(that.transactionalId)
- : that.transactionalId == null;
- }
-
- @Override
- public int hashCode() {
- int result = transactionalId != null ? transactionalId.hashCode() : 0;
- result = 31 * result + (int) (producerId ^ (producerId >>> 32));
- result = 31 * result + (int) epoch;
- return result;
- }
- }
-
- /**
- * Context associated to this instance of the {@link FlinkKafkaProducer}. User for keeping track
- * of the transactionalIds.
- */
- @VisibleForTesting
- @Internal
- public static class KafkaTransactionContext {
- final Set<String> transactionalIds;
-
- @VisibleForTesting
- public KafkaTransactionContext(Set<String> transactionalIds) {
- checkNotNull(transactionalIds);
- this.transactionalIds = transactionalIds;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- FlinkKafkaProducer.KafkaTransactionContext that =
- (FlinkKafkaProducer.KafkaTransactionContext) o;
-
- return transactionalIds.equals(that.transactionalIds);
- }
-
- @Override
- public int hashCode() {
- return transactionalIds.hashCode();
- }
- }
-
- /**
- * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link
- * FlinkKafkaProducer.KafkaTransactionState}.
- */
- @VisibleForTesting
- @Internal
- public static class TransactionStateSerializer
- extends TypeSerializerSingleton<FlinkKafkaProducer.KafkaTransactionState> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public boolean isImmutableType() {
- return true;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState createInstance() {
- return null;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState copy(
- FlinkKafkaProducer.KafkaTransactionState from) {
- return from;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState copy(
- FlinkKafkaProducer.KafkaTransactionState from,
- FlinkKafkaProducer.KafkaTransactionState reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return -1;
- }
-
- @Override
- public void serialize(
- FlinkKafkaProducer.KafkaTransactionState record, DataOutputView target)
- throws IOException {
- if (record.transactionalId == null) {
- target.writeBoolean(false);
- } else {
- target.writeBoolean(true);
- target.writeUTF(record.transactionalId);
- }
- target.writeLong(record.producerId);
- target.writeShort(record.epoch);
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState deserialize(DataInputView source)
- throws IOException {
- String transactionalId = null;
- if (source.readBoolean()) {
- transactionalId = source.readUTF();
- }
- long producerId = source.readLong();
- short epoch = source.readShort();
- return new FlinkKafkaProducer.KafkaTransactionState(
- transactionalId, producerId, epoch, null);
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState deserialize(
- FlinkKafkaProducer.KafkaTransactionState reuse, DataInputView source)
- throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- boolean hasTransactionalId = source.readBoolean();
- target.writeBoolean(hasTransactionalId);
- if (hasTransactionalId) {
- target.writeUTF(source.readUTF());
- }
- target.writeLong(source.readLong());
- target.writeShort(source.readShort());
- }
-
- // -----------------------------------------------------------------------------------
-
- @Override
- public TypeSerializerSnapshot<FlinkKafkaProducer.KafkaTransactionState>
- snapshotConfiguration() {
- return new TransactionStateSerializerSnapshot();
- }
-
- /** Serializer configuration snapshot for compatibility and format evolution. */
- @SuppressWarnings("WeakerAccess")
- public static final class TransactionStateSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<FlinkKafkaProducer.KafkaTransactionState> {
-
- public TransactionStateSerializerSnapshot() {
- super(TransactionStateSerializer::new);
- }
- }
- }
-
- /**
- * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link
- * FlinkKafkaProducer.KafkaTransactionContext}.
- */
- @VisibleForTesting
- @Internal
- public static class ContextStateSerializer
- extends TypeSerializerSingleton<FlinkKafkaProducer.KafkaTransactionContext> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public boolean isImmutableType() {
- return true;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext createInstance() {
- return null;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext copy(
- FlinkKafkaProducer.KafkaTransactionContext from) {
- return from;
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext copy(
- FlinkKafkaProducer.KafkaTransactionContext from,
- FlinkKafkaProducer.KafkaTransactionContext reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return -1;
- }
-
- @Override
- public void serialize(
- FlinkKafkaProducer.KafkaTransactionContext record, DataOutputView target)
- throws IOException {
- int numIds = record.transactionalIds.size();
- target.writeInt(numIds);
- for (String id : record.transactionalIds) {
- target.writeUTF(id);
- }
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext deserialize(DataInputView source)
- throws IOException {
- int numIds = source.readInt();
- Set<String> ids = new HashSet<>(numIds);
- for (int i = 0; i < numIds; i++) {
- ids.add(source.readUTF());
- }
- return new FlinkKafkaProducer.KafkaTransactionContext(ids);
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext deserialize(
- FlinkKafkaProducer.KafkaTransactionContext reuse, DataInputView source)
- throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- int numIds = source.readInt();
- target.writeInt(numIds);
- for (int i = 0; i < numIds; i++) {
- target.writeUTF(source.readUTF());
- }
- }
-
- // -----------------------------------------------------------------------------------
-
- @Override
- public TypeSerializerSnapshot<KafkaTransactionContext> snapshotConfiguration() {
- return new ContextStateSerializerSnapshot();
- }
-
- /** Serializer configuration snapshot for compatibility and format evolution. */
- @SuppressWarnings("WeakerAccess")
- public static final class ContextStateSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<KafkaTransactionContext> {
-
- public ContextStateSerializerSnapshot() {
- super(ContextStateSerializer::new);
- }
- }
- }
-
- /** Keep information required to deduce next safe to use transactional id. */
- public static class NextTransactionalIdHint {
- public int lastParallelism = 0;
- public long nextFreeTransactionalId = 0;
-
- public NextTransactionalIdHint() {
- this(0, 0);
- }
-
- public NextTransactionalIdHint(int parallelism, long nextFreeTransactionalId) {
- this.lastParallelism = parallelism;
- this.nextFreeTransactionalId = nextFreeTransactionalId;
- }
-
- @Override
- public String toString() {
- return "NextTransactionalIdHint["
- + "lastParallelism="
- + lastParallelism
- + ", nextFreeTransactionalId="
- + nextFreeTransactionalId
- + ']';
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- NextTransactionalIdHint that = (NextTransactionalIdHint) o;
-
- if (lastParallelism != that.lastParallelism) {
- return false;
- }
- return nextFreeTransactionalId == that.nextFreeTransactionalId;
- }
-
- @Override
- public int hashCode() {
- int result = lastParallelism;
- result =
- 31 * result
- + (int) (nextFreeTransactionalId ^ (nextFreeTransactionalId >>> 32));
- return result;
- }
- }
-
- /**
- * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link
- * FlinkKafkaProducer.NextTransactionalIdHint}.
- */
- @VisibleForTesting
- @Internal
- public static class NextTransactionalIdHintSerializer
- extends TypeSerializerSingleton<NextTransactionalIdHint> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public boolean isImmutableType() {
- return true;
- }
-
- @Override
- public NextTransactionalIdHint createInstance() {
- return new NextTransactionalIdHint();
- }
-
- @Override
- public NextTransactionalIdHint copy(NextTransactionalIdHint from) {
- return from;
- }
-
- @Override
- public NextTransactionalIdHint copy(
- NextTransactionalIdHint from, NextTransactionalIdHint reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return Long.BYTES + Integer.BYTES;
- }
-
- @Override
- public void serialize(NextTransactionalIdHint record, DataOutputView target)
- throws IOException {
- target.writeLong(record.nextFreeTransactionalId);
- target.writeInt(record.lastParallelism);
- }
-
- @Override
- public NextTransactionalIdHint deserialize(DataInputView source) throws IOException {
- long nextFreeTransactionalId = source.readLong();
- int lastParallelism = source.readInt();
- return new NextTransactionalIdHint(lastParallelism, nextFreeTransactionalId);
- }
-
- @Override
- public NextTransactionalIdHint deserialize(
- NextTransactionalIdHint reuse, DataInputView source) throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- target.writeLong(source.readLong());
- target.writeInt(source.readInt());
- }
-
- @Override
- public TypeSerializerSnapshot<NextTransactionalIdHint> snapshotConfiguration() {
- return new NextTransactionalIdHintSerializerSnapshot();
- }
-
- /** Serializer configuration snapshot for compatibility and format evolution. */
- @SuppressWarnings("WeakerAccess")
- public static final class NextTransactionalIdHintSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<NextTransactionalIdHint> {
-
- public NextTransactionalIdHintSerializerSnapshot() {
- super(NextTransactionalIdHintSerializer::new);
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
deleted file mode 100644
index c5b008c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.api.common.typeutils.SimpleTypeSerializerSnapshot;
-
-/**
- * Compatibility class to make migration possible from the 0.11 connector to the universal one.
- *
- * <p>Problem is that FlinkKafkaProducer (universal) and FlinkKafkaProducer011 have different names
- * and they both defined static classes NextTransactionalIdHint, KafkaTransactionState and
- * KafkaTransactionContext inside the parent classes. This is causing incompatibility problems since
- * for example FlinkKafkaProducer011.KafkaTransactionState and
- * FlinkKafkaProducer.KafkaTransactionState are treated as completely incompatible classes, despite
- * being identical.
- *
- * <p>This issue is solved by using custom serialization logic: keeping a fake/dummy
- * FlinkKafkaProducer011.*Serializer classes in the universal connector (this class), as entry
- * points for the deserialization and converting them to FlinkKafkaProducer.*Serializer counter
- * parts. After all serialized binary data are exactly the same in all of those cases.
- *
- * <p>For more details check FLINK-11249 and the discussion in the pull requests.
- */
-// CHECKSTYLE:OFF: JavadocType
-public class FlinkKafkaProducer011 {
- public static class NextTransactionalIdHintSerializer {
- public static final class NextTransactionalIdHintSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<FlinkKafkaProducer.NextTransactionalIdHint> {
- public NextTransactionalIdHintSerializerSnapshot() {
- super(FlinkKafkaProducer.NextTransactionalIdHintSerializer::new);
- }
- }
- }
-
- public static class ContextStateSerializer {
- public static final class ContextStateSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<FlinkKafkaProducer.KafkaTransactionContext> {
- public ContextStateSerializerSnapshot() {
- super(FlinkKafkaProducer.ContextStateSerializer::new);
- }
- }
- }
-
- public static class TransactionStateSerializer {
- public static final class TransactionStateSerializerSnapshot
- extends SimpleTypeSerializerSnapshot<FlinkKafkaProducer.KafkaTransactionState> {
- public TransactionStateSerializerSnapshot() {
- super(FlinkKafkaProducer.TransactionStateSerializer::new);
- }
- }
- }
-
- public static class NextTransactionalIdHint
- extends FlinkKafkaProducer.NextTransactionalIdHint {}
-}
-// CHECKSTYLE:ON: JavadocType
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
deleted file mode 100644
index 234fd9a..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java
+++ /dev/null
@@ -1,442 +0,0 @@
-/*
- * 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.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.SerializableObject;
-
-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.clients.producer.RecordMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Flink Sink to produce data into a Kafka topic.
- *
- * <p>Please note that this producer provides at-least-once reliability guarantees when checkpoints
- * are enabled and setFlushOnCheckpoint(true) is set. Otherwise, the producer doesn't provide any
- * reliability guarantees.
- *
- * @param <IN> Type of the messages to write into Kafka.
- */
-@Internal
-public abstract class FlinkKafkaProducerBase<IN> extends RichSinkFunction<IN>
- implements CheckpointedFunction {
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
-
- private static final long serialVersionUID = 1L;
-
- /** Configuration key for disabling the metrics reporting. */
- public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
-
- /** User defined properties for the Producer. */
- protected final Properties producerConfig;
-
- /** The name of the default topic this producer is writing data to. */
- protected final String defaultTopicId;
-
- /**
- * (Serializable) SerializationSchema for turning objects used with Flink into. byte[] for
- * Kafka.
- */
- protected final KeyedSerializationSchema<IN> schema;
-
- /** User-provided partitioner for assigning an object to a Kafka partition for each topic. */
- protected final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
-
- /** Partitions of each topic. */
- protected final Map<String, int[]> topicPartitionsMap;
-
- /** Flag indicating whether to accept failures (and log them), or to fail on failures. */
- protected boolean logFailuresOnly;
-
- /**
- * If true, the producer will wait until all outstanding records have been send to the broker.
- */
- protected boolean flushOnCheckpoint = true;
-
- // -------------------------------- Runtime fields ------------------------------------------
-
- /** KafkaProducer instance. */
- protected transient KafkaProducer<byte[], byte[]> producer;
-
- /** The callback than handles error propagation or logging callbacks. */
- protected transient Callback callback;
-
- /** Errors encountered in the async producer are stored here. */
- protected transient volatile Exception asyncException;
-
- /** Lock for accessing the pending records. */
- protected final SerializableObject pendingRecordsLock = new SerializableObject();
-
- /** Number of unacknowledged records. */
- protected long pendingRecords;
-
- /**
- * The main constructor for creating a FlinkKafkaProducer.
- *
- * @param defaultTopicId The default topic to write data to
- * @param serializationSchema A serializable serialization schema for turning user objects into
- * a kafka-consumable byte[] supporting key/value messages
- * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
- * the only required argument.
- * @param customPartitioner A serializable partitioner for assigning messages to Kafka
- * partitions. Passing null will use Kafka's partitioner.
- */
- public FlinkKafkaProducerBase(
- String defaultTopicId,
- KeyedSerializationSchema<IN> serializationSchema,
- Properties producerConfig,
- FlinkKafkaPartitioner<IN> customPartitioner) {
- requireNonNull(defaultTopicId, "TopicID not set");
- requireNonNull(serializationSchema, "serializationSchema not set");
- requireNonNull(producerConfig, "producerConfig not set");
- ClosureCleaner.clean(
- customPartitioner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
- ClosureCleaner.ensureSerializable(serializationSchema);
-
- this.defaultTopicId = defaultTopicId;
- this.schema = serializationSchema;
- this.producerConfig = producerConfig;
- this.flinkKafkaPartitioner = customPartitioner;
-
- // set the producer configuration properties for kafka record key value serializers.
- if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
- this.producerConfig.put(
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
- ByteArraySerializer.class.getName());
- } else {
- LOG.warn(
- "Overwriting the '{}' is not recommended",
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
- }
-
- if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
- this.producerConfig.put(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
- ByteArraySerializer.class.getName());
- } else {
- LOG.warn(
- "Overwriting the '{}' is not recommended",
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
- }
-
- // eagerly ensure that bootstrap servers are set.
- if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
- throw new IllegalArgumentException(
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG
- + " must be supplied in the producer config properties.");
- }
-
- this.topicPartitionsMap = new HashMap<>();
- }
-
- // ---------------------------------- Properties --------------------------
-
- /**
- * Defines whether the producer should fail on errors, or only log them. If this is set to true,
- * then exceptions will be only logged, if set to false, exceptions will be eventually thrown
- * and cause the streaming program to fail (and enter recovery).
- *
- * @param logFailuresOnly The flag to indicate logging-only on exceptions.
- */
- public void setLogFailuresOnly(boolean logFailuresOnly) {
- this.logFailuresOnly = logFailuresOnly;
- }
-
- /**
- * If set to true, the Flink producer will wait for all outstanding messages in the Kafka
- * buffers to be acknowledged by the Kafka producer on a checkpoint. This way, the producer can
- * guarantee that messages in the Kafka buffers are part of the checkpoint.
- *
- * @param flush Flag indicating the flushing mode (true = flush on checkpoint)
- */
- public void setFlushOnCheckpoint(boolean flush) {
- this.flushOnCheckpoint = flush;
- }
-
- /** Used for testing only. */
- @VisibleForTesting
- protected <K, V> KafkaProducer<K, V> getKafkaProducer(Properties props) {
- return new KafkaProducer<>(props);
- }
-
- // ----------------------------------- Utilities --------------------------
-
- /** Initializes the connection to Kafka. */
- @Override
- public void open(Configuration configuration) throws Exception {
- if (schema instanceof KeyedSerializationSchemaWrapper) {
- ((KeyedSerializationSchemaWrapper<IN>) schema)
- .getSerializationSchema()
- .open(
- RuntimeContextInitializationContextAdapters.serializationAdapter(
- getRuntimeContext(),
- metricGroup -> metricGroup.addGroup("user")));
- }
- producer = getKafkaProducer(this.producerConfig);
-
- RuntimeContext ctx = getRuntimeContext();
-
- if (null != flinkKafkaPartitioner) {
- flinkKafkaPartitioner.open(
- ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks());
- }
-
- LOG.info(
- "Starting FlinkKafkaProducer ({}/{}) to produce into default topic {}",
- ctx.getIndexOfThisSubtask() + 1,
- ctx.getNumberOfParallelSubtasks(),
- defaultTopicId);
-
- // register Kafka metrics to Flink accumulators
- if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
- Map<MetricName, ? extends Metric> metrics = this.producer.metrics();
-
- if (metrics == null) {
- // MapR's Kafka implementation returns null here.
- LOG.info("Producer implementation does not support metrics");
- } else {
- final MetricGroup kafkaMetricGroup =
- getRuntimeContext().getMetricGroup().addGroup("KafkaProducer");
- for (Map.Entry<MetricName, ? extends Metric> metric : metrics.entrySet()) {
- kafkaMetricGroup.gauge(
- metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
- }
- }
- }
-
- if (flushOnCheckpoint
- && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
- LOG.warn(
- "Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
- flushOnCheckpoint = false;
- }
-
- if (logFailuresOnly) {
- callback =
- new Callback() {
- @Override
- public void onCompletion(RecordMetadata metadata, Exception e) {
- if (e != null) {
- LOG.error(
- "Error while sending record to Kafka: " + e.getMessage(),
- e);
- }
- acknowledgeMessage();
- }
- };
- } else {
- callback =
- new Callback() {
- @Override
- public void onCompletion(RecordMetadata metadata, Exception exception) {
- if (exception != null && asyncException == null) {
- asyncException = exception;
- }
- acknowledgeMessage();
- }
- };
- }
- }
-
- /**
- * Called when new data arrives to the sink, and forwards it to Kafka.
- *
- * @param next The incoming data
- */
- @Override
- public void invoke(IN next, Context context) throws Exception {
- // propagate asynchronous errors
- checkErroneous();
-
- byte[] serializedKey = schema.serializeKey(next);
- byte[] serializedValue = schema.serializeValue(next);
- String targetTopic = schema.getTargetTopic(next);
- if (targetTopic == null) {
- targetTopic = defaultTopicId;
- }
-
- int[] partitions = this.topicPartitionsMap.get(targetTopic);
- if (null == partitions) {
- partitions = getPartitionsByTopic(targetTopic, producer);
- this.topicPartitionsMap.put(targetTopic, partitions);
- }
-
- ProducerRecord<byte[], byte[]> record;
- if (flinkKafkaPartitioner == null) {
- record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue);
- } else {
- record =
- new ProducerRecord<>(
- targetTopic,
- flinkKafkaPartitioner.partition(
- next, serializedKey, serializedValue, targetTopic, partitions),
- serializedKey,
- serializedValue);
- }
- if (flushOnCheckpoint) {
- synchronized (pendingRecordsLock) {
- pendingRecords++;
- }
- }
- producer.send(record, callback);
- }
-
- @Override
- public void close() throws Exception {
- if (producer != null) {
- producer.close();
- }
-
- // make sure we propagate pending errors
- checkErroneous();
- }
-
- // ------------------- Logic for handling checkpoint flushing -------------------------- //
-
- private void acknowledgeMessage() {
- if (flushOnCheckpoint) {
- synchronized (pendingRecordsLock) {
- pendingRecords--;
- if (pendingRecords == 0) {
- pendingRecordsLock.notifyAll();
- }
- }
- }
- }
-
- /** Flush pending records. */
- protected abstract void flush();
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {
- // nothing to do
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
- // check for asynchronous errors and fail the checkpoint if necessary
- checkErroneous();
-
- if (flushOnCheckpoint) {
- // flushing is activated: We need to wait until pendingRecords is 0
- flush();
- synchronized (pendingRecordsLock) {
- if (pendingRecords != 0) {
- throw new IllegalStateException(
- "Pending record count must be zero at this point: " + pendingRecords);
- }
-
- // if the flushed requests has errors, we should propagate it also and fail the
- // checkpoint
- checkErroneous();
- }
- }
- }
-
- // ----------------------------------- Utilities --------------------------
-
- protected void checkErroneous() throws Exception {
- Exception e = asyncException;
- if (e != null) {
- // prevent double throwing
- asyncException = null;
- throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
- }
- }
-
- public static Properties getPropertiesFromBrokerList(String brokerList) {
- String[] elements = brokerList.split(",");
-
- // validate the broker addresses
- for (String broker : elements) {
- NetUtils.getCorrectHostnamePort(broker);
- }
-
- Properties props = new Properties();
- props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
- return props;
- }
-
- protected static int[] getPartitionsByTopic(
- String topic, KafkaProducer<byte[], byte[]> producer) {
- // the fetched list is immutable, so we're creating a mutable copy in order to sort it
- List<PartitionInfo> partitionsList = new ArrayList<>(producer.partitionsFor(topic));
-
- // sort the partitions by partition id to make sure the fetched partition list is the same
- // across subtasks
- Collections.sort(
- partitionsList,
- new Comparator<PartitionInfo>() {
- @Override
- public int compare(PartitionInfo o1, PartitionInfo o2) {
- return Integer.compare(o1.partition(), o2.partition());
- }
- });
-
- int[] partitions = new int[partitionsList.size()];
- for (int i = 0; i < partitions.length; i++) {
- partitions[i] = partitionsList.get(i).partition();
- }
-
- return partitions;
- }
-
- @VisibleForTesting
- protected long numPendingRecords() {
- synchronized (pendingRecordsLock) {
- return pendingRecords;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java
deleted file mode 100644
index ad977cd..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-
-/**
- * An interface for {@link KafkaSerializationSchema KafkaSerializationSchemas} that need information
- * about the context where the Kafka Producer is running along with information about the available
- * partitions.
- *
- * <p>You only need to override the methods for the information that you need. However, {@link
- * #getTargetTopic(Object)} is required because it is used to determine the available partitions.
- */
-@PublicEvolving
-public interface KafkaContextAware<T> {
-
- /**
- * Sets the number of the parallel subtask that the Kafka Producer is running on. The numbering
- * starts from 0 and goes up to parallelism-1 (parallelism as returned by {@link
- * #setNumParallelInstances(int)}).
- */
- default void setParallelInstanceId(int parallelInstanceId) {}
-
- /** Sets the parallelism with which the parallel task of the Kafka Producer runs. */
- default void setNumParallelInstances(int numParallelInstances) {}
-
- /**
- * Sets the available partitions for the topic returned from {@link #getTargetTopic(Object)}.
- */
- default void setPartitions(int[] partitions) {}
-
- /**
- * Returns the topic that the presented element should be sent to. This is not used for setting
- * the topic (this is done via the {@link org.apache.kafka.clients.producer.ProducerRecord} that
- * is returned from {@link KafkaSerializationSchema#serialize(Object, Long)}, it is only used
- * for getting the available partitions that are presented to {@link #setPartitions(int[])}.
- */
- String getTargetTopic(T element);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java
deleted file mode 100644
index b54b983..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.Serializable;
-
-/**
- * The deserialization schema describes how to turn the Kafka ConsumerRecords into data types
- * (Java/Scala objects) that are processed by Flink.
- *
- * @param <T> The type created by the keyed deserialization schema.
- */
-@PublicEvolving
-public interface KafkaDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
-
- /**
- * Initialization method for the schema. It is called before the actual working methods {@link
- * #deserialize} and thus suitable for one time setup work.
- *
- * <p>The provided {@link DeserializationSchema.InitializationContext} can be used to access
- * additional features such as e.g. registering user metrics.
- *
- * @param context Contextual information that can be used during initialization.
- */
- default void open(DeserializationSchema.InitializationContext context) throws Exception {}
-
- /**
- * Method to decide whether the element signals the end of the stream. If true is returned the
- * element won't be emitted.
- *
- * @param nextElement The element to test for the end-of-stream signal.
- * @return True, if the element signals end of stream, false otherwise.
- */
- boolean isEndOfStream(T nextElement);
-
- /**
- * Deserializes the Kafka record.
- *
- * @param record Kafka record to be deserialized.
- * @return The deserialized message as an object (null if the message cannot be deserialized).
- */
- T deserialize(ConsumerRecord<byte[], byte[]> record) throws Exception;
-
- /**
- * Deserializes the Kafka record.
- *
- * <p>Can output multiple records through the {@link Collector}. Note that number and size of
- * the produced records should be relatively small. Depending on the source implementation
- * records can be buffered in memory or collecting records might delay emitting checkpoint
- * barrier.
- *
- * @param message The message, as a byte array.
- * @param out The collector to put the resulting messages.
- */
- default void deserialize(ConsumerRecord<byte[], byte[]> message, Collector<T> out)
- throws Exception {
- T deserialized = deserialize(message);
- if (deserialized != null) {
- out.collect(deserialized);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java
deleted file mode 100644
index 89e2b92..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-
-/**
- * A {@link KafkaSerializationSchema} defines how to serialize values of type {@code T} into {@link
- * ProducerRecord ProducerRecords}.
- *
- * <p>Please also implement {@link KafkaContextAware} if your serialization schema needs information
- * about the available partitions and the number of parallel subtasks along with the subtask ID on
- * which the Kafka Producer is running.
- *
- * @param <T> the type of values being serialized
- */
-@PublicEvolving
-public interface KafkaSerializationSchema<T> extends Serializable {
-
- /**
- * Initialization method for the schema. It is called before the actual working methods {@link
- * #serialize(Object, Long)} and thus suitable for one time setup work.
- *
- * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
- * additional features such as e.g. registering user metrics.
- *
- * @param context Contextual information that can be used during initialization.
- */
- default void open(SerializationSchema.InitializationContext context) throws Exception {}
-
- /**
- * Serializes given element and returns it as a {@link ProducerRecord}.
- *
- * @param element element to be serialized
- * @param timestamp timestamp (can be null)
- * @return Kafka {@link ProducerRecord}
- */
- ProducerRecord<byte[], byte[]> serialize(T element, @Nullable Long timestamp);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/BoundedMode.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/BoundedMode.java
deleted file mode 100644
index beb2306..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/BoundedMode.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.config;
-
-import org.apache.flink.annotation.Internal;
-
-/** End modes for the Kafka Consumer. */
-@Internal
-public enum BoundedMode {
-
- /** Do not end consuming. */
- UNBOUNDED,
-
- /**
- * End from committed offsets in ZK / Kafka brokers of a specific consumer group. This is
- * evaluated at the start of consumption from a given partition.
- */
- GROUP_OFFSETS,
-
- /**
- * End from the latest offset. This is evaluated at the start of consumption from a given
- * partition.
- */
- LATEST,
-
- /** End from user-supplied timestamp for each partition. */
- TIMESTAMP,
-
- /**
- * End from user-supplied specific offsets for each partition. If an offset for a partition is
- * not provided it will not consume from that partition.
- */
- SPECIFIC_OFFSETS;
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java
deleted file mode 100644
index 32b9d4c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.config;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * The offset commit mode represents the behaviour of how offsets are externally committed back to
- * Kafka brokers / Zookeeper.
- *
- * <p>The exact value of this is determined at runtime in the consumer subtasks.
- */
-@Internal
-public enum OffsetCommitMode {
-
- /** Completely disable offset committing. */
- DISABLED,
-
- /** Commit offsets back to Kafka only when checkpoints are completed. */
- ON_CHECKPOINTS,
-
- /**
- * Commit offsets periodically back to Kafka, using the auto commit functionality of internal
- * Kafka clients.
- */
- KAFKA_PERIODIC;
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java
deleted file mode 100644
index 1394af7..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.config;
-
-import org.apache.flink.annotation.Internal;
-
-/** Utilities for {@link OffsetCommitMode}. */
-@Internal
-public class OffsetCommitModes {
-
- /**
- * Determine the offset commit mode using several configuration values.
- *
- * @param enableAutoCommit whether or not auto committing is enabled in the provided Kafka
- * properties.
- * @param enableCommitOnCheckpoint whether or not committing on checkpoints is enabled.
- * @param enableCheckpointing whether or not checkpoint is enabled for the consumer.
- * @return the offset commit mode to use, based on the configuration values.
- */
- public static OffsetCommitMode fromConfiguration(
- boolean enableAutoCommit,
- boolean enableCommitOnCheckpoint,
- boolean enableCheckpointing) {
-
- if (enableCheckpointing) {
- // if checkpointing is enabled, the mode depends only on whether committing on
- // checkpoints is enabled
- return (enableCommitOnCheckpoint)
- ? OffsetCommitMode.ON_CHECKPOINTS
- : OffsetCommitMode.DISABLED;
- } else {
- // else, the mode depends only on whether auto committing is enabled in the provided
- // Kafka properties
- return (enableAutoCommit) ? OffsetCommitMode.KAFKA_PERIODIC : OffsetCommitMode.DISABLED;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java
deleted file mode 100644
index 0aa43ec..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.config;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-
-/** Startup modes for the Kafka Consumer. */
-@Internal
-public enum StartupMode {
-
- /**
- * Start from committed offsets in ZK / Kafka brokers of a specific consumer group (default).
- */
- GROUP_OFFSETS(KafkaTopicPartitionStateSentinel.GROUP_OFFSET),
-
- /** Start from the earliest offset possible. */
- EARLIEST(KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET),
-
- /** Start from the latest offset. */
- LATEST(KafkaTopicPartitionStateSentinel.LATEST_OFFSET),
-
- /**
- * Start from user-supplied timestamp for each partition. Since this mode will have specific
- * offsets to start with, we do not need a sentinel value; using Long.MIN_VALUE as a
- * placeholder.
- */
- TIMESTAMP(Long.MIN_VALUE),
-
- /**
- * Start from user-supplied specific offsets for each partition. Since this mode will have
- * specific offsets to start with, we do not need a sentinel value; using Long.MIN_VALUE as a
- * placeholder.
- */
- SPECIFIC_OFFSETS(Long.MIN_VALUE);
-
- /** The sentinel offset value corresponding to this startup mode. */
- private long stateSentinel;
-
- StartupMode(long stateSentinel) {
- this.stateSentinel = stateSentinel;
- }
-
- public long getStateSentinel() {
- return stateSentinel;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
deleted file mode 100644
index 841d452..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
+++ /dev/null
@@ -1,611 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.eventtime.WatermarkOutput;
-import org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.SerializedValue;
-
-import javax.annotation.Nonnull;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.COMMITTED_OFFSETS_METRICS_GAUGE;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.CURRENT_OFFSETS_METRICS_GAUGE;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.LEGACY_COMMITTED_OFFSETS_METRICS_GROUP;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.LEGACY_CURRENT_OFFSETS_METRICS_GROUP;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.OFFSETS_BY_PARTITION_METRICS_GROUP;
-import static org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaConsumerMetricConstants.OFFSETS_BY_TOPIC_METRICS_GROUP;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Base class for all fetchers, which implement the connections to Kafka brokers and pull records
- * from Kafka partitions.
- *
- * <p>This fetcher base class implements the logic around emitting records and tracking offsets, as
- * well as around the optional timestamp assignment and watermark generation.
- *
- * @param <T> The type of elements deserialized from Kafka's byte records, and emitted into the
- * Flink data streams.
- * @param <KPH> The type of topic/partition identifier used by Kafka in the specific version.
- */
-@Internal
-public abstract class AbstractFetcher<T, KPH> {
-
- private static final int NO_TIMESTAMPS_WATERMARKS = 0;
- private static final int WITH_WATERMARK_GENERATOR = 1;
-
- // ------------------------------------------------------------------------
-
- /** The source context to emit records and watermarks to. */
- protected final SourceContext<T> sourceContext;
-
- /**
- * Wrapper around our SourceContext for allowing the {@link
- * org.apache.flink.api.common.eventtime.WatermarkGenerator} to emit watermarks and mark
- * idleness.
- */
- protected final WatermarkOutput watermarkOutput;
-
- /** {@link WatermarkOutputMultiplexer} for supporting per-partition watermark generation. */
- private final WatermarkOutputMultiplexer watermarkOutputMultiplexer;
-
- /**
- * The lock that guarantees that record emission and state updates are atomic, from the view of
- * taking a checkpoint.
- */
- protected final Object checkpointLock;
-
- /** All partitions (and their state) that this fetcher is subscribed to. */
- private final List<KafkaTopicPartitionState<T, KPH>> subscribedPartitionStates;
-
- /**
- * Queue of partitions that are not yet assigned to any Kafka clients for consuming. Kafka
- * version-specific implementations of {@link AbstractFetcher#runFetchLoop()} should
- * continuously poll this queue for unassigned partitions, and start consuming them accordingly.
- *
- * <p>All partitions added to this queue are guaranteed to have been added to {@link
- * #subscribedPartitionStates} already.
- */
- protected final ClosableBlockingQueue<KafkaTopicPartitionState<T, KPH>>
- unassignedPartitionsQueue;
-
- /** The mode describing whether the fetcher also generates timestamps and watermarks. */
- private final int timestampWatermarkMode;
-
- /**
- * Optional watermark strategy that will be run per Kafka partition, to exploit per-partition
- * timestamp characteristics. The watermark strategy is kept in serialized form, to deserialize
- * it into multiple copies.
- */
- private final SerializedValue<WatermarkStrategy<T>> watermarkStrategy;
-
- /** User class loader used to deserialize watermark assigners. */
- private final ClassLoader userCodeClassLoader;
-
- // ------------------------------------------------------------------------
- // Metrics
- // ------------------------------------------------------------------------
-
- /**
- * Flag indicating whether or not metrics should be exposed. If {@code true}, offset metrics
- * (e.g. current offset, committed offset) and Kafka-shipped metrics will be registered.
- */
- private final boolean useMetrics;
-
- /**
- * The metric group which all metrics for the consumer should be registered to. This metric
- * group is defined under the user scope {@link
- * KafkaConsumerMetricConstants#KAFKA_CONSUMER_METRICS_GROUP}.
- */
- private final MetricGroup consumerMetricGroup;
-
- @SuppressWarnings("DeprecatedIsStillUsed")
- @Deprecated
- private final MetricGroup legacyCurrentOffsetsMetricGroup;
-
- @SuppressWarnings("DeprecatedIsStillUsed")
- @Deprecated
- private final MetricGroup legacyCommittedOffsetsMetricGroup;
-
- protected AbstractFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> seedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval,
- ClassLoader userCodeClassLoader,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- this.sourceContext = checkNotNull(sourceContext);
- this.watermarkOutput = new SourceContextWatermarkOutputAdapter<>(sourceContext);
- this.watermarkOutputMultiplexer = new WatermarkOutputMultiplexer(watermarkOutput);
- this.checkpointLock = sourceContext.getCheckpointLock();
- this.userCodeClassLoader = checkNotNull(userCodeClassLoader);
-
- this.useMetrics = useMetrics;
- this.consumerMetricGroup = checkNotNull(consumerMetricGroup);
- this.legacyCurrentOffsetsMetricGroup =
- consumerMetricGroup.addGroup(LEGACY_CURRENT_OFFSETS_METRICS_GROUP);
- this.legacyCommittedOffsetsMetricGroup =
- consumerMetricGroup.addGroup(LEGACY_COMMITTED_OFFSETS_METRICS_GROUP);
-
- this.watermarkStrategy = watermarkStrategy;
-
- if (watermarkStrategy == null) {
- timestampWatermarkMode = NO_TIMESTAMPS_WATERMARKS;
- } else {
- timestampWatermarkMode = WITH_WATERMARK_GENERATOR;
- }
-
- this.unassignedPartitionsQueue = new ClosableBlockingQueue<>();
-
- // initialize subscribed partition states with seed partitions
- this.subscribedPartitionStates =
- createPartitionStateHolders(
- seedPartitionsWithInitialOffsets,
- timestampWatermarkMode,
- watermarkStrategy,
- userCodeClassLoader);
-
- // check that all seed partition states have a defined offset
- for (KafkaTopicPartitionState<?, ?> partitionState : subscribedPartitionStates) {
- if (!partitionState.isOffsetDefined()) {
- throw new IllegalArgumentException(
- "The fetcher was assigned seed partitions with undefined initial offsets.");
- }
- }
-
- // all seed partitions are not assigned yet, so should be added to the unassigned partitions
- // queue
- for (KafkaTopicPartitionState<T, KPH> partition : subscribedPartitionStates) {
- unassignedPartitionsQueue.add(partition);
- }
-
- // register metrics for the initial seed partitions
- if (useMetrics) {
- registerOffsetMetrics(consumerMetricGroup, subscribedPartitionStates);
- }
-
- // if we have periodic watermarks, kick off the interval scheduler
- if (timestampWatermarkMode == WITH_WATERMARK_GENERATOR && autoWatermarkInterval > 0) {
- PeriodicWatermarkEmitter<T, KPH> periodicEmitter =
- new PeriodicWatermarkEmitter<>(
- checkpointLock,
- subscribedPartitionStates,
- watermarkOutputMultiplexer,
- processingTimeProvider,
- autoWatermarkInterval);
-
- periodicEmitter.start();
- }
- }
-
- /**
- * Adds a list of newly discovered partitions to the fetcher for consuming.
- *
- * <p>This method creates the partition state holder for each new partition, using {@link
- * KafkaTopicPartitionStateSentinel#EARLIEST_OFFSET} as the starting offset. It uses the
- * earliest offset because there may be delay in discovering a partition after it was created
- * and started receiving records.
- *
- * <p>After the state representation for a partition is created, it is added to the unassigned
- * partitions queue to await to be consumed.
- *
- * @param newPartitions discovered partitions to add
- */
- public void addDiscoveredPartitions(List<KafkaTopicPartition> newPartitions)
- throws IOException, ClassNotFoundException {
- List<KafkaTopicPartitionState<T, KPH>> newPartitionStates =
- createPartitionStateHolders(
- newPartitions,
- KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET,
- timestampWatermarkMode,
- watermarkStrategy,
- userCodeClassLoader);
-
- if (useMetrics) {
- registerOffsetMetrics(consumerMetricGroup, newPartitionStates);
- }
-
- for (KafkaTopicPartitionState<T, KPH> newPartitionState : newPartitionStates) {
- // the ordering is crucial here; first register the state holder, then
- // push it to the partitions queue to be read
- subscribedPartitionStates.add(newPartitionState);
- unassignedPartitionsQueue.add(newPartitionState);
- }
- }
-
- // ------------------------------------------------------------------------
- // Properties
- // ------------------------------------------------------------------------
-
- /**
- * Gets all partitions (with partition state) that this fetcher is subscribed to.
- *
- * @return All subscribed partitions.
- */
- protected final List<KafkaTopicPartitionState<T, KPH>> subscribedPartitionStates() {
- return subscribedPartitionStates;
- }
-
- // ------------------------------------------------------------------------
- // Core fetcher work methods
- // ------------------------------------------------------------------------
-
- public abstract void runFetchLoop() throws Exception;
-
- public abstract void cancel();
-
- // ------------------------------------------------------------------------
- // Kafka version specifics
- // ------------------------------------------------------------------------
-
- /**
- * Commits the given partition offsets to the Kafka brokers (or to ZooKeeper for older Kafka
- * versions). This method is only ever called when the offset commit mode of the consumer is
- * {@link OffsetCommitMode#ON_CHECKPOINTS}.
- *
- * <p>The given offsets are the internal checkpointed offsets, representing the last processed
- * record of each partition. Version-specific implementations of this method need to hold the
- * contract that the given offsets must be incremented by 1 before committing them, so that
- * committed offsets to Kafka represent "the next record to process".
- *
- * @param offsets The offsets to commit to Kafka (implementations must increment offsets by 1
- * before committing).
- * @param commitCallback The callback that the user should trigger when a commit request
- * completes or fails.
- * @throws Exception This method forwards exceptions.
- */
- public final void commitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback commitCallback)
- throws Exception {
- // Ignore sentinels. They might appear here if snapshot has started before actual offsets
- // values
- // replaced sentinels
- doCommitInternalOffsetsToKafka(filterOutSentinels(offsets), commitCallback);
- }
-
- protected abstract void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback commitCallback)
- throws Exception;
-
- private Map<KafkaTopicPartition, Long> filterOutSentinels(
- Map<KafkaTopicPartition, Long> offsets) {
- return offsets.entrySet().stream()
- .filter(entry -> !KafkaTopicPartitionStateSentinel.isSentinel(entry.getValue()))
- .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
- }
-
- /**
- * Creates the Kafka version specific representation of the given topic partition.
- *
- * @param partition The Flink representation of the Kafka topic partition.
- * @return The version-specific Kafka representation of the Kafka topic partition.
- */
- protected abstract KPH createKafkaPartitionHandle(KafkaTopicPartition partition);
-
- // ------------------------------------------------------------------------
- // snapshot and restore the state
- // ------------------------------------------------------------------------
-
- /**
- * Takes a snapshot of the partition offsets.
- *
- * <p>Important: This method must be called under the checkpoint lock.
- *
- * @return A map from partition to current offset.
- */
- public HashMap<KafkaTopicPartition, Long> snapshotCurrentState() {
- // this method assumes that the checkpoint lock is held
- assert Thread.holdsLock(checkpointLock);
-
- HashMap<KafkaTopicPartition, Long> state = new HashMap<>(subscribedPartitionStates.size());
- for (KafkaTopicPartitionState<T, KPH> partition : subscribedPartitionStates) {
- state.put(partition.getKafkaTopicPartition(), partition.getOffset());
- }
- return state;
- }
-
- // ------------------------------------------------------------------------
- // emitting records
- // ------------------------------------------------------------------------
-
- /**
- * Emits a record attaching a timestamp to it.
- *
- * @param records The records to emit
- * @param partitionState The state of the Kafka partition from which the record was fetched
- * @param offset The offset of the corresponding Kafka record
- * @param kafkaEventTimestamp The timestamp of the Kafka record
- */
- protected void emitRecordsWithTimestamps(
- Queue<T> records,
- KafkaTopicPartitionState<T, KPH> partitionState,
- long offset,
- long kafkaEventTimestamp) {
- // emit the records, using the checkpoint lock to guarantee
- // atomicity of record emission and offset state update
- synchronized (checkpointLock) {
- T record;
- while ((record = records.poll()) != null) {
- long timestamp = partitionState.extractTimestamp(record, kafkaEventTimestamp);
- sourceContext.collectWithTimestamp(record, timestamp);
-
- // this might emit a watermark, so do it after emitting the record
- partitionState.onEvent(record, timestamp);
- }
- partitionState.setOffset(offset);
- }
- }
-
- // ------------------------------------------------------------------------
- // Utilities
- // ------------------------------------------------------------------------
-
- /**
- * Utility method that takes the topic partitions and creates the topic partition state holders,
- * depending on the timestamp / watermark mode.
- */
- private List<KafkaTopicPartitionState<T, KPH>> createPartitionStateHolders(
- Map<KafkaTopicPartition, Long> partitionsToInitialOffsets,
- int timestampWatermarkMode,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ClassLoader userCodeClassLoader)
- throws IOException, ClassNotFoundException {
-
- // CopyOnWrite as adding discovered partitions could happen in parallel
- // while different threads iterate the partitions list
- List<KafkaTopicPartitionState<T, KPH>> partitionStates = new CopyOnWriteArrayList<>();
-
- switch (timestampWatermarkMode) {
- case NO_TIMESTAMPS_WATERMARKS:
- {
- for (Map.Entry<KafkaTopicPartition, Long> partitionEntry :
- partitionsToInitialOffsets.entrySet()) {
- // create the kafka version specific partition handle
- KPH kafkaHandle = createKafkaPartitionHandle(partitionEntry.getKey());
-
- KafkaTopicPartitionState<T, KPH> partitionState =
- new KafkaTopicPartitionState<>(
- partitionEntry.getKey(), kafkaHandle);
- partitionState.setOffset(partitionEntry.getValue());
-
- partitionStates.add(partitionState);
- }
-
- return partitionStates;
- }
-
- case WITH_WATERMARK_GENERATOR:
- {
- for (Map.Entry<KafkaTopicPartition, Long> partitionEntry :
- partitionsToInitialOffsets.entrySet()) {
- final KafkaTopicPartition kafkaTopicPartition = partitionEntry.getKey();
- KPH kafkaHandle = createKafkaPartitionHandle(kafkaTopicPartition);
- WatermarkStrategy<T> deserializedWatermarkStrategy =
- watermarkStrategy.deserializeValue(userCodeClassLoader);
-
- // the format of the ID does not matter, as long as it is unique
- final String partitionId =
- kafkaTopicPartition.getTopic()
- + '-'
- + kafkaTopicPartition.getPartition();
- watermarkOutputMultiplexer.registerNewOutput(partitionId, watermark -> {});
- WatermarkOutput immediateOutput =
- watermarkOutputMultiplexer.getImmediateOutput(partitionId);
- WatermarkOutput deferredOutput =
- watermarkOutputMultiplexer.getDeferredOutput(partitionId);
-
- KafkaTopicPartitionStateWithWatermarkGenerator<T, KPH> partitionState =
- new KafkaTopicPartitionStateWithWatermarkGenerator<>(
- partitionEntry.getKey(),
- kafkaHandle,
- deserializedWatermarkStrategy.createTimestampAssigner(
- () -> consumerMetricGroup),
- deserializedWatermarkStrategy.createWatermarkGenerator(
- () -> consumerMetricGroup),
- immediateOutput,
- deferredOutput);
-
- partitionState.setOffset(partitionEntry.getValue());
-
- partitionStates.add(partitionState);
- }
-
- return partitionStates;
- }
-
- default:
- // cannot happen, add this as a guard for the future
- throw new RuntimeException();
- }
- }
-
- /**
- * Shortcut variant of {@link #createPartitionStateHolders(Map, int, SerializedValue,
- * ClassLoader)} that uses the same offset for all partitions when creating their state holders.
- */
- private List<KafkaTopicPartitionState<T, KPH>> createPartitionStateHolders(
- List<KafkaTopicPartition> partitions,
- long initialOffset,
- int timestampWatermarkMode,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ClassLoader userCodeClassLoader)
- throws IOException, ClassNotFoundException {
-
- Map<KafkaTopicPartition, Long> partitionsToInitialOffset = new HashMap<>(partitions.size());
- for (KafkaTopicPartition partition : partitions) {
- partitionsToInitialOffset.put(partition, initialOffset);
- }
-
- return createPartitionStateHolders(
- partitionsToInitialOffset,
- timestampWatermarkMode,
- watermarkStrategy,
- userCodeClassLoader);
- }
-
- // ------------------------- Metrics ----------------------------------
-
- /**
- * For each partition, register a new metric group to expose current offsets and committed
- * offsets. Per-partition metric groups can be scoped by user variables {@link
- * KafkaConsumerMetricConstants#OFFSETS_BY_TOPIC_METRICS_GROUP} and {@link
- * KafkaConsumerMetricConstants#OFFSETS_BY_PARTITION_METRICS_GROUP}.
- *
- * <p>Note: this method also registers gauges for deprecated offset metrics, to maintain
- * backwards compatibility.
- *
- * @param consumerMetricGroup The consumer metric group
- * @param partitionOffsetStates The partition offset state holders, whose values will be used to
- * update metrics
- */
- private void registerOffsetMetrics(
- MetricGroup consumerMetricGroup,
- List<KafkaTopicPartitionState<T, KPH>> partitionOffsetStates) {
-
- for (KafkaTopicPartitionState<T, KPH> ktp : partitionOffsetStates) {
- MetricGroup topicPartitionGroup =
- consumerMetricGroup
- .addGroup(OFFSETS_BY_TOPIC_METRICS_GROUP, ktp.getTopic())
- .addGroup(
- OFFSETS_BY_PARTITION_METRICS_GROUP,
- Integer.toString(ktp.getPartition()));
-
- topicPartitionGroup.gauge(
- CURRENT_OFFSETS_METRICS_GAUGE,
- new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET));
- topicPartitionGroup.gauge(
- COMMITTED_OFFSETS_METRICS_GAUGE,
- new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET));
-
- legacyCurrentOffsetsMetricGroup.gauge(
- getLegacyOffsetsMetricsGaugeName(ktp),
- new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET));
- legacyCommittedOffsetsMetricGroup.gauge(
- getLegacyOffsetsMetricsGaugeName(ktp),
- new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET));
- }
- }
-
- private static String getLegacyOffsetsMetricsGaugeName(KafkaTopicPartitionState<?, ?> ktp) {
- return ktp.getTopic() + "-" + ktp.getPartition();
- }
-
- /** Gauge types. */
- private enum OffsetGaugeType {
- CURRENT_OFFSET,
- COMMITTED_OFFSET
- }
-
- /** Gauge for getting the offset of a KafkaTopicPartitionState. */
- private static class OffsetGauge implements Gauge<Long> {
-
- private final KafkaTopicPartitionState<?, ?> ktp;
- private final OffsetGaugeType gaugeType;
-
- OffsetGauge(KafkaTopicPartitionState<?, ?> ktp, OffsetGaugeType gaugeType) {
- this.ktp = ktp;
- this.gaugeType = gaugeType;
- }
-
- @Override
- public Long getValue() {
- switch (gaugeType) {
- case COMMITTED_OFFSET:
- return ktp.getCommittedOffset();
- case CURRENT_OFFSET:
- return ktp.getOffset();
- default:
- throw new RuntimeException("Unknown gauge type: " + gaugeType);
- }
- }
- }
- // ------------------------------------------------------------------------
-
- /**
- * The periodic watermark emitter. In its given interval, it checks all partitions for the
- * current event time watermark, and possibly emits the next watermark.
- */
- private static class PeriodicWatermarkEmitter<T, KPH> implements ProcessingTimeCallback {
-
- private final Object checkpointLock;
-
- private final List<KafkaTopicPartitionState<T, KPH>> allPartitions;
-
- private final WatermarkOutputMultiplexer watermarkOutputMultiplexer;
-
- private final ProcessingTimeService timerService;
-
- private final long interval;
-
- // -------------------------------------------------
-
- PeriodicWatermarkEmitter(
- Object checkpointLock,
- List<KafkaTopicPartitionState<T, KPH>> allPartitions,
- WatermarkOutputMultiplexer watermarkOutputMultiplexer,
- ProcessingTimeService timerService,
- long autoWatermarkInterval) {
- this.checkpointLock = checkpointLock;
- this.allPartitions = checkNotNull(allPartitions);
- this.watermarkOutputMultiplexer = watermarkOutputMultiplexer;
- this.timerService = checkNotNull(timerService);
- this.interval = autoWatermarkInterval;
- }
-
- // -------------------------------------------------
-
- public void start() {
- timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
- }
-
- @Override
- public void onProcessingTime(long timestamp) {
-
- synchronized (checkpointLock) {
- for (KafkaTopicPartitionState<?, ?> state : allPartitions) {
- state.onPeriodicEmit();
- }
-
- watermarkOutputMultiplexer.onPeriodicEmit();
- }
-
- // schedule the next watermark
- timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java
deleted file mode 100644
index 05e078f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java
+++ /dev/null
@@ -1,256 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Base class for all partition discoverers.
- *
- * <p>This partition discoverer base class implements the logic around bookkeeping discovered
- * partitions, and using the information to determine whether or not there are new partitions that
- * the consumer subtask should subscribe to.
- *
- * <p>Subclass implementations should simply implement the logic of using the version-specific Kafka
- * clients to fetch topic and partition metadata.
- *
- * <p>Since Kafka clients are generally not thread-safe, partition discoverers should not be
- * concurrently accessed. The only exception for this would be the {@link #wakeup()} call, which
- * allows the discoverer to be interrupted during a {@link #discoverPartitions()} call.
- */
-@Internal
-public abstract class AbstractPartitionDiscoverer {
-
- /** Describes whether we are discovering partitions for fixed topics or a topic pattern. */
- private final KafkaTopicsDescriptor topicsDescriptor;
-
- /** Index of the consumer subtask that this partition discoverer belongs to. */
- private final int indexOfThisSubtask;
-
- /** The total number of consumer subtasks. */
- private final int numParallelSubtasks;
-
- /** Flag to determine whether or not the discoverer is closed. */
- private volatile boolean closed = true;
-
- /**
- * Flag to determine whether or not the discoverer had been woken up. When set to {@code true},
- * {@link #discoverPartitions()} would be interrupted as early as possible. Once interrupted,
- * the flag is reset.
- */
- private volatile boolean wakeup;
-
- /**
- * Map of topics to they're largest discovered partition id seen by this subtask. This state may
- * be updated whenever {@link AbstractPartitionDiscoverer#discoverPartitions()} or {@link
- * AbstractPartitionDiscoverer#setAndCheckDiscoveredPartition(KafkaTopicPartition)} is called.
- *
- * <p>This is used to remove old partitions from the fetched partition lists. It is sufficient
- * to keep track of only the largest partition id because Kafka partition numbers are only
- * allowed to be increased and has incremental ids.
- */
- private Set<KafkaTopicPartition> discoveredPartitions;
-
- public AbstractPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks) {
-
- this.topicsDescriptor = checkNotNull(topicsDescriptor);
- this.indexOfThisSubtask = indexOfThisSubtask;
- this.numParallelSubtasks = numParallelSubtasks;
- this.discoveredPartitions = new HashSet<>();
- }
-
- /**
- * Opens the partition discoverer, initializing all required Kafka connections.
- *
- * <p>NOTE: thread-safety is not guaranteed.
- */
- public void open() throws Exception {
- closed = false;
- initializeConnections();
- }
-
- /**
- * Closes the partition discoverer, cleaning up all Kafka connections.
- *
- * <p>NOTE: thread-safety is not guaranteed.
- */
- public void close() throws Exception {
- closed = true;
- closeConnections();
- }
-
- /**
- * Interrupt an in-progress discovery attempt by throwing a {@link WakeupException}. If no
- * attempt is in progress, the immediate next attempt will throw a {@link WakeupException}.
- *
- * <p>This method can be called concurrently from a different thread.
- */
- public void wakeup() {
- wakeup = true;
- wakeupConnections();
- }
-
- /**
- * Execute a partition discovery attempt for this subtask. This method lets the partition
- * discoverer update what partitions it has discovered so far.
- *
- * @return List of discovered new partitions that this subtask should subscribe to.
- */
- public List<KafkaTopicPartition> discoverPartitions() throws WakeupException, ClosedException {
- if (!closed && !wakeup) {
- try {
- List<KafkaTopicPartition> newDiscoveredPartitions;
-
- // (1) get all possible partitions, based on whether we are subscribed to fixed
- // topics or a topic pattern
- if (topicsDescriptor.isFixedTopics()) {
- newDiscoveredPartitions =
- getAllPartitionsForTopics(topicsDescriptor.getFixedTopics());
- } else {
- List<String> matchedTopics = getAllTopics();
-
- // retain topics that match the pattern
- Iterator<String> iter = matchedTopics.iterator();
- while (iter.hasNext()) {
- if (!topicsDescriptor.isMatchingTopic(iter.next())) {
- iter.remove();
- }
- }
-
- if (matchedTopics.size() != 0) {
- // get partitions only for matched topics
- newDiscoveredPartitions = getAllPartitionsForTopics(matchedTopics);
- } else {
- newDiscoveredPartitions = null;
- }
- }
-
- // (2) eliminate partition that are old partitions or should not be subscribed by
- // this subtask
- if (newDiscoveredPartitions == null || newDiscoveredPartitions.isEmpty()) {
- throw new RuntimeException(
- "Unable to retrieve any partitions with KafkaTopicsDescriptor: "
- + topicsDescriptor);
- } else {
- Iterator<KafkaTopicPartition> iter = newDiscoveredPartitions.iterator();
- KafkaTopicPartition nextPartition;
- while (iter.hasNext()) {
- nextPartition = iter.next();
- if (!setAndCheckDiscoveredPartition(nextPartition)) {
- iter.remove();
- }
- }
- }
-
- return newDiscoveredPartitions;
- } catch (WakeupException e) {
- // the actual topic / partition metadata fetching methods
- // may be woken up midway; reset the wakeup flag and rethrow
- wakeup = false;
- throw e;
- }
- } else if (!closed && wakeup) {
- // may have been woken up before the method call
- wakeup = false;
- throw new WakeupException();
- } else {
- throw new ClosedException();
- }
- }
-
- /**
- * Sets a partition as discovered. Partitions are considered as new if its partition id is
- * larger than all partition ids previously seen for the topic it belongs to. Therefore, for a
- * set of discovered partitions, the order that this method is invoked with each partition is
- * important.
- *
- * <p>If the partition is indeed newly discovered, this method also returns whether the new
- * partition should be subscribed by this subtask.
- *
- * @param partition the partition to set and check
- * @return {@code true}, if the partition wasn't seen before and should be subscribed by this
- * subtask; {@code false} otherwise
- */
- public boolean setAndCheckDiscoveredPartition(KafkaTopicPartition partition) {
- if (isUndiscoveredPartition(partition)) {
- discoveredPartitions.add(partition);
-
- return KafkaTopicPartitionAssigner.assign(partition, numParallelSubtasks)
- == indexOfThisSubtask;
- }
-
- return false;
- }
-
- // ------------------------------------------------------------------------
- // Kafka version specifics
- // ------------------------------------------------------------------------
-
- /** Establish the required connections in order to fetch topics and partitions metadata. */
- protected abstract void initializeConnections() throws Exception;
-
- /**
- * Attempt to eagerly wakeup from blocking calls to Kafka in {@link
- * AbstractPartitionDiscoverer#getAllTopics()} and {@link
- * AbstractPartitionDiscoverer#getAllPartitionsForTopics(List)}.
- *
- * <p>If the invocation indeed results in interrupting an actual blocking Kafka call, the
- * implementations of {@link AbstractPartitionDiscoverer#getAllTopics()} and {@link
- * AbstractPartitionDiscoverer#getAllPartitionsForTopics(List)} are responsible of throwing a
- * {@link WakeupException}.
- */
- protected abstract void wakeupConnections();
-
- /** Close all established connections. */
- protected abstract void closeConnections() throws Exception;
-
- /** Fetch the list of all topics from Kafka. */
- protected abstract List<String> getAllTopics() throws WakeupException;
-
- /** Fetch the list of all partitions for a specific topics list from Kafka. */
- protected abstract List<KafkaTopicPartition> getAllPartitionsForTopics(List<String> topics)
- throws WakeupException;
-
- // ------------------------------------------------------------------------
- // Utilities
- // ------------------------------------------------------------------------
-
- /** Signaling exception to indicate that an actual Kafka call was interrupted. */
- public static final class WakeupException extends Exception {
- private static final long serialVersionUID = 1L;
- }
-
- /** Thrown if this discoverer was used to discover partitions after it was closed. */
- public static final class ClosedException extends Exception {
- private static final long serialVersionUID = 1L;
- }
-
- private boolean isUndiscoveredPartition(KafkaTopicPartition partition) {
- return !discoveredPartitions.contains(partition);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
deleted file mode 100644
index 2280061..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java
+++ /dev/null
@@ -1,500 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * A special form of blocking queue with two additions:
- *
- * <ol>
- * <li>The queue can be closed atomically when empty. Adding elements after the queue is closed
- * fails. This allows queue consumers to atomically discover that no elements are available
- * and mark themselves as shut down.
- * <li>The queue allows to poll batches of elements in one polling call.
- * </ol>
- *
- * <p>The queue has no capacity restriction and is safe for multiple producers and consumers.
- *
- * <p>Note: Null elements are prohibited.
- *
- * @param <E> The type of elements in the queue.
- */
-@Internal
-public class ClosableBlockingQueue<E> {
-
- /** The lock used to make queue accesses and open checks atomic. */
- private final ReentrantLock lock;
-
- /** The condition on which blocking get-calls wait if the queue is empty. */
- private final Condition nonEmpty;
-
- /** The deque of elements. */
- private final ArrayDeque<E> elements;
-
- /** Flag marking the status of the queue. */
- private volatile boolean open;
-
- // ------------------------------------------------------------------------
-
- /** Creates a new empty queue. */
- public ClosableBlockingQueue() {
- this(10);
- }
-
- /**
- * Creates a new empty queue, reserving space for at least the specified number of elements. The
- * queue can still grow, of more elements are added than the reserved space.
- *
- * @param initialSize The number of elements to reserve space for.
- */
- public ClosableBlockingQueue(int initialSize) {
- this.lock = new ReentrantLock(true);
- this.nonEmpty = this.lock.newCondition();
-
- this.elements = new ArrayDeque<>(initialSize);
- this.open = true;
- }
-
- /**
- * Creates a new queue that contains the given elements.
- *
- * @param initialElements The elements to initially add to the queue.
- */
- public ClosableBlockingQueue(Collection<? extends E> initialElements) {
- this(initialElements.size());
- this.elements.addAll(initialElements);
- }
-
- // ------------------------------------------------------------------------
- // Size and status
- // ------------------------------------------------------------------------
-
- /**
- * Gets the number of elements currently in the queue.
- *
- * @return The number of elements currently in the queue.
- */
- public int size() {
- lock.lock();
- try {
- return elements.size();
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Checks whether the queue is empty (has no elements).
- *
- * @return True, if the queue is empty; false, if it is non-empty.
- */
- public boolean isEmpty() {
- return size() == 0;
- }
-
- /**
- * Checks whether the queue is currently open, meaning elements can be added and polled.
- *
- * @return True, if the queue is open; false, if it is closed.
- */
- public boolean isOpen() {
- return open;
- }
-
- /**
- * Tries to close the queue. Closing the queue only succeeds when no elements are in the queue
- * when this method is called. Checking whether the queue is empty, and marking the queue as
- * closed is one atomic operation.
- *
- * @return True, if the queue is closed, false if the queue remains open.
- */
- public boolean close() {
- lock.lock();
- try {
- if (open) {
- if (elements.isEmpty()) {
- open = false;
- nonEmpty.signalAll();
- return true;
- } else {
- return false;
- }
- } else {
- // already closed
- return true;
- }
- } finally {
- lock.unlock();
- }
- }
-
- // ------------------------------------------------------------------------
- // Adding / Removing elements
- // ------------------------------------------------------------------------
-
- /**
- * Tries to add an element to the queue, if the queue is still open. Checking whether the queue
- * is open and adding the element is one atomic operation.
- *
- * <p>Unlike the {@link #add(Object)} method, this method never throws an exception, but only
- * indicates via the return code if the element was added or the queue was closed.
- *
- * @param element The element to add.
- * @return True, if the element was added, false if the queue was closes.
- */
- public boolean addIfOpen(E element) {
- requireNonNull(element);
-
- lock.lock();
- try {
- if (open) {
- elements.addLast(element);
- if (elements.size() == 1) {
- nonEmpty.signalAll();
- }
- }
- return open;
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Adds the element to the queue, or fails with an exception, if the queue is closed. Checking
- * whether the queue is open and adding the element is one atomic operation.
- *
- * @param element The element to add.
- * @throws IllegalStateException Thrown, if the queue is closed.
- */
- public void add(E element) throws IllegalStateException {
- requireNonNull(element);
-
- lock.lock();
- try {
- if (open) {
- elements.addLast(element);
- if (elements.size() == 1) {
- nonEmpty.signalAll();
- }
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Returns the queue's next element without removing it, if the queue is non-empty. Otherwise,
- * returns null.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and getting the next element is one atomic operation.
- *
- * <p>This method never blocks.
- *
- * @return The queue's next element, or null, if the queue is empty.
- * @throws IllegalStateException Thrown, if the queue is closed.
- */
- public E peek() {
- lock.lock();
- try {
- if (open) {
- if (elements.size() > 0) {
- return elements.getFirst();
- } else {
- return null;
- }
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Returns the queue's next element and removes it, the queue is non-empty. Otherwise, this
- * method returns null.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the next element is one atomic operation.
- *
- * <p>This method never blocks.
- *
- * @return The queue's next element, or null, if the queue is empty.
- * @throws IllegalStateException Thrown, if the queue is closed.
- */
- public E poll() {
- lock.lock();
- try {
- if (open) {
- if (elements.size() > 0) {
- return elements.removeFirst();
- } else {
- return null;
- }
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Returns all of the queue's current elements in a list, if the queue is non-empty. Otherwise,
- * this method returns null.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the elements is one atomic operation.
- *
- * <p>This method never blocks.
- *
- * @return All of the queue's elements, or null, if the queue is empty.
- * @throws IllegalStateException Thrown, if the queue is closed.
- */
- public List<E> pollBatch() {
- lock.lock();
- try {
- if (open) {
- if (elements.size() > 0) {
- ArrayList<E> result = new ArrayList<>(elements);
- elements.clear();
- return result;
- } else {
- return null;
- }
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Returns the next element in the queue. If the queue is empty, this method waits until at
- * least one element is added.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the next element is one atomic operation.
- *
- * @return The next element in the queue, never null.
- * @throws IllegalStateException Thrown, if the queue is closed.
- * @throws InterruptedException Throw, if the thread is interrupted while waiting for an element
- * to be added.
- */
- public E getElementBlocking() throws InterruptedException {
- lock.lock();
- try {
- while (open && elements.isEmpty()) {
- nonEmpty.await();
- }
-
- if (open) {
- return elements.removeFirst();
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Returns the next element in the queue. If the queue is empty, this method waits at most a
- * certain time until an element becomes available. If no element is available after that time,
- * the method returns null.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the next element is one atomic operation.
- *
- * @param timeoutMillis The number of milliseconds to block, at most.
- * @return The next element in the queue, or null, if the timeout expires before an element is
- * available.
- * @throws IllegalStateException Thrown, if the queue is closed.
- * @throws InterruptedException Throw, if the thread is interrupted while waiting for an element
- * to be added.
- */
- public E getElementBlocking(long timeoutMillis) throws InterruptedException {
- if (timeoutMillis == 0L) {
- // wait forever case
- return getElementBlocking();
- } else if (timeoutMillis < 0L) {
- throw new IllegalArgumentException("invalid timeout");
- }
-
- final long deadline = System.nanoTime() + timeoutMillis * 1_000_000L;
-
- lock.lock();
- try {
- while (open && elements.isEmpty() && timeoutMillis > 0) {
- nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
- timeoutMillis = (deadline - System.nanoTime()) / 1_000_000L;
- }
-
- if (!open) {
- throw new IllegalStateException("queue is closed");
- } else if (elements.isEmpty()) {
- return null;
- } else {
- return elements.removeFirst();
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Gets all the elements found in the list, or blocks until at least one element was added. If
- * the queue is empty when this method is called, it blocks until at least one element is added.
- *
- * <p>This method always returns a list with at least one element.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the next element is one atomic operation.
- *
- * @return A list with all elements in the queue, always at least one element.
- * @throws IllegalStateException Thrown, if the queue is closed.
- * @throws InterruptedException Throw, if the thread is interrupted while waiting for an element
- * to be added.
- */
- public List<E> getBatchBlocking() throws InterruptedException {
- lock.lock();
- try {
- while (open && elements.isEmpty()) {
- nonEmpty.await();
- }
- if (open) {
- ArrayList<E> result = new ArrayList<>(elements);
- elements.clear();
- return result;
- } else {
- throw new IllegalStateException("queue is closed");
- }
- } finally {
- lock.unlock();
- }
- }
-
- /**
- * Gets all the elements found in the list, or blocks until at least one element was added. This
- * method is similar as {@link #getBatchBlocking()}, but takes a number of milliseconds that the
- * method will maximally wait before returning.
- *
- * <p>This method never returns null, but an empty list, if the queue is empty when the method
- * is called and the request times out before an element was added.
- *
- * <p>The method throws an {@code IllegalStateException} if the queue is closed. Checking
- * whether the queue is open and removing the next element is one atomic operation.
- *
- * @param timeoutMillis The number of milliseconds to wait, at most.
- * @return A list with all elements in the queue, possible an empty list.
- * @throws IllegalStateException Thrown, if the queue is closed.
- * @throws InterruptedException Throw, if the thread is interrupted while waiting for an element
- * to be added.
- */
- public List<E> getBatchBlocking(long timeoutMillis) throws InterruptedException {
- if (timeoutMillis == 0L) {
- // wait forever case
- return getBatchBlocking();
- } else if (timeoutMillis < 0L) {
- throw new IllegalArgumentException("invalid timeout");
- }
-
- final long deadline = System.nanoTime() + timeoutMillis * 1_000_000L;
-
- lock.lock();
- try {
- while (open && elements.isEmpty() && timeoutMillis > 0) {
- nonEmpty.await(timeoutMillis, TimeUnit.MILLISECONDS);
- timeoutMillis = (deadline - System.nanoTime()) / 1_000_000L;
- }
-
- if (!open) {
- throw new IllegalStateException("queue is closed");
- } else if (elements.isEmpty()) {
- return Collections.emptyList();
- } else {
- ArrayList<E> result = new ArrayList<>(elements);
- elements.clear();
- return result;
- }
- } finally {
- lock.unlock();
- }
- }
-
- // ------------------------------------------------------------------------
- // Standard Utilities
- // ------------------------------------------------------------------------
-
- @Override
- public int hashCode() {
- int hashCode = 17;
- for (E element : elements) {
- hashCode = 31 * hashCode + element.hashCode();
- }
- return hashCode;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == this) {
- return true;
- } else if (obj != null && obj.getClass() == ClosableBlockingQueue.class) {
- @SuppressWarnings("unchecked")
- ClosableBlockingQueue<E> that = (ClosableBlockingQueue<E>) obj;
-
- if (this.elements.size() == that.elements.size()) {
- Iterator<E> thisElements = this.elements.iterator();
- for (E thatNext : that.elements) {
- E thisNext = thisElements.next();
- if (!(thisNext == null ? thatNext == null : thisNext.equals(thatNext))) {
- return false;
- }
- }
- return true;
- } else {
- return false;
- }
- } else {
- return false;
- }
- }
-
- @Override
- public String toString() {
- return elements.toString();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
deleted file mode 100644
index 204a01b..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import javax.annotation.Nullable;
-
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * A proxy that communicates exceptions between threads. Typically used if an exception from a
- * spawned thread needs to be recognized by the "parent" (spawner) thread.
- *
- * <p>The spawned thread would set the exception via {@link #reportError(Throwable)}. The parent
- * would check (at certain points) for exceptions via {@link #checkAndThrowException()}. Optionally,
- * the parent can pass itself in the constructor to be interrupted as soon as an exception occurs.
- *
- * <pre>{@code
- * final ExceptionProxy errorProxy = new ExceptionProxy(Thread.currentThread());
- *
- * Thread subThread = new Thread() {
- *
- * public void run() {
- * try {
- * doSomething();
- * } catch (Throwable t) {
- * errorProxy.reportError(
- * } finally {
- * doSomeCleanup();
- * }
- * }
- * };
- * subThread.start();
- *
- * doSomethingElse();
- * errorProxy.checkAndThrowException();
- *
- * doSomethingMore();
- * errorProxy.checkAndThrowException();
- *
- * try {
- * subThread.join();
- * } catch (InterruptedException e) {
- * errorProxy.checkAndThrowException();
- * // restore interrupted status, if not caused by an exception
- * Thread.currentThread().interrupt();
- * }
- * }</pre>
- */
-@Internal
-public class ExceptionProxy {
-
- /** The thread that should be interrupted when an exception occurs. */
- private final Thread toInterrupt;
-
- /** The exception to throw. */
- private final AtomicReference<Throwable> exception;
-
- /**
- * Creates an exception proxy that interrupts the given thread upon report of an exception. The
- * thread to interrupt may be null.
- *
- * @param toInterrupt The thread to interrupt upon an exception. May be null.
- */
- public ExceptionProxy(@Nullable Thread toInterrupt) {
- this.toInterrupt = toInterrupt;
- this.exception = new AtomicReference<>();
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * Sets the exception and interrupts the target thread, if no other exception has occurred so
- * far.
- *
- * <p>The exception is only set (and the interruption is only triggered), if no other exception
- * was set before.
- *
- * @param t The exception that occurred
- */
- public void reportError(Throwable t) {
- // set the exception, if it is the first (and the exception is non null)
- if (t != null && exception.compareAndSet(null, t) && toInterrupt != null) {
- toInterrupt.interrupt();
- }
- }
-
- /**
- * Checks whether an exception has been set via {@link #reportError(Throwable)}. If yes, that
- * exception if re-thrown by this method.
- *
- * @throws Exception This method re-throws the exception, if set.
- */
- public void checkAndThrowException() throws Exception {
- Throwable t = exception.get();
- if (t != null) {
- if (t instanceof Exception) {
- throw (Exception) t;
- } else if (t instanceof Error) {
- throw (Error) t;
- } else {
- throw new Exception(t);
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java
deleted file mode 100644
index a424a81..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java
+++ /dev/null
@@ -1,421 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava30.com.google.common.base.Joiner;
-
-import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.clients.producer.internals.TransactionManager;
-import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.requests.FindCoordinatorRequest;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.time.Duration;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.Future;
-
-/** Internal flink kafka producer. */
-@PublicEvolving
-public class FlinkKafkaInternalProducer<K, V> implements Producer<K, V> {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class);
-
- protected final KafkaProducer<K, V> kafkaProducer;
-
- // This lock and closed flag are introduced to workaround KAFKA-6635. Because the bug is only
- // fixed in
- // Kafka 2.3.0, we need this workaround before Kafka dependency is bumped to 2.3.0 to avoid
- // deadlock
- // between a transaction committing / aborting thread and a producer closing thread.
- // TODO: remove the workaround after Kafka dependency is bumped to 2.3.0+
- private final Object producerClosingLock;
- private volatile boolean closed;
-
- @Nullable protected final String transactionalId;
-
- public FlinkKafkaInternalProducer(Properties properties) {
- transactionalId = properties.getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG);
- kafkaProducer = new KafkaProducer<>(properties);
- producerClosingLock = new Object();
- closed = false;
- }
-
- // -------------------------------- Simple proxy method calls --------------------------------
-
- @Override
- public void initTransactions() {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- kafkaProducer.initTransactions();
- }
- }
-
- @Override
- public void beginTransaction() throws ProducerFencedException {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- kafkaProducer.beginTransaction();
- }
- }
-
- @Override
- public void commitTransaction() throws ProducerFencedException {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- kafkaProducer.commitTransaction();
- }
- }
-
- @Override
- public void abortTransaction() throws ProducerFencedException {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- kafkaProducer.abortTransaction();
- }
- }
-
- @Override
- public void sendOffsetsToTransaction(
- Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId)
- throws ProducerFencedException {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- kafkaProducer.sendOffsetsToTransaction(offsets, consumerGroupId);
- }
- }
-
- @Override
- public void sendOffsetsToTransaction(
- Map<TopicPartition, OffsetAndMetadata> map, ConsumerGroupMetadata consumerGroupMetadata)
- throws ProducerFencedException {
- kafkaProducer.sendOffsetsToTransaction(map, consumerGroupMetadata);
- }
-
- @Override
- public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
- return kafkaProducer.send(record);
- }
-
- @Override
- public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
- return kafkaProducer.send(record, callback);
- }
-
- @Override
- public List<PartitionInfo> partitionsFor(String topic) {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- return kafkaProducer.partitionsFor(topic);
- }
- }
-
- @Override
- public Map<MetricName, ? extends Metric> metrics() {
- return kafkaProducer.metrics();
- }
-
- @Override
- public void close() {
- throw new UnsupportedOperationException(
- "Close without timeout is now allowed because it can leave lingering Kafka threads.");
- }
-
- @Override
- public void close(Duration duration) {
- synchronized (producerClosingLock) {
- kafkaProducer.close(duration);
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "Closed internal KafkaProducer {}. Stacktrace: {}",
- System.identityHashCode(this),
- Joiner.on("\n").join(Thread.currentThread().getStackTrace()));
- }
- closed = true;
- }
- }
-
- // -------------------------------- New methods or methods with changed behaviour
- // --------------------------------
-
- @Override
- public void flush() {
- kafkaProducer.flush();
- if (transactionalId != null) {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- flushNewPartitions();
- }
- }
- }
-
- /**
- * Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously
- * obtained ones, so that we can resume transaction after a restart. Implementation of this
- * method is based on {@link KafkaProducer#initTransactions}.
- * https://github.com/apache/kafka/commit/5d2422258cb975a137a42a4e08f03573c49a387e#diff-f4ef1afd8792cd2a2e9069cd7ddea630
- */
- public void resumeTransaction(long producerId, short epoch) {
- synchronized (producerClosingLock) {
- ensureNotClosed();
- Preconditions.checkState(
- producerId >= 0 && epoch >= 0,
- "Incorrect values for producerId %s and epoch %s",
- producerId,
- epoch);
- LOG.info(
- "Attempting to resume transaction {} with producerId {} and epoch {}",
- transactionalId,
- producerId,
- epoch);
-
- Object transactionManager = getField(kafkaProducer, "transactionManager");
- synchronized (transactionManager) {
- Object topicPartitionBookkeeper =
- getField(transactionManager, "topicPartitionBookkeeper");
-
- invoke(
- transactionManager,
- "transitionTo",
- getEnum(
- "org.apache.kafka.clients.producer.internals.TransactionManager$State.INITIALIZING"));
- invoke(topicPartitionBookkeeper, "reset");
-
- setField(
- transactionManager,
- "producerIdAndEpoch",
- createProducerIdAndEpoch(producerId, epoch));
-
- invoke(
- transactionManager,
- "transitionTo",
- getEnum(
- "org.apache.kafka.clients.producer.internals.TransactionManager$State.READY"));
-
- invoke(
- transactionManager,
- "transitionTo",
- getEnum(
- "org.apache.kafka.clients.producer.internals.TransactionManager$State.IN_TRANSACTION"));
- setField(transactionManager, "transactionStarted", true);
- }
- }
- }
-
- public String getTransactionalId() {
- return transactionalId;
- }
-
- public long getProducerId() {
- Object transactionManager = getField(kafkaProducer, "transactionManager");
- Object producerIdAndEpoch = getField(transactionManager, "producerIdAndEpoch");
- return (long) getField(producerIdAndEpoch, "producerId");
- }
-
- public short getEpoch() {
- Object transactionManager = getField(kafkaProducer, "transactionManager");
- Object producerIdAndEpoch = getField(transactionManager, "producerIdAndEpoch");
- return (short) getField(producerIdAndEpoch, "epoch");
- }
-
- @VisibleForTesting
- public int getTransactionCoordinatorId() {
- Object transactionManager = getField(kafkaProducer, "transactionManager");
- Node node =
- (Node)
- invoke(
- transactionManager,
- "coordinator",
- FindCoordinatorRequest.CoordinatorType.TRANSACTION);
- return node.id();
- }
-
- private void ensureNotClosed() {
- if (closed) {
- throw new IllegalStateException(
- String.format(
- "The producer %s has already been closed",
- System.identityHashCode(this)));
- }
- }
-
- private Object createProducerIdAndEpoch(long producerId, short epoch) {
- try {
- Field field = TransactionManager.class.getDeclaredField("producerIdAndEpoch");
- Class<?> clazz = field.getType();
- Constructor<?> constructor = clazz.getDeclaredConstructor(Long.TYPE, Short.TYPE);
- constructor.setAccessible(true);
- return constructor.newInstance(producerId, epoch);
- } catch (InvocationTargetException
- | InstantiationException
- | IllegalAccessException
- | NoSuchFieldException
- | NoSuchMethodException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- /**
- * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction}
- * is also adding new partitions to the transaction. flushNewPartitions method is moving this
- * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise resumeTransaction
- * would require to restore state of the not yet added/"in-flight" partitions.
- */
- private void flushNewPartitions() {
- LOG.info("Flushing new partitions");
- TransactionalRequestResult result = enqueueNewPartitions();
- Object sender = getField(kafkaProducer, "sender");
- invoke(sender, "wakeup");
- result.await();
- }
-
- /**
- * Enqueues new transactions at the transaction manager and returns a {@link
- * TransactionalRequestResult} that allows waiting on them.
- *
- * <p>If there are no new transactions we return a {@link TransactionalRequestResult} that is
- * already done.
- */
- private TransactionalRequestResult enqueueNewPartitions() {
- Object transactionManager = getField(kafkaProducer, "transactionManager");
- synchronized (transactionManager) {
- Object newPartitionsInTransaction =
- getField(transactionManager, "newPartitionsInTransaction");
- Object newPartitionsInTransactionIsEmpty =
- invoke(newPartitionsInTransaction, "isEmpty");
- TransactionalRequestResult result;
- if (newPartitionsInTransactionIsEmpty instanceof Boolean
- && !((Boolean) newPartitionsInTransactionIsEmpty)) {
- Object txnRequestHandler =
- invoke(transactionManager, "addPartitionsToTransactionHandler");
- invoke(
- transactionManager,
- "enqueueRequest",
- new Class[] {txnRequestHandler.getClass().getSuperclass()},
- new Object[] {txnRequestHandler});
- result =
- (TransactionalRequestResult)
- getField(
- txnRequestHandler,
- txnRequestHandler.getClass().getSuperclass(),
- "result");
- } else {
- // we don't have an operation but this operation string is also used in
- // addPartitionsToTransactionHandler.
- result = new TransactionalRequestResult("AddPartitionsToTxn");
- result.done();
- }
- return result;
- }
- }
-
- protected static Enum<?> getEnum(String enumFullName) {
- String[] x = enumFullName.split("\\.(?=[^\\.]+$)");
- if (x.length == 2) {
- String enumClassName = x[0];
- String enumName = x[1];
- try {
- Class<Enum> cl = (Class<Enum>) Class.forName(enumClassName);
- return Enum.valueOf(cl, enumName);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
- return null;
- }
-
- protected static Object invoke(Object object, String methodName, Object... args) {
- Class<?>[] argTypes = new Class[args.length];
- for (int i = 0; i < args.length; i++) {
- argTypes[i] = args[i].getClass();
- }
- return invoke(object, methodName, argTypes, args);
- }
-
- private static Object invoke(
- Object object, String methodName, Class<?>[] argTypes, Object[] args) {
- try {
- Method method = object.getClass().getDeclaredMethod(methodName, argTypes);
- method.setAccessible(true);
- return method.invoke(object, args);
- } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- /**
- * Gets and returns the field {@code fieldName} from the given Object {@code object} using
- * reflection.
- */
- protected static Object getField(Object object, String fieldName) {
- return getField(object, object.getClass(), fieldName);
- }
-
- /**
- * Gets and returns the field {@code fieldName} from the given Object {@code object} using
- * reflection.
- */
- private static Object getField(Object object, Class<?> clazz, String fieldName) {
- try {
- Field field = clazz.getDeclaredField(fieldName);
- field.setAccessible(true);
- return field.get(object);
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-
- /**
- * Sets the field {@code fieldName} on the given Object {@code object} to {@code value} using
- * reflection.
- */
- protected static void setField(Object object, String fieldName, Object value) {
- try {
- Field field = object.getClass().getDeclaredField(fieldName);
- field.setAccessible(true);
- field.set(object, value);
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Incompatible KafkaProducer version", e);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java
deleted file mode 100644
index 7fc50e6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-
-import javax.annotation.Nonnull;
-import javax.annotation.concurrent.ThreadSafe;
-
-import java.io.Closeable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The Handover is a utility to hand over data (a buffer of records) and exception from a
- * <i>producer</i> thread to a <i>consumer</i> thread. It effectively behaves like a "size one
- * blocking queue", with some extras around exception reporting, closing, and waking up thread
- * without {@link Thread#interrupt() interrupting} threads.
- *
- * <p>This class is used in the Flink Kafka Consumer to hand over data and exceptions between the
- * thread that runs the KafkaConsumer class and the main thread.
- *
- * <p>The Handover has the notion of "waking up" the producer thread with a {@link WakeupException}
- * rather than a thread interrupt.
- *
- * <p>The Handover can also be "closed", signalling from one thread to the other that it the thread
- * has terminated.
- */
-@ThreadSafe
-@Internal
-public final class Handover implements Closeable {
-
- private final Object lock = new Object();
-
- private ConsumerRecords<byte[], byte[]> next;
- private Throwable error;
- private boolean wakeupProducer;
-
- /**
- * Polls the next element from the Handover, possibly blocking until the next element is
- * available. This method behaves similar to polling from a blocking queue.
- *
- * <p>If an exception was handed in by the producer ({@link #reportError(Throwable)}), then that
- * exception is thrown rather than an element being returned.
- *
- * @return The next element (buffer of records, never null).
- * @throws ClosedException Thrown if the Handover was {@link #close() closed}.
- * @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method.
- */
- @Nonnull
- public ConsumerRecords<byte[], byte[]> pollNext() throws Exception {
- synchronized (lock) {
- while (next == null && error == null) {
- lock.wait();
- }
-
- ConsumerRecords<byte[], byte[]> n = next;
- if (n != null) {
- next = null;
- lock.notifyAll();
- return n;
- } else {
- ExceptionUtils.rethrowException(error, error.getMessage());
-
- // this statement cannot be reached since the above method always throws an
- // exception
- // this is only here to silence the compiler and any warnings
- return ConsumerRecords.empty();
- }
- }
- }
-
- /**
- * Hands over an element from the producer. If the Handover already has an element that was not
- * yet picked up by the consumer thread, this call blocks until the consumer picks up that
- * previous element.
- *
- * <p>This behavior is similar to a "size one" blocking queue.
- *
- * @param element The next element to hand over.
- * @throws InterruptedException Thrown, if the thread is interrupted while blocking for the
- * Handover to be empty.
- * @throws WakeupException Thrown, if the {@link #wakeupProducer()} method is called while
- * blocking for the Handover to be empty.
- * @throws ClosedException Thrown if the Handover was closed or concurrently being closed.
- */
- public void produce(final ConsumerRecords<byte[], byte[]> element)
- throws InterruptedException, WakeupException, ClosedException {
-
- checkNotNull(element);
-
- synchronized (lock) {
- while (next != null && !wakeupProducer) {
- lock.wait();
- }
-
- wakeupProducer = false;
-
- // if there is still an element, we must have been woken up
- if (next != null) {
- throw new WakeupException();
- }
- // if there is no error, then this is open and can accept this element
- else if (error == null) {
- next = element;
- lock.notifyAll();
- }
- // an error marks this as closed for the producer
- else {
- throw new ClosedException();
- }
- }
- }
-
- /**
- * Reports an exception. The consumer will throw the given exception immediately, if it is
- * currently blocked in the {@link #pollNext()} method, or the next time it calls that method.
- *
- * <p>After this method has been called, no call to either {@link #produce(ConsumerRecords)} or
- * {@link #pollNext()} will ever return regularly any more, but will always return
- * exceptionally.
- *
- * <p>If another exception was already reported, this method does nothing.
- *
- * <p>For the producer, the Handover will appear as if it was {@link #close() closed}.
- *
- * @param t The exception to report.
- */
- public void reportError(Throwable t) {
- checkNotNull(t);
-
- synchronized (lock) {
- // do not override the initial exception
- if (error == null) {
- error = t;
- }
- next = null;
- lock.notifyAll();
- }
- }
-
- /**
- * Closes the handover. Both the {@link #produce(ConsumerRecords)} method and the {@link
- * #pollNext()} will throw a {@link ClosedException} on any currently blocking and future
- * invocations.
- *
- * <p>If an exception was previously reported via the {@link #reportError(Throwable)} method,
- * that exception will not be overridden. The consumer thread will throw that exception upon
- * calling {@link #pollNext()}, rather than the {@code ClosedException}.
- */
- @Override
- public void close() {
- synchronized (lock) {
- next = null;
- wakeupProducer = false;
-
- if (error == null) {
- error = new ClosedException();
- }
- lock.notifyAll();
- }
- }
-
- /**
- * Wakes the producer thread up. If the producer thread is currently blocked in the {@link
- * #produce(ConsumerRecords)} method, it will exit the method throwing a {@link
- * WakeupException}.
- */
- public void wakeupProducer() {
- synchronized (lock) {
- wakeupProducer = true;
- lock.notifyAll();
- }
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * An exception thrown by the Handover in the {@link #pollNext()} or {@link
- * #produce(ConsumerRecords)} method, after the Handover was closed via {@link #close()}.
- */
- public static final class ClosedException extends Exception {
- private static final long serialVersionUID = 1L;
- }
-
- /**
- * A special exception thrown bv the Handover in the {@link #produce(ConsumerRecords)} method
- * when the producer is woken up from a blocking call via {@link #wakeupProducer()}.
- */
- public static final class WakeupException extends Exception {
- private static final long serialVersionUID = 1L;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java
deleted file mode 100644
index d766677..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * A callback interface that the source operator can implement to trigger custom actions when a
- * commit request completes, which should normally be triggered from checkpoint complete event.
- */
-@Internal
-public interface KafkaCommitCallback {
-
- /**
- * A callback method the user can implement to provide asynchronous handling of commit request
- * completion. This method will be called when the commit request sent to the server has been
- * acknowledged without error.
- */
- void onSuccess();
-
- /**
- * A callback method the user can implement to provide asynchronous handling of commit request
- * failure. This method will be called when the commit request failed.
- *
- * @param cause Kafka commit failure cause returned by kafka client
- */
- void onException(Throwable cause);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java
deleted file mode 100644
index f7f40b8..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java
+++ /dev/null
@@ -1,564 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
-
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.WakeupException;
-import org.slf4j.Logger;
-
-import javax.annotation.Nonnull;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The thread the runs the {@link KafkaConsumer}, connecting to the brokers and polling records. The
- * thread pushes the data into a {@link Handover} to be picked up by the fetcher that will
- * deserialize and emit the records.
- *
- * <p><b>IMPORTANT:</b> This thread must not be interrupted when attempting to shut it down. The
- * Kafka consumer code was found to not always handle interrupts well, and to even deadlock in
- * certain situations.
- *
- * <p>Implementation Note: This code is written to be reusable in later versions of the
- * KafkaConsumer. Because Kafka is not maintaining binary compatibility, we use a "call bridge" as
- * an indirection to the KafkaConsumer calls that change signature.
- */
-@Internal
-public class KafkaConsumerThread<T> extends Thread {
-
- /** Logger for this consumer. */
- private final Logger log;
-
- /** The handover of data and exceptions between the consumer thread and the task thread. */
- private final Handover handover;
-
- /** The next offsets that the main thread should commit and the commit callback. */
- private final AtomicReference<
- Tuple2<Map<TopicPartition, OffsetAndMetadata>, KafkaCommitCallback>>
- nextOffsetsToCommit;
-
- /** The configuration for the Kafka consumer. */
- private final Properties kafkaProperties;
-
- /** The queue of unassigned partitions that we need to assign to the Kafka consumer. */
- private final ClosableBlockingQueue<KafkaTopicPartitionState<T, TopicPartition>>
- unassignedPartitionsQueue;
-
- /** The maximum number of milliseconds to wait for a fetch batch. */
- private final long pollTimeout;
-
- /** Flag whether to add Kafka's metrics to the Flink metrics. */
- private final boolean useMetrics;
-
- /**
- * @deprecated We should only be publishing to the {{@link #consumerMetricGroup}}. This is kept
- * to retain compatibility for metrics.
- */
- @Deprecated private final MetricGroup subtaskMetricGroup;
-
- /** We get this from the outside to publish metrics. */
- private final MetricGroup consumerMetricGroup;
-
- /** Reference to the Kafka consumer, once it is created. */
- private volatile KafkaConsumer<byte[], byte[]> consumer;
-
- /** This lock is used to isolate the consumer for partition reassignment. */
- private final Object consumerReassignmentLock;
-
- /** Indication if this consumer has any assigned partition. */
- private boolean hasAssignedPartitions;
-
- /**
- * Flag to indicate whether an external operation ({@link #setOffsetsToCommit(Map,
- * KafkaCommitCallback)} or {@link #shutdown()}) had attempted to wakeup the consumer while it
- * was isolated for partition reassignment.
- */
- private volatile boolean hasBufferedWakeup;
-
- /** Flag to mark the main work loop as alive. */
- private volatile boolean running;
-
- /** Flag tracking whether the latest commit request has completed. */
- private volatile boolean commitInProgress;
-
- public KafkaConsumerThread(
- Logger log,
- Handover handover,
- Properties kafkaProperties,
- ClosableBlockingQueue<KafkaTopicPartitionState<T, TopicPartition>>
- unassignedPartitionsQueue,
- String threadName,
- long pollTimeout,
- boolean useMetrics,
- MetricGroup consumerMetricGroup,
- MetricGroup subtaskMetricGroup) {
-
- super(threadName);
- setDaemon(true);
-
- this.log = checkNotNull(log);
- this.handover = checkNotNull(handover);
- this.kafkaProperties = checkNotNull(kafkaProperties);
- this.consumerMetricGroup = checkNotNull(consumerMetricGroup);
- this.subtaskMetricGroup = checkNotNull(subtaskMetricGroup);
-
- this.unassignedPartitionsQueue = checkNotNull(unassignedPartitionsQueue);
-
- this.pollTimeout = pollTimeout;
- this.useMetrics = useMetrics;
-
- this.consumerReassignmentLock = new Object();
- this.nextOffsetsToCommit = new AtomicReference<>();
- this.running = true;
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public void run() {
- // early exit check
- if (!running) {
- return;
- }
-
- // this is the means to talk to FlinkKafkaConsumer's main thread
- final Handover handover = this.handover;
-
- // This method initializes the KafkaConsumer and guarantees it is torn down properly.
- // This is important, because the consumer has multi-threading issues,
- // including concurrent 'close()' calls.
- try {
- this.consumer = getConsumer(kafkaProperties);
- } catch (Throwable t) {
- handover.reportError(t);
- return;
- }
-
- // from here on, the consumer is guaranteed to be closed properly
- try {
- // register Kafka's very own metrics in Flink's metric reporters
- if (useMetrics) {
- // register Kafka metrics to Flink
- Map<MetricName, ? extends Metric> metrics = consumer.metrics();
- if (metrics == null) {
- // MapR's Kafka implementation returns null here.
- log.info("Consumer implementation does not support metrics");
- } else {
- // we have Kafka metrics, register them
- for (Map.Entry<MetricName, ? extends Metric> metric : metrics.entrySet()) {
- consumerMetricGroup.gauge(
- metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
-
- // TODO this metric is kept for compatibility purposes; should remove in the
- // future
- subtaskMetricGroup.gauge(
- metric.getKey().name(), new KafkaMetricWrapper(metric.getValue()));
- }
- }
- }
-
- // early exit check
- if (!running) {
- return;
- }
-
- // the latest bulk of records. May carry across the loop if the thread is woken up
- // from blocking on the handover
- ConsumerRecords<byte[], byte[]> records = null;
-
- // reused variable to hold found unassigned new partitions.
- // found partitions are not carried across loops using this variable;
- // they are carried across via re-adding them to the unassigned partitions queue
- List<KafkaTopicPartitionState<T, TopicPartition>> newPartitions;
-
- // main fetch loop
- while (running) {
-
- // check if there is something to commit
- if (!commitInProgress) {
- // get and reset the work-to-be committed, so we don't repeatedly commit the
- // same
- final Tuple2<Map<TopicPartition, OffsetAndMetadata>, KafkaCommitCallback>
- commitOffsetsAndCallback = nextOffsetsToCommit.getAndSet(null);
-
- if (commitOffsetsAndCallback != null) {
- log.debug("Sending async offset commit request to Kafka broker");
-
- // also record that a commit is already in progress
- // the order here matters! first set the flag, then send the commit command.
- commitInProgress = true;
- retryOnceOnWakeup(
- () ->
- consumer.commitAsync(
- commitOffsetsAndCallback.f0,
- new CommitCallback(commitOffsetsAndCallback.f1)),
- "commitAsync");
- }
- }
-
- try {
- if (hasAssignedPartitions) {
- newPartitions = unassignedPartitionsQueue.pollBatch();
- } else {
- // if no assigned partitions block until we get at least one
- // instead of hot spinning this loop. We rely on a fact that
- // unassignedPartitionsQueue will be closed on a shutdown, so
- // we don't block indefinitely
- newPartitions = unassignedPartitionsQueue.getBatchBlocking();
- }
- if (newPartitions != null) {
- reassignPartitions(newPartitions);
- }
- } catch (AbortedReassignmentException e) {
- continue;
- }
-
- if (!hasAssignedPartitions) {
- // Without assigned partitions KafkaConsumer.poll will throw an exception
- continue;
- }
-
- // get the next batch of records, unless we did not manage to hand the old batch
- // over
- if (records == null) {
- try {
- records = consumer.poll(Duration.ofMillis(pollTimeout));
- } catch (WakeupException we) {
- continue;
- }
- }
-
- try {
- handover.produce(records);
- records = null;
- } catch (Handover.WakeupException e) {
- // fall through the loop
- }
- }
- // end main fetch loop
- } catch (Throwable t) {
- // let the main thread know and exit
- // it may be that this exception comes because the main thread closed the handover, in
- // which case the below reporting is irrelevant, but does not hurt either
- handover.reportError(t);
- } finally {
- // make sure the handover is closed if it is not already closed or has an error
- handover.close();
-
- // make sure the KafkaConsumer is closed
- try {
- consumer.close();
- } catch (Throwable t) {
- log.warn("Error while closing Kafka consumer", t);
- }
- }
- }
-
- /**
- * Shuts this thread down, waking up the thread gracefully if blocked (without
- * Thread.interrupt() calls).
- */
- public void shutdown() {
- running = false;
-
- // wake up all blocking calls on the queue
- unassignedPartitionsQueue.close();
-
- // We cannot call close() on the KafkaConsumer, because it will actually throw
- // an exception if a concurrent call is in progress
-
- // this wakes up the consumer if it is blocked handing over records
- handover.wakeupProducer();
-
- // this wakes up the consumer if it is blocked in a kafka poll
- synchronized (consumerReassignmentLock) {
- if (consumer != null) {
- consumer.wakeup();
- } else {
- // the consumer is currently isolated for partition reassignment;
- // set this flag so that the wakeup state is restored once the reassignment is
- // complete
- hasBufferedWakeup = true;
- }
- }
- }
-
- /**
- * Tells this thread to commit a set of offsets. This method does not block, the committing
- * operation will happen asynchronously.
- *
- * <p>Only one commit operation may be pending at any time. If the committing takes longer than
- * the frequency with which this method is called, then some commits may be skipped due to being
- * superseded by newer ones.
- *
- * @param offsetsToCommit The offsets to commit
- * @param commitCallback callback when Kafka commit completes
- */
- void setOffsetsToCommit(
- Map<TopicPartition, OffsetAndMetadata> offsetsToCommit,
- @Nonnull KafkaCommitCallback commitCallback) {
-
- // record the work to be committed by the main consumer thread and make sure the consumer
- // notices that
- if (nextOffsetsToCommit.getAndSet(Tuple2.of(offsetsToCommit, commitCallback)) != null) {
- log.warn(
- "Committing offsets to Kafka takes longer than the checkpoint interval. "
- + "Skipping commit of previous offsets because newer complete checkpoint offsets are available. "
- + "This does not compromise Flink's checkpoint integrity.");
- }
-
- // if the consumer is blocked in a poll() or handover operation, wake it up to commit soon
- handover.wakeupProducer();
-
- synchronized (consumerReassignmentLock) {
- if (consumer != null) {
- consumer.wakeup();
- } else {
- // the consumer is currently isolated for partition reassignment;
- // set this flag so that the wakeup state is restored once the reassignment is
- // complete
- hasBufferedWakeup = true;
- }
- }
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * Reestablishes the assigned partitions for the consumer. The reassigned partitions consists of
- * the provided new partitions and whatever partitions was already previously assigned to the
- * consumer.
- *
- * <p>The reassignment process is protected against wakeup calls, so that after this method
- * returns, the consumer is either untouched or completely reassigned with the correct offset
- * positions.
- *
- * <p>If the consumer was already woken-up prior to a reassignment resulting in an interruption
- * any time during the reassignment, the consumer is guaranteed to roll back as if it was
- * untouched. On the other hand, if there was an attempt to wakeup the consumer during the
- * reassignment, the wakeup call is "buffered" until the reassignment completes.
- *
- * <p>This method is exposed for testing purposes.
- */
- @VisibleForTesting
- void reassignPartitions(List<KafkaTopicPartitionState<T, TopicPartition>> newPartitions)
- throws Exception {
- if (newPartitions.size() == 0) {
- return;
- }
- hasAssignedPartitions = true;
- boolean reassignmentStarted = false;
-
- // since the reassignment may introduce several Kafka blocking calls that cannot be
- // interrupted,
- // the consumer needs to be isolated from external wakeup calls in setOffsetsToCommit() and
- // shutdown()
- // until the reassignment is complete.
- final KafkaConsumer<byte[], byte[]> consumerTmp;
- synchronized (consumerReassignmentLock) {
- consumerTmp = this.consumer;
- this.consumer = null;
- }
-
- final Map<TopicPartition, Long> oldPartitionAssignmentsToPosition = new HashMap<>();
- try {
- for (TopicPartition oldPartition : consumerTmp.assignment()) {
- oldPartitionAssignmentsToPosition.put(
- oldPartition, consumerTmp.position(oldPartition));
- }
-
- final List<TopicPartition> newPartitionAssignments =
- new ArrayList<>(
- newPartitions.size() + oldPartitionAssignmentsToPosition.size());
- newPartitionAssignments.addAll(oldPartitionAssignmentsToPosition.keySet());
- newPartitionAssignments.addAll(convertKafkaPartitions(newPartitions));
-
- // reassign with the new partitions
- consumerTmp.assign(newPartitionAssignments);
- reassignmentStarted = true;
-
- // old partitions should be seeked to their previous position
- for (Map.Entry<TopicPartition, Long> oldPartitionToPosition :
- oldPartitionAssignmentsToPosition.entrySet()) {
- consumerTmp.seek(
- oldPartitionToPosition.getKey(), oldPartitionToPosition.getValue());
- }
-
- // offsets in the state of new partitions may still be placeholder sentinel values if we
- // are:
- // (1) starting fresh,
- // (2) checkpoint / savepoint state we were restored with had not completely
- // been replaced with actual offset values yet, or
- // (3) the partition was newly discovered after startup;
- // replace those with actual offsets, according to what the sentinel value represent.
- for (KafkaTopicPartitionState<T, TopicPartition> newPartitionState : newPartitions) {
- if (newPartitionState.getOffset()
- == KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET) {
- consumerTmp.seekToBeginning(
- Collections.singletonList(newPartitionState.getKafkaPartitionHandle()));
- newPartitionState.setOffset(
- consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1);
- } else if (newPartitionState.getOffset()
- == KafkaTopicPartitionStateSentinel.LATEST_OFFSET) {
- consumerTmp.seekToEnd(
- Collections.singletonList(newPartitionState.getKafkaPartitionHandle()));
- newPartitionState.setOffset(
- consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1);
- } else if (newPartitionState.getOffset()
- == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) {
- // the KafkaConsumer by default will automatically seek the consumer position
- // to the committed group offset, so we do not need to do it.
-
- newPartitionState.setOffset(
- consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1);
- } else {
- consumerTmp.seek(
- newPartitionState.getKafkaPartitionHandle(),
- newPartitionState.getOffset() + 1);
- }
- }
- } catch (WakeupException e) {
- // a WakeupException may be thrown if the consumer was invoked wakeup()
- // before it was isolated for the reassignment. In this case, we abort the
- // reassignment and just re-expose the original consumer.
-
- synchronized (consumerReassignmentLock) {
- this.consumer = consumerTmp;
-
- // if reassignment had already started and affected the consumer,
- // we do a full roll back so that it is as if it was left untouched
- if (reassignmentStarted) {
- this.consumer.assign(
- new ArrayList<>(oldPartitionAssignmentsToPosition.keySet()));
-
- for (Map.Entry<TopicPartition, Long> oldPartitionToPosition :
- oldPartitionAssignmentsToPosition.entrySet()) {
- this.consumer.seek(
- oldPartitionToPosition.getKey(), oldPartitionToPosition.getValue());
- }
- }
-
- // no need to restore the wakeup state in this case,
- // since only the last wakeup call is effective anyways
- hasBufferedWakeup = false;
-
- // re-add all new partitions back to the unassigned partitions queue to be picked up
- // again
- for (KafkaTopicPartitionState<T, TopicPartition> newPartition : newPartitions) {
- unassignedPartitionsQueue.add(newPartition);
- }
-
- // this signals the main fetch loop to continue through the loop
- throw new AbortedReassignmentException();
- }
- }
-
- // reassignment complete; expose the reassigned consumer
- synchronized (consumerReassignmentLock) {
- this.consumer = consumerTmp;
-
- // restore wakeup state for the consumer if necessary
- if (hasBufferedWakeup) {
- this.consumer.wakeup();
- hasBufferedWakeup = false;
- }
- }
- }
-
- @VisibleForTesting
- KafkaConsumer<byte[], byte[]> getConsumer(Properties kafkaProperties) {
- return new KafkaConsumer<>(kafkaProperties);
- }
-
- private void retryOnceOnWakeup(Runnable consumerCall, String description) {
- try {
- consumerCall.run();
- } catch (WakeupException we) {
- log.info(
- "Caught WakeupException while executing Kafka consumer call for {}. Will retry it once.",
- description);
- consumerCall.run();
- }
- }
-
- // ------------------------------------------------------------------------
- // Utilities
- // ------------------------------------------------------------------------
-
- private static <T> List<TopicPartition> convertKafkaPartitions(
- List<KafkaTopicPartitionState<T, TopicPartition>> partitions) {
- ArrayList<TopicPartition> result = new ArrayList<>(partitions.size());
- for (KafkaTopicPartitionState<T, TopicPartition> p : partitions) {
- result.add(p.getKafkaPartitionHandle());
- }
- return result;
- }
-
- private class CommitCallback implements OffsetCommitCallback {
-
- private final KafkaCommitCallback internalCommitCallback;
-
- CommitCallback(KafkaCommitCallback internalCommitCallback) {
- this.internalCommitCallback = checkNotNull(internalCommitCallback);
- }
-
- @Override
- public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception ex) {
- commitInProgress = false;
-
- if (ex != null) {
- log.warn(
- "Committing offsets to Kafka failed. This does not compromise Flink's checkpoints.",
- ex);
- internalCommitCallback.onException(ex);
- } else {
- internalCommitCallback.onSuccess();
- }
- }
- }
-
- /**
- * Utility exception that serves as a signal for the main loop to continue through the loop if a
- * reassignment attempt was aborted due to an pre-reassignment wakeup call on the consumer.
- */
- private static class AbortedReassignmentException extends Exception {
- private static final long serialVersionUID = 1L;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java
deleted file mode 100644
index d53e4ff..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.util.Collector;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-/**
- * A simple wrapper for using the DeserializationSchema with the KafkaDeserializationSchema
- * interface.
- *
- * @param <T> The type created by the deserialization schema.
- */
-@Internal
-public class KafkaDeserializationSchemaWrapper<T> implements KafkaDeserializationSchema<T> {
-
- private static final long serialVersionUID = 2651665280744549932L;
-
- private final DeserializationSchema<T> deserializationSchema;
-
- public KafkaDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
- this.deserializationSchema = deserializationSchema;
- }
-
- @Override
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- this.deserializationSchema.open(context);
- }
-
- @Override
- public T deserialize(ConsumerRecord<byte[], byte[]> record) throws Exception {
- throw new UnsupportedOperationException("Should never be called");
- }
-
- @Override
- public void deserialize(ConsumerRecord<byte[], byte[]> message, Collector<T> out)
- throws Exception {
- deserializationSchema.deserialize(message.value(), out);
- }
-
- @Override
- public boolean isEndOfStream(T nextElement) {
- return deserializationSchema.isEndOfStream(nextElement);
- }
-
- @Override
- public TypeInformation<T> getProducedType() {
- return deserializationSchema.getProducedType();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java
deleted file mode 100644
index bee2339..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nonnull;
-
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Queue;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * A fetcher that fetches data from Kafka brokers via the Kafka consumer API.
- *
- * @param <T> The type of elements produced by the fetcher.
- */
-@Internal
-public class KafkaFetcher<T> extends AbstractFetcher<T, TopicPartition> {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaFetcher.class);
-
- // ------------------------------------------------------------------------
-
- /** The schema to convert between Kafka's byte messages, and Flink's objects. */
- private final KafkaDeserializationSchema<T> deserializer;
-
- /** A collector to emit records in batch (bundle). * */
- private final KafkaCollector kafkaCollector;
-
- /** The handover of data and exceptions between the consumer thread and the task thread. */
- final Handover handover;
-
- /**
- * The thread that runs the actual KafkaConsumer and hand the record batches to this fetcher.
- */
- final KafkaConsumerThread consumerThread;
-
- /** Flag to mark the main work loop as alive. */
- volatile boolean running = true;
-
- // ------------------------------------------------------------------------
-
- public KafkaFetcher(
- SourceFunction.SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval,
- ClassLoader userCodeClassLoader,
- String taskNameWithSubtasks,
- KafkaDeserializationSchema<T> deserializer,
- Properties kafkaProperties,
- long pollTimeout,
- MetricGroup subtaskMetricGroup,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- super(
- sourceContext,
- assignedPartitionsWithInitialOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- userCodeClassLoader,
- consumerMetricGroup,
- useMetrics);
-
- this.deserializer = deserializer;
- this.handover = new Handover();
-
- this.consumerThread =
- new KafkaConsumerThread(
- LOG,
- handover,
- kafkaProperties,
- unassignedPartitionsQueue,
- getFetcherName() + " for " + taskNameWithSubtasks,
- pollTimeout,
- useMetrics,
- consumerMetricGroup,
- subtaskMetricGroup);
- this.kafkaCollector = new KafkaCollector();
- }
-
- // ------------------------------------------------------------------------
- // Fetcher work methods
- // ------------------------------------------------------------------------
-
- @Override
- public void runFetchLoop() throws Exception {
- try {
- // kick off the actual Kafka consumer
- consumerThread.start();
-
- while (running) {
- // this blocks until we get the next records
- // it automatically re-throws exceptions encountered in the consumer thread
- final ConsumerRecords<byte[], byte[]> records = handover.pollNext();
-
- // get the records for each topic partition
- for (KafkaTopicPartitionState<T, TopicPartition> partition :
- subscribedPartitionStates()) {
-
- List<ConsumerRecord<byte[], byte[]>> partitionRecords =
- records.records(partition.getKafkaPartitionHandle());
-
- partitionConsumerRecordsHandler(partitionRecords, partition);
- }
- }
- } finally {
- // this signals the consumer thread that no more work is to be done
- consumerThread.shutdown();
- }
-
- // on a clean exit, wait for the runner thread
- try {
- consumerThread.join();
- } catch (InterruptedException e) {
- // may be the result of a wake-up interruption after an exception.
- // we ignore this here and only restore the interruption state
- Thread.currentThread().interrupt();
- }
- }
-
- @Override
- public void cancel() {
- // flag the main thread to exit. A thread interrupt will come anyways.
- running = false;
- handover.close();
- consumerThread.shutdown();
- }
-
- /** Gets the name of this fetcher, for thread naming and logging purposes. */
- protected String getFetcherName() {
- return "Kafka Fetcher";
- }
-
- protected void partitionConsumerRecordsHandler(
- List<ConsumerRecord<byte[], byte[]>> partitionRecords,
- KafkaTopicPartitionState<T, TopicPartition> partition)
- throws Exception {
-
- for (ConsumerRecord<byte[], byte[]> record : partitionRecords) {
- deserializer.deserialize(record, kafkaCollector);
-
- // emit the actual records. this also updates offset state atomically and emits
- // watermarks
- emitRecordsWithTimestamps(
- kafkaCollector.getRecords(), partition, record.offset(), record.timestamp());
-
- if (kafkaCollector.isEndOfStreamSignalled()) {
- // end of stream signaled
- running = false;
- break;
- }
- }
- }
-
- // ------------------------------------------------------------------------
- // Implement Methods of the AbstractFetcher
- // ------------------------------------------------------------------------
-
- @Override
- public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) {
- return new TopicPartition(partition.getTopic(), partition.getPartition());
- }
-
- @Override
- protected void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback commitCallback)
- throws Exception {
-
- @SuppressWarnings("unchecked")
- List<KafkaTopicPartitionState<T, TopicPartition>> partitions = subscribedPartitionStates();
-
- Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>(partitions.size());
-
- for (KafkaTopicPartitionState<T, TopicPartition> partition : partitions) {
- Long lastProcessedOffset = offsets.get(partition.getKafkaTopicPartition());
- if (lastProcessedOffset != null) {
- checkState(lastProcessedOffset >= 0, "Illegal offset value to commit");
-
- // committed offsets through the KafkaConsumer need to be 1 more than the last
- // processed offset.
- // This does not affect Flink's checkpoints/saved state.
- long offsetToCommit = lastProcessedOffset + 1;
-
- offsetsToCommit.put(
- partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offsetToCommit));
- partition.setCommittedOffset(offsetToCommit);
- }
- }
-
- // record the work to be committed by the main consumer thread and make sure the consumer
- // notices that
- consumerThread.setOffsetsToCommit(offsetsToCommit, commitCallback);
- }
-
- private class KafkaCollector implements Collector<T> {
- private final Queue<T> records = new ArrayDeque<>();
-
- private boolean endOfStreamSignalled = false;
-
- @Override
- public void collect(T record) {
- // do not emit subsequent elements if the end of the stream reached
- if (endOfStreamSignalled || deserializer.isEndOfStream(record)) {
- endOfStreamSignalled = true;
- return;
- }
- records.add(record);
- }
-
- public Queue<T> getRecords() {
- return records;
- }
-
- public boolean isEndOfStreamSignalled() {
- return endOfStreamSignalled;
- }
-
- @Override
- public void close() {}
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java
deleted file mode 100644
index ec78899..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.PartitionInfo;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A partition discoverer that can be used to discover topics and partitions metadata from Kafka
- * brokers via the Kafka high-level consumer API.
- */
-@Internal
-public class KafkaPartitionDiscoverer extends AbstractPartitionDiscoverer {
-
- private final Properties kafkaProperties;
-
- private KafkaConsumer<?, ?> kafkaConsumer;
-
- public KafkaPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks,
- Properties kafkaProperties) {
-
- super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks);
- this.kafkaProperties = checkNotNull(kafkaProperties);
- }
-
- @Override
- protected void initializeConnections() {
- this.kafkaConsumer = new KafkaConsumer<>(kafkaProperties);
- }
-
- @Override
- protected List<String> getAllTopics() throws AbstractPartitionDiscoverer.WakeupException {
- try {
- return new ArrayList<>(kafkaConsumer.listTopics().keySet());
- } catch (org.apache.kafka.common.errors.WakeupException e) {
- // rethrow our own wakeup exception
- throw new AbstractPartitionDiscoverer.WakeupException();
- }
- }
-
- @Override
- protected List<KafkaTopicPartition> getAllPartitionsForTopics(List<String> topics)
- throws WakeupException, RuntimeException {
- final List<KafkaTopicPartition> partitions = new LinkedList<>();
-
- try {
- for (String topic : topics) {
- final List<PartitionInfo> kafkaPartitions = kafkaConsumer.partitionsFor(topic);
-
- if (kafkaPartitions == null) {
- throw new RuntimeException(
- String.format(
- "Could not fetch partitions for %s. Make sure that the topic exists.",
- topic));
- }
-
- for (PartitionInfo partitionInfo : kafkaPartitions) {
- partitions.add(
- new KafkaTopicPartition(
- partitionInfo.topic(), partitionInfo.partition()));
- }
- }
- } catch (org.apache.kafka.common.errors.WakeupException e) {
- // rethrow our own wakeup exception
- throw new WakeupException();
- }
-
- return partitions;
- }
-
- @Override
- protected void wakeupConnections() {
- if (this.kafkaConsumer != null) {
- this.kafkaConsumer.wakeup();
- }
- }
-
- @Override
- protected void closeConnections() throws Exception {
- if (this.kafkaConsumer != null) {
- this.kafkaConsumer.close();
-
- // de-reference the consumer to avoid closing multiple times
- this.kafkaConsumer = null;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java
deleted file mode 100644
index 73b1d42..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.streaming.connectors.kafka.KafkaContextAware;
-import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import javax.annotation.Nullable;
-
-/**
- * An adapter from old style interfaces such as {@link
- * org.apache.flink.api.common.serialization.SerializationSchema}, {@link
- * org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner} to the {@link
- * KafkaSerializationSchema}.
- */
-@Internal
-public class KafkaSerializationSchemaWrapper<T>
- implements KafkaSerializationSchema<T>, KafkaContextAware<T> {
-
- private final FlinkKafkaPartitioner<T> partitioner;
- private final SerializationSchema<T> serializationSchema;
- private final String topic;
- private boolean writeTimestamp;
-
- private int[] partitions;
- private int parallelInstanceId;
- private int numParallelInstances;
-
- public KafkaSerializationSchemaWrapper(
- String topic,
- FlinkKafkaPartitioner<T> partitioner,
- boolean writeTimestamp,
- SerializationSchema<T> serializationSchema) {
- this.partitioner = partitioner;
- this.serializationSchema = serializationSchema;
- this.topic = topic;
- this.writeTimestamp = writeTimestamp;
- }
-
- @Override
- public void open(SerializationSchema.InitializationContext context) throws Exception {
- serializationSchema.open(context);
- if (partitioner != null) {
- partitioner.open(parallelInstanceId, numParallelInstances);
- }
- }
-
- @Override
- public ProducerRecord<byte[], byte[]> serialize(T element, @Nullable Long timestamp) {
- byte[] serialized = serializationSchema.serialize(element);
- final Integer partition;
- if (partitioner != null) {
- partition = partitioner.partition(element, null, serialized, topic, partitions);
- } else {
- partition = null;
- }
-
- final Long timestampToWrite;
- if (writeTimestamp) {
- timestampToWrite = timestamp;
- } else {
- timestampToWrite = null;
- }
-
- return new ProducerRecord<>(topic, partition, timestampToWrite, null, serialized);
- }
-
- @Override
- public String getTargetTopic(T element) {
- return topic;
- }
-
- @Override
- public void setPartitions(int[] partitions) {
- this.partitions = partitions;
- }
-
- @Override
- public void setParallelInstanceId(int parallelInstanceId) {
- this.parallelInstanceId = parallelInstanceId;
- }
-
- @Override
- public void setNumParallelInstances(int numParallelInstances) {
- this.numParallelInstances = numParallelInstances;
- }
-
- public void setWriteTimestamp(boolean writeTimestamp) {
- this.writeTimestamp = writeTimestamp;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java
deleted file mode 100644
index fe7ee7f..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.ByteSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.core.memory.DataInputDeserializer;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.SerializedValue;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_REC_WITHOUT_TIMESTAMP;
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_REC_WITH_TIMESTAMP;
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer.KafkaSerializer.TAG_WATERMARK;
-
-/** Fetch data from Kafka for Kafka Shuffle. */
-@Internal
-public class KafkaShuffleFetcher<T> extends KafkaFetcher<T> {
- /** The handler to check and generate watermarks from fetched records. * */
- private final WatermarkHandler watermarkHandler;
-
- /** The schema to convert between Kafka's byte messages, and Flink's objects. */
- private final KafkaShuffleElementDeserializer kafkaShuffleDeserializer;
-
- public KafkaShuffleFetcher(
- SourceFunction.SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval,
- ClassLoader userCodeClassLoader,
- String taskNameWithSubtasks,
- KafkaDeserializationSchema<T> deserializer,
- Properties kafkaProperties,
- long pollTimeout,
- MetricGroup subtaskMetricGroup,
- MetricGroup consumerMetricGroup,
- boolean useMetrics,
- TypeSerializer<T> typeSerializer,
- int producerParallelism)
- throws Exception {
- super(
- sourceContext,
- assignedPartitionsWithInitialOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- userCodeClassLoader,
- taskNameWithSubtasks,
- deserializer,
- kafkaProperties,
- pollTimeout,
- subtaskMetricGroup,
- consumerMetricGroup,
- useMetrics);
-
- this.kafkaShuffleDeserializer = new KafkaShuffleElementDeserializer<>(typeSerializer);
- this.watermarkHandler = new WatermarkHandler(producerParallelism);
- }
-
- @Override
- protected String getFetcherName() {
- return "Kafka Shuffle Fetcher";
- }
-
- @Override
- protected void partitionConsumerRecordsHandler(
- List<ConsumerRecord<byte[], byte[]>> partitionRecords,
- KafkaTopicPartitionState<T, TopicPartition> partition)
- throws Exception {
-
- for (ConsumerRecord<byte[], byte[]> record : partitionRecords) {
- final KafkaShuffleElement element = kafkaShuffleDeserializer.deserialize(record);
-
- // TODO: Do we need to check the end of stream if reaching the end watermark
- // TODO: Currently, if one of the partition sends an end-of-stream signal the fetcher
- // stops running.
- // The current "ending of stream" logic in KafkaFetcher a bit strange: if any partition
- // has a record
- // signaled as "END_OF_STREAM", the fetcher will stop running. Notice that the signal is
- // coming from
- // the deserializer, which means from Kafka data itself. But it is possible that other
- // topics
- // and partitions still have data to read. Finishing reading Partition0 can not
- // guarantee that Partition1
- // also finishes.
- if (element.isRecord()) {
- // timestamp is inherent from upstream
- // If using ProcessTime, timestamp is going to be ignored (upstream does not include
- // timestamp as well)
- // If using IngestionTime, timestamp is going to be overwritten
- // If using EventTime, timestamp is going to be used
- synchronized (checkpointLock) {
- KafkaShuffleRecord<T> elementAsRecord = element.asRecord();
- sourceContext.collectWithTimestamp(
- elementAsRecord.value,
- elementAsRecord.timestamp == null
- ? record.timestamp()
- : elementAsRecord.timestamp);
- partition.setOffset(record.offset());
- }
- } else if (element.isWatermark()) {
- final KafkaShuffleWatermark watermark = element.asWatermark();
- Optional<Watermark> newWatermark =
- watermarkHandler.checkAndGetNewWatermark(watermark);
- newWatermark.ifPresent(sourceContext::emitWatermark);
- }
- }
- }
-
- /** An element in a KafkaShuffle. Can be a record or a Watermark. */
- @VisibleForTesting
- public abstract static class KafkaShuffleElement {
-
- public boolean isRecord() {
- return getClass() == KafkaShuffleRecord.class;
- }
-
- public boolean isWatermark() {
- return getClass() == KafkaShuffleWatermark.class;
- }
-
- public <T> KafkaShuffleRecord<T> asRecord() {
- return (KafkaShuffleRecord<T>) this;
- }
-
- public KafkaShuffleWatermark asWatermark() {
- return (KafkaShuffleWatermark) this;
- }
- }
-
- /**
- * A watermark element in a KafkaShuffle. It includes - subtask index where the watermark is
- * coming from - watermark timestamp
- */
- @VisibleForTesting
- public static class KafkaShuffleWatermark extends KafkaShuffleElement {
- final int subtask;
- final long watermark;
-
- KafkaShuffleWatermark(int subtask, long watermark) {
- this.subtask = subtask;
- this.watermark = watermark;
- }
-
- public int getSubtask() {
- return subtask;
- }
-
- public long getWatermark() {
- return watermark;
- }
- }
-
- /**
- * One value with Type T in a KafkaShuffle. This stores the value and an optional associated
- * timestamp.
- */
- @VisibleForTesting
- public static class KafkaShuffleRecord<T> extends KafkaShuffleElement {
- final T value;
- final Long timestamp;
-
- KafkaShuffleRecord(T value) {
- this.value = value;
- this.timestamp = null;
- }
-
- KafkaShuffleRecord(long timestamp, T value) {
- this.value = value;
- this.timestamp = timestamp;
- }
-
- public T getValue() {
- return value;
- }
-
- public Long getTimestamp() {
- return timestamp;
- }
- }
-
- /** Deserializer for KafkaShuffleElement. */
- @VisibleForTesting
- public static class KafkaShuffleElementDeserializer<T> implements Serializable {
- private static final long serialVersionUID = 1000001L;
-
- private final TypeSerializer<T> typeSerializer;
-
- private transient DataInputDeserializer dis;
-
- @VisibleForTesting
- public KafkaShuffleElementDeserializer(TypeSerializer<T> typeSerializer) {
- this.typeSerializer = typeSerializer;
- }
-
- @VisibleForTesting
- public KafkaShuffleElement deserialize(ConsumerRecord<byte[], byte[]> record)
- throws Exception {
- byte[] value = record.value();
-
- if (dis != null) {
- dis.setBuffer(value);
- } else {
- dis = new DataInputDeserializer(value);
- }
-
- // version byte
- ByteSerializer.INSTANCE.deserialize(dis);
- int tag = ByteSerializer.INSTANCE.deserialize(dis);
-
- if (tag == TAG_REC_WITHOUT_TIMESTAMP) {
- return new KafkaShuffleRecord<>(typeSerializer.deserialize(dis));
- } else if (tag == TAG_REC_WITH_TIMESTAMP) {
- return new KafkaShuffleRecord<>(
- LongSerializer.INSTANCE.deserialize(dis), typeSerializer.deserialize(dis));
- } else if (tag == TAG_WATERMARK) {
- return new KafkaShuffleWatermark(
- IntSerializer.INSTANCE.deserialize(dis),
- LongSerializer.INSTANCE.deserialize(dis));
- }
-
- throw new UnsupportedOperationException("Unsupported tag format");
- }
- }
-
- /** WatermarkHandler to check and generate watermarks from fetched records. */
- private static class WatermarkHandler {
- private final int producerParallelism;
- private final Map<Integer, Long> subtaskWatermark;
-
- private long currentMinWatermark = Long.MIN_VALUE;
-
- WatermarkHandler(int producerParallelism) {
- this.producerParallelism = producerParallelism;
- this.subtaskWatermark = new HashMap<>(producerParallelism);
- }
-
- private Optional<Watermark> checkAndGetNewWatermark(KafkaShuffleWatermark newWatermark) {
- // watermarks is incremental for the same partition and PRODUCER subtask
- Long currentSubTaskWatermark = subtaskWatermark.get(newWatermark.subtask);
-
- // watermark is strictly increasing
- Preconditions.checkState(
- (currentSubTaskWatermark == null)
- || (currentSubTaskWatermark < newWatermark.watermark),
- "Watermark should always increase: current : new "
- + currentSubTaskWatermark
- + ":"
- + newWatermark.watermark);
-
- subtaskWatermark.put(newWatermark.subtask, newWatermark.watermark);
-
- if (subtaskWatermark.values().size() < producerParallelism) {
- return Optional.empty();
- }
-
- long minWatermark =
- subtaskWatermark.values().stream()
- .min(Comparator.naturalOrder())
- .orElse(Long.MIN_VALUE);
- if (currentMinWatermark < minWatermark) {
- currentMinWatermark = minWatermark;
- return Optional.of(new Watermark(minWatermark));
- } else {
- return Optional.empty();
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
deleted file mode 100644
index f262a22..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Flink's description of a partition in a Kafka topic. Serializable, and common across all Kafka
- * consumer subclasses (0.8, 0.9, ...)
- *
- * <p>Note: This class must not change in its structure, because it would change the serialization
- * format and make previous savepoints unreadable.
- */
-@PublicEvolving
-public final class KafkaTopicPartition implements Serializable {
-
- /**
- * THIS SERIAL VERSION UID MUST NOT CHANGE, BECAUSE IT WOULD BREAK READING OLD SERIALIZED
- * INSTANCES FROM SAVEPOINTS.
- */
- private static final long serialVersionUID = 722083576322742325L;
-
- // ------------------------------------------------------------------------
-
- private final String topic;
- private final int partition;
- private final int cachedHash;
-
- public KafkaTopicPartition(String topic, int partition) {
- this.topic = requireNonNull(topic);
- this.partition = partition;
- this.cachedHash = 31 * topic.hashCode() + partition;
- }
-
- // ------------------------------------------------------------------------
-
- public String getTopic() {
- return topic;
- }
-
- public int getPartition() {
- return partition;
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public String toString() {
- return "KafkaTopicPartition{" + "topic='" + topic + '\'' + ", partition=" + partition + '}';
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- } else if (o instanceof KafkaTopicPartition) {
- KafkaTopicPartition that = (KafkaTopicPartition) o;
- return this.partition == that.partition && this.topic.equals(that.topic);
- } else {
- return false;
- }
- }
-
- @Override
- public int hashCode() {
- return cachedHash;
- }
-
- // ------------------------------------------------------------------------
- // Utilities
- // ------------------------------------------------------------------------
-
- public static String toString(Map<KafkaTopicPartition, Long> map) {
- StringBuilder sb = new StringBuilder();
- for (Map.Entry<KafkaTopicPartition, Long> p : map.entrySet()) {
- KafkaTopicPartition ktp = p.getKey();
- sb.append(ktp.getTopic())
- .append(":")
- .append(ktp.getPartition())
- .append("=")
- .append(p.getValue())
- .append(", ");
- }
- return sb.toString();
- }
-
- public static String toString(List<KafkaTopicPartition> partitions) {
- StringBuilder sb = new StringBuilder();
- for (KafkaTopicPartition p : partitions) {
- sb.append(p.getTopic()).append(":").append(p.getPartition()).append(", ");
- }
- return sb.toString();
- }
-
- public static List<KafkaTopicPartition> dropLeaderData(
- List<KafkaTopicPartitionLeader> partitionInfos) {
- List<KafkaTopicPartition> ret = new ArrayList<>(partitionInfos.size());
- for (KafkaTopicPartitionLeader ktpl : partitionInfos) {
- ret.add(ktpl.getTopicPartition());
- }
- return ret;
- }
-
- /** A {@link java.util.Comparator} for {@link KafkaTopicPartition}s. */
- public static class Comparator implements java.util.Comparator<KafkaTopicPartition> {
- @Override
- public int compare(KafkaTopicPartition p1, KafkaTopicPartition p2) {
- if (!p1.getTopic().equals(p2.getTopic())) {
- return p1.getTopic().compareTo(p2.getTopic());
- } else {
- return Integer.compare(p1.getPartition(), p2.getPartition());
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java
deleted file mode 100644
index be61e8a..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-/** Utility for assigning Kafka partitions to consumer subtasks. */
-@Internal
-public class KafkaTopicPartitionAssigner {
-
- /**
- * Returns the index of the target subtask that a specific Kafka partition should be assigned
- * to.
- *
- * <p>The resulting distribution of partitions of a single topic has the following contract:
- *
- * <ul>
- * <li>1. Uniformly distributed across subtasks
- * <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
- * indices) by using the partition id as the offset from a starting index (i.e., the index
- * of the subtask which partition 0 of the topic will be assigned to, determined using the
- * topic name).
- * </ul>
- *
- * <p>The above contract is crucial and cannot be broken. Consumer subtasks rely on this
- * contract to locally filter out partitions that it should not subscribe to, guaranteeing that
- * all partitions of a single topic will always be assigned to some subtask in a uniformly
- * distributed manner.
- *
- * @param partition the Kafka partition
- * @param numParallelSubtasks total number of parallel subtasks
- * @return index of the target subtask that the Kafka partition should be assigned to.
- */
- public static int assign(KafkaTopicPartition partition, int numParallelSubtasks) {
- return assign(partition.getTopic(), partition.getPartition(), numParallelSubtasks);
- }
-
- public static int assign(String topic, int partition, int numParallelSubtasks) {
- int startIndex = ((topic.hashCode() * 31) & 0x7FFFFFFF) % numParallelSubtasks;
-
- // here, the assumption is that the id of Kafka partitions are always ascending
- // starting from 0, and therefore can be used directly as the offset clockwise from the
- // start index
- return (startIndex + partition) % numParallelSubtasks;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
deleted file mode 100644
index a2ef128..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.kafka.common.Node;
-
-import java.io.Serializable;
-
-/**
- * Serializable Topic Partition info with leader Node information. This class is used at runtime.
- */
-@Internal
-public class KafkaTopicPartitionLeader implements Serializable {
-
- private static final long serialVersionUID = 9145855900303748582L;
-
- private final int leaderId;
- private final int leaderPort;
- private final String leaderHost;
- private final KafkaTopicPartition topicPartition;
- private final int cachedHash;
-
- public KafkaTopicPartitionLeader(KafkaTopicPartition topicPartition, Node leader) {
- this.topicPartition = topicPartition;
- if (leader == null) {
- this.leaderId = -1;
- this.leaderHost = null;
- this.leaderPort = -1;
- } else {
- this.leaderId = leader.id();
- this.leaderPort = leader.port();
- this.leaderHost = leader.host();
- }
- int cachedHash = (leader == null) ? 14 : leader.hashCode();
- this.cachedHash = 31 * cachedHash + topicPartition.hashCode();
- }
-
- public KafkaTopicPartition getTopicPartition() {
- return topicPartition;
- }
-
- public Node getLeader() {
- if (this.leaderId == -1) {
- return null;
- } else {
- return new Node(leaderId, leaderHost, leaderPort);
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof KafkaTopicPartitionLeader)) {
- return false;
- }
-
- KafkaTopicPartitionLeader that = (KafkaTopicPartitionLeader) o;
-
- if (!topicPartition.equals(that.topicPartition)) {
- return false;
- }
- return leaderId == that.leaderId
- && leaderPort == that.leaderPort
- && leaderHost.equals(that.leaderHost);
- }
-
- @Override
- public int hashCode() {
- return cachedHash;
- }
-
- @Override
- public String toString() {
- return "KafkaTopicPartitionLeader{"
- + "leaderId="
- + leaderId
- + ", leaderPort="
- + leaderPort
- + ", leaderHost='"
- + leaderHost
- + '\''
- + ", topic="
- + topicPartition.getTopic()
- + ", partition="
- + topicPartition.getPartition()
- + '}';
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
deleted file mode 100644
index c09df34..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * The state that the Flink Kafka Consumer holds for each Kafka partition. Includes the Kafka
- * descriptor for partitions.
- *
- * <p>This class describes the most basic state (only the offset), subclasses define more elaborate
- * state, containing current watermarks and timestamp extractors.
- *
- * @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
- */
-@Internal
-public class KafkaTopicPartitionState<T, KPH> {
-
- // ------------------------------------------------------------------------
-
- /** The Flink description of a Kafka partition. */
- private final KafkaTopicPartition partition;
-
- /** The Kafka description of a Kafka partition (varies across different Kafka versions). */
- private final KPH kafkaPartitionHandle;
-
- /** The offset within the Kafka partition that we already processed. */
- private volatile long offset;
-
- /** The offset of the Kafka partition that has been committed. */
- private volatile long committedOffset;
-
- // ------------------------------------------------------------------------
-
- public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) {
- this.partition = partition;
- this.kafkaPartitionHandle = kafkaPartitionHandle;
- this.offset = KafkaTopicPartitionStateSentinel.OFFSET_NOT_SET;
- this.committedOffset = KafkaTopicPartitionStateSentinel.OFFSET_NOT_SET;
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * Gets Flink's descriptor for the Kafka Partition.
- *
- * @return The Flink partition descriptor.
- */
- public final KafkaTopicPartition getKafkaTopicPartition() {
- return partition;
- }
-
- /**
- * Gets Kafka's descriptor for the Kafka Partition.
- *
- * @return The Kafka partition descriptor.
- */
- public final KPH getKafkaPartitionHandle() {
- return kafkaPartitionHandle;
- }
-
- public final String getTopic() {
- return partition.getTopic();
- }
-
- public final int getPartition() {
- return partition.getPartition();
- }
-
- /**
- * The current offset in the partition. This refers to the offset last element that we retrieved
- * and emitted successfully. It is the offset that should be stored in a checkpoint.
- */
- public final long getOffset() {
- return offset;
- }
-
- public final void setOffset(long offset) {
- this.offset = offset;
- }
-
- public final boolean isOffsetDefined() {
- return offset != KafkaTopicPartitionStateSentinel.OFFSET_NOT_SET;
- }
-
- public final void setCommittedOffset(long offset) {
- this.committedOffset = offset;
- }
-
- public final long getCommittedOffset() {
- return committedOffset;
- }
-
- public long extractTimestamp(T record, long kafkaEventTimestamp) {
- return kafkaEventTimestamp;
- }
-
- public void onEvent(T event, long timestamp) {
- // do nothing
- }
-
- public void onPeriodicEmit() {
- // do nothing
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public String toString() {
- return "Partition: "
- + partition
- + ", KafkaPartitionHandle="
- + kafkaPartitionHandle
- + ", offset="
- + (isOffsetDefined() ? String.valueOf(offset) : "(not set)");
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java
deleted file mode 100644
index 5be431c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * Magic values used to represent special offset states before partitions are actually read.
- *
- * <p>The values are all negative. Negative offsets are not used by Kafka (invalid), so we pick a
- * number that is probably (hopefully) not used by Kafka as a magic number for anything else.
- */
-@Internal
-public class KafkaTopicPartitionStateSentinel {
-
- /** Magic number that defines an unset offset. */
- public static final long OFFSET_NOT_SET = -915623761776L;
-
- /**
- * Magic number that defines the partition should start from the earliest offset.
- *
- * <p>This is used as a placeholder so that the actual earliest offset can be evaluated lazily
- * when the partition will actually start to be read by the consumer.
- */
- public static final long EARLIEST_OFFSET = -915623761775L;
-
- /**
- * Magic number that defines the partition should start from the latest offset.
- *
- * <p>This is used as a placeholder so that the actual latest offset can be evaluated lazily
- * when the partition will actually start to be read by the consumer.
- */
- public static final long LATEST_OFFSET = -915623761774L;
-
- /**
- * Magic number that defines the partition should start from its committed group offset in
- * Kafka.
- *
- * <p>This is used as a placeholder so that the actual committed group offset can be evaluated
- * lazily when the partition will actually start to be read by the consumer.
- */
- public static final long GROUP_OFFSET = -915623761773L;
-
- public static boolean isSentinel(long offset) {
- return offset < 0;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java
deleted file mode 100644
index 6c84340..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.eventtime.TimestampAssigner;
-import org.apache.flink.api.common.eventtime.WatermarkGenerator;
-import org.apache.flink.api.common.eventtime.WatermarkOutput;
-
-/**
- * A special version of the per-kafka-partition-state that additionally holds a {@link
- * TimestampAssigner}, {@link WatermarkGenerator}, an immediate {@link WatermarkOutput}, and a
- * deferred {@link WatermarkOutput} for this partition.
- *
- * <p>See {@link org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer} for an
- * explanation of immediate and deferred {@link WatermarkOutput WatermarkOutputs.}.
- *
- * @param <T> The type of records handled by the watermark generator
- * @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
- */
-@Internal
-public final class KafkaTopicPartitionStateWithWatermarkGenerator<T, KPH>
- extends KafkaTopicPartitionState<T, KPH> {
-
- private final TimestampAssigner<T> timestampAssigner;
-
- private final WatermarkGenerator<T> watermarkGenerator;
-
- /**
- * Refer to {@link org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer} for a
- * description of immediate/deferred output.
- */
- private final WatermarkOutput immediateOutput;
-
- /**
- * Refer to {@link org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer} for a
- * description of immediate/deferred output.
- */
- private final WatermarkOutput deferredOutput;
-
- // ------------------------------------------------------------------------
-
- public KafkaTopicPartitionStateWithWatermarkGenerator(
- KafkaTopicPartition partition,
- KPH kafkaPartitionHandle,
- TimestampAssigner<T> timestampAssigner,
- WatermarkGenerator<T> watermarkGenerator,
- WatermarkOutput immediateOutput,
- WatermarkOutput deferredOutput) {
- super(partition, kafkaPartitionHandle);
-
- this.timestampAssigner = timestampAssigner;
- this.watermarkGenerator = watermarkGenerator;
- this.immediateOutput = immediateOutput;
- this.deferredOutput = deferredOutput;
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public long extractTimestamp(T record, long kafkaEventTimestamp) {
- return timestampAssigner.extractTimestamp(record, kafkaEventTimestamp);
- }
-
- @Override
- public void onEvent(T event, long timestamp) {
- watermarkGenerator.onEvent(event, timestamp, immediateOutput);
- }
-
- @Override
- public void onPeriodicEmit() {
- watermarkGenerator.onPeriodicEmit(deferredOutput);
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public String toString() {
- return "KafkaTopicPartitionStateWithPeriodicWatermarks: partition="
- + getKafkaTopicPartition()
- + ", offset="
- + getOffset();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java
deleted file mode 100644
index 8261a2b..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/**
- * A Kafka Topics Descriptor describes how the consumer subscribes to Kafka topics - either a fixed
- * list of topics, or a topic pattern.
- */
-@Internal
-public class KafkaTopicsDescriptor implements Serializable {
-
- private static final long serialVersionUID = -3807227764764900975L;
-
- private final List<String> fixedTopics;
- private final Pattern topicPattern;
-
- public KafkaTopicsDescriptor(
- @Nullable List<String> fixedTopics, @Nullable Pattern topicPattern) {
- checkArgument(
- (fixedTopics != null && topicPattern == null)
- || (fixedTopics == null && topicPattern != null),
- "Exactly one of either fixedTopics or topicPattern must be specified.");
-
- if (fixedTopics != null) {
- checkArgument(
- !fixedTopics.isEmpty(),
- "If subscribing to a fixed topics list, the supplied list cannot be empty.");
- }
-
- this.fixedTopics = fixedTopics;
- this.topicPattern = topicPattern;
- }
-
- public boolean isFixedTopics() {
- return fixedTopics != null;
- }
-
- public boolean isTopicPattern() {
- return topicPattern != null;
- }
-
- /**
- * Check if the input topic matches the topics described by this KafkaTopicDescriptor.
- *
- * @return true if found a match.
- */
- public boolean isMatchingTopic(String topic) {
- if (isFixedTopics()) {
- return getFixedTopics().contains(topic);
- } else {
- return topicPattern.matcher(topic).matches();
- }
- }
-
- public List<String> getFixedTopics() {
- return fixedTopics;
- }
-
- @Override
- public String toString() {
- return (fixedTopics == null)
- ? "Topic Regex Pattern (" + topicPattern.pattern() + ")"
- : "Fixed Topics (" + fixedTopics + ")";
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java
deleted file mode 100644
index c95cd9c..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-
-/**
- * A simple wrapper for using the SerializationSchema with the KeyedSerializationSchema interface.
- *
- * @param <T> The type to serialize
- */
-@Internal
-public class KeyedSerializationSchemaWrapper<T> implements KeyedSerializationSchema<T> {
-
- private static final long serialVersionUID = 1351665280744549933L;
-
- private final SerializationSchema<T> serializationSchema;
-
- public KeyedSerializationSchemaWrapper(SerializationSchema<T> serializationSchema) {
- this.serializationSchema = serializationSchema;
- }
-
- public SerializationSchema<T> getSerializationSchema() {
- return serializationSchema;
- }
-
- @Override
- public byte[] serializeKey(T element) {
- return null;
- }
-
- @Override
- public byte[] serializeValue(T element) {
- return serializationSchema.serialize(element);
- }
-
- @Override
- public String getTargetTopic(T element) {
- return null; // we are never overriding the topic
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java
deleted file mode 100644
index 8e56743..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.api.common.eventtime.Watermark;
-import org.apache.flink.api.common.eventtime.WatermarkOutput;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-
-/**
- * A {@link org.apache.flink.api.common.eventtime.WatermarkOutput} that forwards calls to a {@link
- * org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}.
- */
-public class SourceContextWatermarkOutputAdapter<T> implements WatermarkOutput {
- private final SourceContext<T> sourceContext;
-
- public SourceContextWatermarkOutputAdapter(SourceContext<T> sourceContext) {
- this.sourceContext = sourceContext;
- }
-
- @Override
- public void emitWatermark(Watermark watermark) {
- sourceContext.emitWatermark(
- new org.apache.flink.streaming.api.watermark.Watermark(watermark.getTimestamp()));
- }
-
- @Override
- public void markIdle() {
- sourceContext.markAsTemporarilyIdle();
- }
-
- @Override
- public void markActive() {
- // will be set active with next watermark
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java
deleted file mode 100644
index e21355e..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.annotation.Internal;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Class responsible for generating transactional ids to use when communicating with Kafka.
- *
- * <p>It guarantees that:
- *
- * <ul>
- * <li>generated ids to use will never clash with ids to use from different subtasks
- * <li>generated ids to abort will never clash with ids to abort from different subtasks
- * <li>generated ids to use will never clash with ids to abort from different subtasks
- * </ul>
- *
- * <p>In other words, any particular generated id will always be assigned to one and only one
- * subtask.
- */
-@Internal
-public class TransactionalIdsGenerator {
- private final String prefix;
- private final int subtaskIndex;
- private final int totalNumberOfSubtasks;
- private final int poolSize;
- private final int safeScaleDownFactor;
-
- public TransactionalIdsGenerator(
- String prefix,
- int subtaskIndex,
- int totalNumberOfSubtasks,
- int poolSize,
- int safeScaleDownFactor) {
- checkArgument(subtaskIndex < totalNumberOfSubtasks);
- checkArgument(poolSize > 0);
- checkArgument(safeScaleDownFactor > 0);
- checkArgument(subtaskIndex >= 0);
-
- this.prefix = checkNotNull(prefix);
- this.subtaskIndex = subtaskIndex;
- this.totalNumberOfSubtasks = totalNumberOfSubtasks;
- this.poolSize = poolSize;
- this.safeScaleDownFactor = safeScaleDownFactor;
- }
-
- /**
- * Range of available transactional ids to use is: [nextFreeTransactionalId,
- * nextFreeTransactionalId + parallelism * kafkaProducersPoolSize) loop below picks in a
- * deterministic way a subrange of those available transactional ids based on index of this
- * subtask.
- */
- public Set<String> generateIdsToUse(long nextFreeTransactionalId) {
- Set<String> transactionalIds = new HashSet<>();
- for (int i = 0; i < poolSize; i++) {
- long transactionalId = nextFreeTransactionalId + subtaskIndex * poolSize + i;
- transactionalIds.add(generateTransactionalId(transactionalId));
- }
- return transactionalIds;
- }
-
- /**
- * If we have to abort previous transactional id in case of restart after a failure BEFORE first
- * checkpoint completed, we don't know what was the parallelism used in previous attempt. In
- * that case we must guess the ids range to abort based on current configured pool size, current
- * parallelism and safeScaleDownFactor.
- */
- public Set<String> generateIdsToAbort() {
- Set<String> idsToAbort = new HashSet<>();
- for (int i = 0; i < safeScaleDownFactor; i++) {
- idsToAbort.addAll(generateIdsToUse(i * poolSize * totalNumberOfSubtasks));
- }
- return idsToAbort;
- }
-
- private String generateTransactionalId(long transactionalId) {
- return prefix + "-" + transactionalId;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java
deleted file mode 100644
index 30d4697..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.internals.metrics;
-
-import org.apache.flink.annotation.Internal;
-
-/**
- * A collection of Kafka consumer metrics related constant strings.
- *
- * <p>The names must not be changed, as that would break backward compatibility for the consumer's
- * metrics.
- */
-@Internal
-public class KafkaConsumerMetricConstants {
-
- public static final String KAFKA_CONSUMER_METRICS_GROUP = "KafkaConsumer";
-
- // ------------------------------------------------------------------------
- // Per-subtask metrics
- // ------------------------------------------------------------------------
-
- public static final String COMMITS_SUCCEEDED_METRICS_COUNTER = "commitsSucceeded";
- public static final String COMMITS_FAILED_METRICS_COUNTER = "commitsFailed";
-
- // ------------------------------------------------------------------------
- // Per-partition metrics
- // ------------------------------------------------------------------------
-
- public static final String OFFSETS_BY_TOPIC_METRICS_GROUP = "topic";
- public static final String OFFSETS_BY_PARTITION_METRICS_GROUP = "partition";
-
- public static final String CURRENT_OFFSETS_METRICS_GAUGE = "currentOffsets";
- public static final String COMMITTED_OFFSETS_METRICS_GAUGE = "committedOffsets";
-
- // ------------------------------------------------------------------------
- // Legacy metrics
- // ------------------------------------------------------------------------
-
- public static final String LEGACY_CURRENT_OFFSETS_METRICS_GROUP = "current-offsets";
- public static final String LEGACY_COMMITTED_OFFSETS_METRICS_GROUP = "committed-offsets";
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapper.java
deleted file mode 100644
index b5d19ac..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapper.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.internals.metrics;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.metrics.Gauge;
-
-import org.apache.kafka.common.Metric;
-
-/** Gauge for getting the current value of a Kafka metric. */
-@Internal
-public class KafkaMetricMutableWrapper implements Gauge<Double> {
- private Metric kafkaMetric;
-
- public KafkaMetricMutableWrapper(Metric metric) {
- this.kafkaMetric = metric;
- }
-
- @Override
- public Double getValue() {
- final Object metricValue = kafkaMetric.metricValue();
- // Previously KafkaMetric supported KafkaMetric#value that always returned a Double value.
- // Since this method has been deprecated and is removed in future releases we have to
- // manually check if the returned value is Double. Internally, KafkaMetric#value also
- // returned 0.0 for all not "measurable" values, so we restored the original behavior.
- return metricValue instanceof Double ? (Double) metricValue : 0.0;
- }
-
- public void setKafkaMetric(Metric kafkaMetric) {
- this.kafkaMetric = kafkaMetric;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
deleted file mode 100644
index 1ab41ce..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.internals.metrics;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.metrics.Gauge;
-
-/** Gauge for getting the current value of a Kafka metric. */
-@Internal
-public class KafkaMetricWrapper implements Gauge<Double> {
- private final org.apache.kafka.common.Metric kafkaMetric;
-
- public KafkaMetricWrapper(org.apache.kafka.common.Metric metric) {
- this.kafkaMetric = metric;
- }
-
- @Override
- public Double getValue() {
- final Object metricValue = kafkaMetric.metricValue();
- // Previously KafkaMetric supported KafkaMetric#value that always returned a Double value.
- // Since this method has been deprecated and is removed in future releases we have to
- // manually check if the returned value is Double. Internally, KafkaMetric#value also
- // returned 0.0 for all not "measurable" values, so we restored the original behavior.
- return metricValue instanceof Double ? (Double) metricValue : 0.0;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java
deleted file mode 100644
index 16b57f6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.partitioner;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.Preconditions;
-
-/**
- * A partitioner ensuring that each internal Flink partition ends up in one Kafka partition.
- *
- * <p>Note, one Kafka partition can contain multiple Flink partitions.
- *
- * <p>There are a couple of cases to consider.
- *
- * <h3>More Flink partitions than kafka partitions</h3>
- *
- * <pre>
- * Flink Sinks: Kafka Partitions
- * 1 ----------------> 1
- * 2 --------------/
- * 3 -------------/
- * 4 ------------/
- * </pre>
- *
- * <p>Some (or all) kafka partitions contain the output of more than one flink partition
- *
- * <h3>Fewer Flink partitions than Kafka</h3>
- *
- * <pre>
- * Flink Sinks: Kafka Partitions
- * 1 ----------------> 1
- * 2 ----------------> 2
- * 3
- * 4
- * 5
- * </pre>
- *
- * <p>Not all Kafka partitions contain data To avoid such an unbalanced partitioning, use a
- * round-robin kafka partitioner (note that this will cause a lot of network connections between all
- * the Flink instances and all the Kafka brokers).
- */
-@PublicEvolving
-public class FlinkFixedPartitioner<T> extends FlinkKafkaPartitioner<T> {
-
- private static final long serialVersionUID = -3785320239953858777L;
-
- private int parallelInstanceId;
-
- @Override
- public void open(int parallelInstanceId, int parallelInstances) {
- Preconditions.checkArgument(
- parallelInstanceId >= 0, "Id of this subtask cannot be negative.");
- Preconditions.checkArgument(
- parallelInstances > 0, "Number of subtasks must be larger than 0.");
-
- this.parallelInstanceId = parallelInstanceId;
- }
-
- @Override
- public int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions) {
- Preconditions.checkArgument(
- partitions != null && partitions.length > 0,
- "Partitions of the target topic is empty.");
-
- return partitions[parallelInstanceId % partitions.length];
- }
-
- @Override
- public boolean equals(Object o) {
- return this == o || o instanceof FlinkFixedPartitioner;
- }
-
- @Override
- public int hashCode() {
- return FlinkFixedPartitioner.class.hashCode();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java
deleted file mode 100644
index 2fb89e2..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.partitioner;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-
-/**
- * A {@link FlinkKafkaPartitioner} wraps logic on how to partition records across partitions of
- * multiple Kafka topics.
- */
-@PublicEvolving
-public abstract class FlinkKafkaPartitioner<T> implements Serializable {
-
- private static final long serialVersionUID = -9086719227828020494L;
-
- /**
- * Initializer for the partitioner. This is called once on each parallel sink instance of the
- * Flink Kafka producer. This method should be overridden if necessary.
- *
- * @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink
- * @param parallelInstances the total number of parallel instances
- */
- public void open(int parallelInstanceId, int parallelInstances) {
- // overwrite this method if needed.
- }
-
- /**
- * Determine the id of the partition that the record should be written to.
- *
- * @param record the record value
- * @param key serialized key of the record
- * @param value serialized value of the record
- * @param targetTopic target topic for the record
- * @param partitions found partitions for the target topic
- * @return the id of the target partition
- */
- public abstract int partition(
- T record, byte[] key, byte[] value, String targetTopic, int[] partitions);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java
deleted file mode 100644
index ae9af29..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java
+++ /dev/null
@@ -1,402 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.annotation.Experimental;
-import org.apache.flink.api.common.operators.Keys;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamUtils;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.transformations.LegacySinkTransformation;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.PropertiesUtil;
-
-import java.util.Properties;
-
-/**
- * {@link FlinkKafkaShuffle} uses Kafka as a message bus to shuffle and persist data at the same
- * time.
- *
- * <p>Persisting shuffle data is useful when - you would like to reuse the shuffle data and/or, -
- * you would like to avoid a full restart of a pipeline during failure recovery
- *
- * <p>Persisting shuffle is achieved by wrapping a {@link FlinkKafkaShuffleProducer} and a {@link
- * FlinkKafkaShuffleConsumer} together into a {@link FlinkKafkaShuffle}. Here is an example how to
- * use a {@link FlinkKafkaShuffle}.
- *
- * <pre>{@code
- * StreamExecutionEnvironment env = ... // create execution environment
- * DataStream<X> source = env.addSource(...) // add data stream source
- * DataStream<Y> dataStream = ... // some transformation(s) based on source
- *
- * KeyedStream<Y, KEY> keyedStream = FlinkKafkaShuffle
- * .persistentKeyBy( // keyBy shuffle through kafka
- * dataStream, // data stream to be shuffled
- * topic, // Kafka topic written to
- * producerParallelism, // the number of tasks of a Kafka Producer
- * numberOfPartitions, // the number of partitions of the Kafka topic written to
- * kafkaProperties, // kafka properties for Kafka Producer and Consumer
- * keySelector<Y, KEY>); // key selector to retrieve key from `dataStream'
- *
- * keyedStream.transform... // some other transformation(s)
- *
- * KeyedStream<Y, KEY> keyedStreamReuse = FlinkKafkaShuffle
- * .readKeyBy( // Read the Kafka shuffle data again for other usages
- * topic, // the topic of Kafka where data is persisted
- * env, // execution environment, and it can be a new environment
- * typeInformation<Y>, // type information of the data persisted in Kafka
- * kafkaProperties, // kafka properties for Kafka Consumer
- * keySelector<Y, KEY>); // key selector to retrieve key
- *
- * keyedStreamReuse.transform... // some other transformation(s)
- * }</pre>
- *
- * <p>Usage of {@link FlinkKafkaShuffle#persistentKeyBy} is similar to {@link
- * DataStream#keyBy(KeySelector)}. The differences are:
- *
- * <p>1). Partitioning is done through {@link FlinkKafkaShuffleProducer}. {@link
- * FlinkKafkaShuffleProducer} decides which partition a key goes when writing to Kafka
- *
- * <p>2). Shuffle data can be reused through {@link FlinkKafkaShuffle#readKeyBy}, as shown in the
- * example above.
- *
- * <p>3). Job execution is decoupled by the persistent Kafka message bus. In the example, the job
- * execution graph is decoupled to three regions: `KafkaShuffleProducer', `KafkaShuffleConsumer' and
- * `KafkaShuffleConsumerReuse' through `PERSISTENT DATA` as shown below. If any region fails the
- * execution, the other two keep progressing.
- *
- * <pre>
- * source -> ... KafkaShuffleProducer -> PERSISTENT DATA -> KafkaShuffleConsumer -> ...
- * |
- * | ----------> KafkaShuffleConsumerReuse -> ...
- * </pre>
- */
-@Experimental
-public class FlinkKafkaShuffle {
- static final String PRODUCER_PARALLELISM = "producer parallelism";
- static final String PARTITION_NUMBER = "partition number";
-
- /**
- * Uses Kafka as a message bus to persist keyBy shuffle.
- *
- * <p>Persisting keyBy shuffle is achieved by wrapping a {@link FlinkKafkaShuffleProducer} and
- * {@link FlinkKafkaShuffleConsumer} together.
- *
- * <p>On the producer side, {@link FlinkKafkaShuffleProducer} is similar to {@link
- * DataStream#keyBy(KeySelector)}. They use the same key group assignment function {@link
- * KeyGroupRangeAssignment#assignKeyToParallelOperator} to decide which partition a key goes.
- * Hence, each producer task can potentially write to each Kafka partition based on where the
- * key goes. Here, `numberOfPartitions` equals to the key group size. In the case of using
- * {@link TimeCharacteristic#EventTime}, each producer task broadcasts its watermark to ALL of
- * the Kafka partitions to make sure watermark information is propagated correctly.
- *
- * <p>On the consumer side, each consumer task should read partitions equal to the key group
- * indices it is assigned. `numberOfPartitions` is the maximum parallelism of the consumer. This
- * version only supports numberOfPartitions = consumerParallelism. In the case of using {@link
- * TimeCharacteristic#EventTime}, a consumer task is responsible to emit watermarks. Watermarks
- * are read from the corresponding Kafka partitions. Notice that a consumer task only starts to
- * emit a watermark after reading at least one watermark from each producer task to make sure
- * watermarks are monotonically increasing. Hence a consumer task needs to know
- * `producerParallelism` as well.
- *
- * @see FlinkKafkaShuffle#writeKeyBy
- * @see FlinkKafkaShuffle#readKeyBy
- * @param dataStream Data stream to be shuffled
- * @param topic Kafka topic written to
- * @param producerParallelism Parallelism of producer
- * @param numberOfPartitions Number of partitions
- * @param properties Kafka properties
- * @param keySelector Key selector to retrieve key from `dataStream'
- * @param <T> Type of the input data stream
- * @param <K> Type of key
- */
- public static <T, K> KeyedStream<T, K> persistentKeyBy(
- DataStream<T> dataStream,
- String topic,
- int producerParallelism,
- int numberOfPartitions,
- Properties properties,
- KeySelector<T, K> keySelector) {
- // KafkaProducer#propsToMap uses Properties purely as a HashMap without considering the
- // default properties
- // So we have to flatten the default property to first level elements.
- Properties kafkaProperties = PropertiesUtil.flatten(properties);
- kafkaProperties.setProperty(PRODUCER_PARALLELISM, String.valueOf(producerParallelism));
- kafkaProperties.setProperty(PARTITION_NUMBER, String.valueOf(numberOfPartitions));
-
- StreamExecutionEnvironment env = dataStream.getExecutionEnvironment();
-
- writeKeyBy(dataStream, topic, kafkaProperties, keySelector);
- return readKeyBy(topic, env, dataStream.getType(), kafkaProperties, keySelector);
- }
-
- /**
- * Uses Kafka as a message bus to persist keyBy shuffle.
- *
- * <p>Persisting keyBy shuffle is achieved by wrapping a {@link FlinkKafkaShuffleProducer} and
- * {@link FlinkKafkaShuffleConsumer} together.
- *
- * <p>On the producer side, {@link FlinkKafkaShuffleProducer} is similar to {@link
- * DataStream#keyBy(KeySelector)}. They use the same key group assignment function {@link
- * KeyGroupRangeAssignment#assignKeyToParallelOperator} to decide which partition a key goes.
- * Hence, each producer task can potentially write to each Kafka partition based on where the
- * key goes. Here, `numberOfPartitions` equals to the key group size. In the case of using
- * {@link TimeCharacteristic#EventTime}, each producer task broadcasts its watermark to ALL of
- * the Kafka partitions to make sure watermark information is propagated correctly.
- *
- * <p>On the consumer side, each consumer task should read partitions equal to the key group
- * indices it is assigned. `numberOfPartitions` is the maximum parallelism of the consumer. This
- * version only supports numberOfPartitions = consumerParallelism. In the case of using {@link
- * TimeCharacteristic#EventTime}, a consumer task is responsible to emit watermarks. Watermarks
- * are read from the corresponding Kafka partitions. Notice that a consumer task only starts to
- * emit a watermark after reading at least one watermark from each producer task to make sure
- * watermarks are monotonically increasing. Hence a consumer task needs to know
- * `producerParallelism` as well.
- *
- * @see FlinkKafkaShuffle#writeKeyBy
- * @see FlinkKafkaShuffle#readKeyBy
- * @param dataStream Data stream to be shuffled
- * @param topic Kafka topic written to
- * @param producerParallelism Parallelism of producer
- * @param numberOfPartitions Number of partitions
- * @param properties Kafka properties
- * @param fields Key positions from the input data stream
- * @param <T> Type of the input data stream
- */
- public static <T> KeyedStream<T, Tuple> persistentKeyBy(
- DataStream<T> dataStream,
- String topic,
- int producerParallelism,
- int numberOfPartitions,
- Properties properties,
- int... fields) {
- return persistentKeyBy(
- dataStream,
- topic,
- producerParallelism,
- numberOfPartitions,
- properties,
- keySelector(dataStream, fields));
- }
-
- /**
- * The write side of {@link FlinkKafkaShuffle#persistentKeyBy}.
- *
- * <p>This function contains a {@link FlinkKafkaShuffleProducer} to shuffle and persist data in
- * Kafka. {@link FlinkKafkaShuffleProducer} uses the same key group assignment function {@link
- * KeyGroupRangeAssignment#assignKeyToParallelOperator} to decide which partition a key goes.
- * Hence, each producer task can potentially write to each Kafka partition based on the key.
- * Here, the number of partitions equals to the key group size. In the case of using {@link
- * TimeCharacteristic#EventTime}, each producer task broadcasts each watermark to all of the
- * Kafka partitions to make sure watermark information is propagated properly.
- *
- * <p>Attention: make sure kafkaProperties include {@link
- * FlinkKafkaShuffle#PRODUCER_PARALLELISM} and {@link FlinkKafkaShuffle#PARTITION_NUMBER}
- * explicitly. {@link FlinkKafkaShuffle#PRODUCER_PARALLELISM} is the parallelism of the
- * producer. {@link FlinkKafkaShuffle#PARTITION_NUMBER} is the number of partitions. They are
- * not necessarily the same and allowed to be set independently.
- *
- * @see FlinkKafkaShuffle#persistentKeyBy
- * @see FlinkKafkaShuffle#readKeyBy
- * @param dataStream Data stream to be shuffled
- * @param topic Kafka topic written to
- * @param kafkaProperties Kafka properties for Kafka Producer
- * @param keySelector Key selector to retrieve key from `dataStream'
- * @param <T> Type of the input data stream
- * @param <K> Type of key
- */
- public static <T, K> void writeKeyBy(
- DataStream<T> dataStream,
- String topic,
- Properties kafkaProperties,
- KeySelector<T, K> keySelector) {
-
- StreamExecutionEnvironment env = dataStream.getExecutionEnvironment();
- TypeSerializer<T> typeSerializer = dataStream.getType().createSerializer(env.getConfig());
-
- // write data to Kafka
- FlinkKafkaShuffleProducer<T, K> kafkaProducer =
- new FlinkKafkaShuffleProducer<>(
- topic,
- typeSerializer,
- kafkaProperties,
- env.clean(keySelector),
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE,
- FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
-
- // make sure the sink parallelism is set to producerParallelism
- Preconditions.checkArgument(
- kafkaProperties.getProperty(PRODUCER_PARALLELISM) != null,
- "Missing producer parallelism for Kafka Shuffle");
- int producerParallelism =
- PropertiesUtil.getInt(kafkaProperties, PRODUCER_PARALLELISM, Integer.MIN_VALUE);
-
- addKafkaShuffle(dataStream, kafkaProducer, producerParallelism);
- }
-
- /**
- * The write side of {@link FlinkKafkaShuffle#persistentKeyBy}.
- *
- * <p>This function contains a {@link FlinkKafkaShuffleProducer} to shuffle and persist data in
- * Kafka. {@link FlinkKafkaShuffleProducer} uses the same key group assignment function {@link
- * KeyGroupRangeAssignment#assignKeyToParallelOperator} to decide which partition a key goes.
- *
- * <p>Hence, each producer task can potentially write to each Kafka partition based on the key.
- * Here, the number of partitions equals to the key group size. In the case of using {@link
- * TimeCharacteristic#EventTime}, each producer task broadcasts each watermark to all of the
- * Kafka partitions to make sure watermark information is propagated properly.
- *
- * <p>Attention: make sure kafkaProperties include {@link
- * FlinkKafkaShuffle#PRODUCER_PARALLELISM} and {@link FlinkKafkaShuffle#PARTITION_NUMBER}
- * explicitly. {@link FlinkKafkaShuffle#PRODUCER_PARALLELISM} is the parallelism of the
- * producer. {@link FlinkKafkaShuffle#PARTITION_NUMBER} is the number of partitions. They are
- * not necessarily the same and allowed to be set independently.
- *
- * @see FlinkKafkaShuffle#persistentKeyBy
- * @see FlinkKafkaShuffle#readKeyBy
- * @param dataStream Data stream to be shuffled
- * @param topic Kafka topic written to
- * @param kafkaProperties Kafka properties for Kafka Producer
- * @param fields Key positions from the input data stream
- * @param <T> Type of the input data stream
- */
- public static <T> void writeKeyBy(
- DataStream<T> dataStream, String topic, Properties kafkaProperties, int... fields) {
- writeKeyBy(dataStream, topic, kafkaProperties, keySelector(dataStream, fields));
- }
-
- /**
- * The read side of {@link FlinkKafkaShuffle#persistentKeyBy}.
- *
- * <p>Each consumer task should read kafka partitions equal to the key group indices it is
- * assigned. The number of kafka partitions is the maximum parallelism of the consumer. This
- * version only supports numberOfPartitions = consumerParallelism. In the case of using {@link
- * TimeCharacteristic#EventTime}, a consumer task is responsible to emit watermarks. Watermarks
- * are read from the corresponding Kafka partitions. Notice that a consumer task only starts to
- * emit a watermark after receiving at least one watermark from each producer task to make sure
- * watermarks are monotonically increasing. Hence a consumer task needs to know
- * `producerParallelism` as well.
- *
- * <p>Attention: make sure kafkaProperties include {@link
- * FlinkKafkaShuffle#PRODUCER_PARALLELISM} and {@link FlinkKafkaShuffle#PARTITION_NUMBER}
- * explicitly. {@link FlinkKafkaShuffle#PRODUCER_PARALLELISM} is the parallelism of the
- * producer. {@link FlinkKafkaShuffle#PARTITION_NUMBER} is the number of partitions. They are
- * not necessarily the same and allowed to be set independently.
- *
- * @see FlinkKafkaShuffle#persistentKeyBy
- * @see FlinkKafkaShuffle#writeKeyBy
- * @param topic The topic of Kafka where data is persisted
- * @param env Execution environment. readKeyBy's environment can be different from writeKeyBy's
- * @param typeInformation Type information of the data persisted in Kafka
- * @param kafkaProperties kafka properties for Kafka Consumer
- * @param keySelector key selector to retrieve key
- * @param <T> Schema type
- * @param <K> Key type
- * @return Keyed data stream
- */
- public static <T, K> KeyedStream<T, K> readKeyBy(
- String topic,
- StreamExecutionEnvironment env,
- TypeInformation<T> typeInformation,
- Properties kafkaProperties,
- KeySelector<T, K> keySelector) {
-
- TypeSerializer<T> typeSerializer = typeInformation.createSerializer(env.getConfig());
- TypeInformationSerializationSchema<T> schema =
- new TypeInformationSerializationSchema<>(typeInformation, typeSerializer);
-
- SourceFunction<T> kafkaConsumer =
- new FlinkKafkaShuffleConsumer<>(topic, schema, typeSerializer, kafkaProperties);
-
- // TODO: consider situations where numberOfPartitions != consumerParallelism
- Preconditions.checkArgument(
- kafkaProperties.getProperty(PARTITION_NUMBER) != null,
- "Missing partition number for Kafka Shuffle");
- int numberOfPartitions =
- PropertiesUtil.getInt(kafkaProperties, PARTITION_NUMBER, Integer.MIN_VALUE);
- // Set the parallelism / max parallelism of the keyed stream in consumer side as the number
- // of kafka partitions
- DataStream<T> outputDataStream =
- env.addSource(kafkaConsumer)
- .setParallelism(numberOfPartitions)
- .setMaxParallelism(numberOfPartitions);
-
- return DataStreamUtils.reinterpretAsKeyedStream(outputDataStream, keySelector);
- }
-
- /**
- * Adds a {@link StreamKafkaShuffleSink} to {@link DataStream}.
- *
- * <p>{@link StreamKafkaShuffleSink} is associated a {@link FlinkKafkaShuffleProducer}.
- *
- * @param inputStream Input data stream connected to the shuffle
- * @param kafkaShuffleProducer Kafka shuffle sink function that can handle both records and
- * watermark
- * @param producerParallelism The number of tasks writing to the kafka shuffle
- */
- private static <T, K> void addKafkaShuffle(
- DataStream<T> inputStream,
- FlinkKafkaShuffleProducer<T, K> kafkaShuffleProducer,
- int producerParallelism) {
-
- // read the output type of the input Transform to coax out errors about MissingTypeInfo
- inputStream.getTransformation().getOutputType();
-
- StreamKafkaShuffleSink<T> shuffleSinkOperator =
- new StreamKafkaShuffleSink<>(kafkaShuffleProducer);
- LegacySinkTransformation<T> transformation =
- new LegacySinkTransformation<>(
- inputStream.getTransformation(),
- "kafka_shuffle",
- shuffleSinkOperator,
- inputStream.getExecutionEnvironment().getParallelism(),
- false);
- inputStream.getExecutionEnvironment().addOperator(transformation);
- transformation.setParallelism(producerParallelism);
- }
-
- // A better place to put this function is DataStream; but put it here for now to avoid changing
- // DataStream
- private static <T> KeySelector<T, Tuple> keySelector(DataStream<T> source, int... fields) {
- KeySelector<T, Tuple> keySelector;
- if (source.getType() instanceof BasicArrayTypeInfo
- || source.getType() instanceof PrimitiveArrayTypeInfo) {
- keySelector = KeySelectorUtil.getSelectorForArray(fields, source.getType());
- } else {
- Keys<T> keys = new Keys.ExpressionKeys<>(fields, source.getType());
- keySelector =
- KeySelectorUtil.getSelectorForKeys(
- keys, source.getType(), source.getExecutionEnvironment().getConfig());
- }
-
- return keySelector;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java
deleted file mode 100644
index 886343b..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.PropertiesUtil;
-import org.apache.flink.util.SerializedValue;
-
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle.PRODUCER_PARALLELISM;
-
-/** Flink Kafka Shuffle Consumer Function. */
-@Internal
-public class FlinkKafkaShuffleConsumer<T> extends FlinkKafkaConsumer<T> {
- private final TypeSerializer<T> typeSerializer;
- private final int producerParallelism;
-
- FlinkKafkaShuffleConsumer(
- String topic,
- TypeInformationSerializationSchema<T> schema,
- TypeSerializer<T> typeSerializer,
- Properties props) {
- // The schema is needed to call the right FlinkKafkaConsumer constructor.
- // It is never used, can be `null`, but `null` confuses the compiler.
- super(topic, schema, props);
- this.typeSerializer = typeSerializer;
-
- Preconditions.checkArgument(
- props.getProperty(PRODUCER_PARALLELISM) != null,
- "Missing producer parallelism for Kafka Shuffle");
- producerParallelism = PropertiesUtil.getInt(props, PRODUCER_PARALLELISM, Integer.MAX_VALUE);
- }
-
- @Override
- protected AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- // make sure that auto commit is disabled when our offset commit mode is ON_CHECKPOINTS;
- // this overwrites whatever setting the user configured in the properties
- adjustAutoCommitConfig(properties, offsetCommitMode);
-
- return new KafkaShuffleFetcher<>(
- sourceContext,
- assignedPartitionsWithInitialOffsets,
- watermarkStrategy,
- runtimeContext.getProcessingTimeService(),
- runtimeContext.getExecutionConfig().getAutoWatermarkInterval(),
- runtimeContext.getUserCodeClassLoader(),
- runtimeContext.getTaskNameWithSubtasks(),
- deserializer,
- properties,
- pollTimeout,
- runtimeContext.getMetricGroup(),
- consumerMetricGroup,
- useMetrics,
- typeSerializer,
- producerParallelism);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java
deleted file mode 100644
index e05e8f9..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.core.memory.DataOutputSerializer;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaException;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionAssigner;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.PropertiesUtil;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle.PARTITION_NUMBER;
-
-/**
- * Flink Kafka Shuffle Producer Function. It is different from {@link FlinkKafkaProducer} in the way
- * handling elements and watermarks
- */
-@Internal
-public class FlinkKafkaShuffleProducer<IN, KEY> extends FlinkKafkaProducer<IN> {
- private final KafkaSerializer<IN> kafkaSerializer;
- private final KeySelector<IN, KEY> keySelector;
- private final int numberOfPartitions;
-
- private final Map<Integer, Integer> subtaskToPartitionMap;
-
- FlinkKafkaShuffleProducer(
- String defaultTopicId,
- TypeSerializer<IN> typeSerializer,
- Properties props,
- KeySelector<IN, KEY> keySelector,
- Semantic semantic,
- int kafkaProducersPoolSize) {
- super(
- defaultTopicId,
- (element, timestamp) -> null,
- props,
- semantic,
- kafkaProducersPoolSize);
-
- this.kafkaSerializer = new KafkaSerializer<>(typeSerializer);
- this.keySelector = keySelector;
-
- Preconditions.checkArgument(
- props.getProperty(PARTITION_NUMBER) != null,
- "Missing partition number for Kafka Shuffle");
- numberOfPartitions = PropertiesUtil.getInt(props, PARTITION_NUMBER, Integer.MIN_VALUE);
- subtaskToPartitionMap = new HashMap<>();
- }
-
- /**
- * This is the function invoked to handle each element.
- *
- * @param transaction Transaction state; elements are written to Kafka in transactions to
- * guarantee different level of data consistency
- * @param next Element to handle
- * @param context Context needed to handle the element
- * @throws FlinkKafkaException for kafka error
- */
- @Override
- public void invoke(KafkaTransactionState transaction, IN next, Context context)
- throws FlinkKafkaException {
- checkErroneous();
-
- // write timestamp to Kafka if timestamp is available
- Long timestamp = context.timestamp();
-
- int[] partitions = getPartitions(transaction);
- int partitionIndex;
- try {
- int subtaskIndex =
- KeyGroupRangeAssignment.assignKeyToParallelOperator(
- keySelector.getKey(next), partitions.length, partitions.length);
- partitionIndex = subtaskToPartitionMap.get(subtaskIndex);
- } catch (Exception e) {
- throw new RuntimeException("Fail to assign a partition number to record", e);
- }
-
- ProducerRecord<byte[], byte[]> record =
- new ProducerRecord<>(
- defaultTopicId,
- partitionIndex,
- timestamp,
- null,
- kafkaSerializer.serializeRecord(next, timestamp));
-
- pendingRecords.incrementAndGet();
- transaction.getProducer().send(record, callback);
- }
-
- /**
- * This is the function invoked to handle each watermark.
- *
- * @param watermark Watermark to handle
- * @throws FlinkKafkaException For kafka error
- */
- public void invoke(Watermark watermark) throws FlinkKafkaException {
- checkErroneous();
- KafkaTransactionState transaction = currentTransaction();
-
- int[] partitions = getPartitions(transaction);
- int subtask = getRuntimeContext().getIndexOfThisSubtask();
-
- // broadcast watermark
- long timestamp = watermark.getTimestamp();
- for (int partition : partitions) {
- ProducerRecord<byte[], byte[]> record =
- new ProducerRecord<>(
- defaultTopicId,
- partition,
- timestamp,
- null,
- kafkaSerializer.serializeWatermark(watermark, subtask));
-
- pendingRecords.incrementAndGet();
- transaction.getProducer().send(record, callback);
- }
- }
-
- private int[] getPartitions(KafkaTransactionState transaction) {
- int[] partitions = topicPartitionsMap.get(defaultTopicId);
- if (partitions == null) {
- partitions = getPartitionsByTopic(defaultTopicId, transaction.getProducer());
- topicPartitionsMap.put(defaultTopicId, partitions);
- for (int i = 0; i < partitions.length; i++) {
- subtaskToPartitionMap.put(
- KafkaTopicPartitionAssigner.assign(
- defaultTopicId, partitions[i], partitions.length),
- partitions[i]);
- }
- }
-
- Preconditions.checkArgument(partitions.length == numberOfPartitions);
-
- return partitions;
- }
-
- /** Flink Kafka Shuffle Serializer. */
- public static final class KafkaSerializer<IN> implements Serializable {
- public static final int TAG_REC_WITH_TIMESTAMP = 0;
- public static final int TAG_REC_WITHOUT_TIMESTAMP = 1;
- public static final int TAG_WATERMARK = 2;
-
- private static final long serialVersionUID = 2000002L;
- // easy for updating SerDe format later
- private static final int KAFKA_SHUFFLE_VERSION = 0;
-
- private final TypeSerializer<IN> serializer;
-
- private transient DataOutputSerializer dos;
-
- KafkaSerializer(TypeSerializer<IN> serializer) {
- this.serializer = serializer;
- }
-
- /** Format: Version(byte), TAG(byte), [timestamp(long)], record. */
- byte[] serializeRecord(IN record, Long timestamp) {
- if (dos == null) {
- dos = new DataOutputSerializer(16);
- }
-
- try {
- dos.write(KAFKA_SHUFFLE_VERSION);
-
- if (timestamp == null) {
- dos.write(TAG_REC_WITHOUT_TIMESTAMP);
- } else {
- dos.write(TAG_REC_WITH_TIMESTAMP);
- dos.writeLong(timestamp);
- }
- serializer.serialize(record, dos);
-
- } catch (IOException e) {
- throw new RuntimeException("Unable to serialize record", e);
- }
-
- byte[] ret = dos.getCopyOfBuffer();
- dos.clear();
- return ret;
- }
-
- /** Format: Version(byte), TAG(byte), subtask(int), timestamp(long). */
- byte[] serializeWatermark(Watermark watermark, int subtask) {
- if (dos == null) {
- dos = new DataOutputSerializer(16);
- }
-
- try {
- dos.write(KAFKA_SHUFFLE_VERSION);
- dos.write(TAG_WATERMARK);
- dos.writeInt(subtask);
- dos.writeLong(watermark.getTimestamp());
- } catch (IOException e) {
- throw new RuntimeException("Unable to serialize watermark", e);
- }
-
- byte[] ret = dos.getCopyOfBuffer();
- dos.clear();
- return ret;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java
deleted file mode 100644
index 8bd7784..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-/**
- * A customized {@link StreamOperator} for executing {@link FlinkKafkaShuffleProducer} that handle
- * both elements and watermarks. If the shuffle sink is determined to be useful to other sinks in
- * the future, we should abstract this operator to data stream api. For now, we keep the operator
- * this way to avoid public interface change.
- */
-@Internal
-class StreamKafkaShuffleSink<IN> extends StreamSink<IN> {
-
- public StreamKafkaShuffleSink(FlinkKafkaShuffleProducer flinkKafkaShuffleProducer) {
- super(flinkKafkaShuffleProducer);
- }
-
- @Override
- public void processWatermark(Watermark mark) throws Exception {
- super.processWatermark(mark);
- ((FlinkKafkaShuffleProducer) userFunction).invoke(mark);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java
deleted file mode 100644
index 9179828..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java
+++ /dev/null
@@ -1,273 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.types.DeserializationException;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-/** A specific {@link KafkaSerializationSchema} for {@link KafkaDynamicSource}. */
-class DynamicKafkaDeserializationSchema implements KafkaDeserializationSchema<RowData> {
-
- private static final long serialVersionUID = 1L;
-
- private final @Nullable DeserializationSchema<RowData> keyDeserialization;
-
- private final DeserializationSchema<RowData> valueDeserialization;
-
- private final boolean hasMetadata;
-
- private final BufferingCollector keyCollector;
-
- private final OutputProjectionCollector outputCollector;
-
- private final TypeInformation<RowData> producedTypeInfo;
-
- private final boolean upsertMode;
-
- DynamicKafkaDeserializationSchema(
- int physicalArity,
- @Nullable DeserializationSchema<RowData> keyDeserialization,
- int[] keyProjection,
- DeserializationSchema<RowData> valueDeserialization,
- int[] valueProjection,
- boolean hasMetadata,
- MetadataConverter[] metadataConverters,
- TypeInformation<RowData> producedTypeInfo,
- boolean upsertMode) {
- if (upsertMode) {
- Preconditions.checkArgument(
- keyDeserialization != null && keyProjection.length > 0,
- "Key must be set in upsert mode for deserialization schema.");
- }
- this.keyDeserialization = keyDeserialization;
- this.valueDeserialization = valueDeserialization;
- this.hasMetadata = hasMetadata;
- this.keyCollector = new BufferingCollector();
- this.outputCollector =
- new OutputProjectionCollector(
- physicalArity,
- keyProjection,
- valueProjection,
- metadataConverters,
- upsertMode);
- this.producedTypeInfo = producedTypeInfo;
- this.upsertMode = upsertMode;
- }
-
- @Override
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- if (keyDeserialization != null) {
- keyDeserialization.open(context);
- }
- valueDeserialization.open(context);
- }
-
- @Override
- public boolean isEndOfStream(RowData nextElement) {
- return false;
- }
-
- @Override
- public RowData deserialize(ConsumerRecord<byte[], byte[]> record) throws Exception {
- throw new IllegalStateException("A collector is required for deserializing.");
- }
-
- @Override
- public void deserialize(ConsumerRecord<byte[], byte[]> record, Collector<RowData> collector)
- throws Exception {
- // shortcut in case no output projection is required,
- // also not for a cartesian product with the keys
- if (keyDeserialization == null && !hasMetadata) {
- valueDeserialization.deserialize(record.value(), collector);
- return;
- }
-
- // buffer key(s)
- if (keyDeserialization != null) {
- keyDeserialization.deserialize(record.key(), keyCollector);
- }
-
- // project output while emitting values
- outputCollector.inputRecord = record;
- outputCollector.physicalKeyRows = keyCollector.buffer;
- outputCollector.outputCollector = collector;
- if (record.value() == null && upsertMode) {
- // collect tombstone messages in upsert mode by hand
- outputCollector.collect(null);
- } else {
- valueDeserialization.deserialize(record.value(), outputCollector);
- }
- keyCollector.buffer.clear();
- }
-
- @Override
- public TypeInformation<RowData> getProducedType() {
- return producedTypeInfo;
- }
-
- // --------------------------------------------------------------------------------------------
-
- interface MetadataConverter extends Serializable {
- Object read(ConsumerRecord<?, ?> record);
- }
-
- // --------------------------------------------------------------------------------------------
-
- private static final class BufferingCollector implements Collector<RowData>, Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private final List<RowData> buffer = new ArrayList<>();
-
- @Override
- public void collect(RowData record) {
- buffer.add(record);
- }
-
- @Override
- public void close() {
- // nothing to do
- }
- }
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Emits a row with key, value, and metadata fields.
- *
- * <p>The collector is able to handle the following kinds of keys:
- *
- * <ul>
- * <li>No key is used.
- * <li>A key is used.
- * <li>The deserialization schema emits multiple keys.
- * <li>Keys and values have overlapping fields.
- * <li>Keys are used and value is null.
- * </ul>
- */
- private static final class OutputProjectionCollector
- implements Collector<RowData>, Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private final int physicalArity;
-
- private final int[] keyProjection;
-
- private final int[] valueProjection;
-
- private final MetadataConverter[] metadataConverters;
-
- private final boolean upsertMode;
-
- private transient ConsumerRecord<?, ?> inputRecord;
-
- private transient List<RowData> physicalKeyRows;
-
- private transient Collector<RowData> outputCollector;
-
- OutputProjectionCollector(
- int physicalArity,
- int[] keyProjection,
- int[] valueProjection,
- MetadataConverter[] metadataConverters,
- boolean upsertMode) {
- this.physicalArity = physicalArity;
- this.keyProjection = keyProjection;
- this.valueProjection = valueProjection;
- this.metadataConverters = metadataConverters;
- this.upsertMode = upsertMode;
- }
-
- @Override
- public void collect(RowData physicalValueRow) {
- // no key defined
- if (keyProjection.length == 0) {
- emitRow(null, (GenericRowData) physicalValueRow);
- return;
- }
-
- // otherwise emit a value for each key
- for (RowData physicalKeyRow : physicalKeyRows) {
- emitRow((GenericRowData) physicalKeyRow, (GenericRowData) physicalValueRow);
- }
- }
-
- @Override
- public void close() {
- // nothing to do
- }
-
- private void emitRow(
- @Nullable GenericRowData physicalKeyRow,
- @Nullable GenericRowData physicalValueRow) {
- final RowKind rowKind;
- if (physicalValueRow == null) {
- if (upsertMode) {
- rowKind = RowKind.DELETE;
- } else {
- throw new DeserializationException(
- "Invalid null value received in non-upsert mode. Could not to set row kind for output record.");
- }
- } else {
- rowKind = physicalValueRow.getRowKind();
- }
-
- final int metadataArity = metadataConverters.length;
- final GenericRowData producedRow =
- new GenericRowData(rowKind, physicalArity + metadataArity);
-
- for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) {
- assert physicalKeyRow != null;
- producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos));
- }
-
- if (physicalValueRow != null) {
- for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) {
- producedRow.setField(
- valueProjection[valuePos], physicalValueRow.getField(valuePos));
- }
- }
-
- for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) {
- producedRow.setField(
- physicalArity + metadataPos,
- metadataConverters[metadataPos].read(inputRecord));
- }
-
- outputCollector.collect(producedRow);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java
deleted file mode 100644
index 7908ade..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-
-import javax.annotation.Nullable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. */
-class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema<RowData> {
-
- private final String topic;
- private final FlinkKafkaPartitioner<RowData> partitioner;
- @Nullable private final SerializationSchema<RowData> keySerialization;
- private final SerializationSchema<RowData> valueSerialization;
- private final RowData.FieldGetter[] keyFieldGetters;
- private final RowData.FieldGetter[] valueFieldGetters;
- private final boolean hasMetadata;
- private final int[] metadataPositions;
- private final boolean upsertMode;
-
- DynamicKafkaRecordSerializationSchema(
- String topic,
- @Nullable FlinkKafkaPartitioner<RowData> partitioner,
- @Nullable SerializationSchema<RowData> keySerialization,
- SerializationSchema<RowData> valueSerialization,
- RowData.FieldGetter[] keyFieldGetters,
- RowData.FieldGetter[] valueFieldGetters,
- boolean hasMetadata,
- int[] metadataPositions,
- boolean upsertMode) {
- if (upsertMode) {
- Preconditions.checkArgument(
- keySerialization != null && keyFieldGetters.length > 0,
- "Key must be set in upsert mode for serialization schema.");
- }
- this.topic = checkNotNull(topic);
- this.partitioner = partitioner;
- this.keySerialization = keySerialization;
- this.valueSerialization = checkNotNull(valueSerialization);
- this.keyFieldGetters = keyFieldGetters;
- this.valueFieldGetters = valueFieldGetters;
- this.hasMetadata = hasMetadata;
- this.metadataPositions = metadataPositions;
- this.upsertMode = upsertMode;
- }
-
- @Override
- public ProducerRecord<byte[], byte[]> serialize(
- RowData consumedRow, KafkaSinkContext context, Long timestamp) {
- // shortcut in case no input projection is required
- if (keySerialization == null && !hasMetadata) {
- final byte[] valueSerialized = valueSerialization.serialize(consumedRow);
- return new ProducerRecord<>(
- topic,
- extractPartition(
- consumedRow,
- null,
- valueSerialized,
- context.getPartitionsForTopic(topic)),
- null,
- valueSerialized);
- }
- final byte[] keySerialized;
- if (keySerialization == null) {
- keySerialized = null;
- } else {
- final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters);
- keySerialized = keySerialization.serialize(keyRow);
- }
-
- final byte[] valueSerialized;
- final RowKind kind = consumedRow.getRowKind();
- if (upsertMode) {
- if (kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE) {
- // transform the message as the tombstone message
- valueSerialized = null;
- } else {
- // make the message to be INSERT to be compliant with the INSERT-ONLY format
- final RowData valueRow =
- DynamicKafkaRecordSerializationSchema.createProjectedRow(
- consumedRow, kind, valueFieldGetters);
- valueRow.setRowKind(RowKind.INSERT);
- valueSerialized = valueSerialization.serialize(valueRow);
- }
- } else {
- final RowData valueRow =
- DynamicKafkaRecordSerializationSchema.createProjectedRow(
- consumedRow, kind, valueFieldGetters);
- valueSerialized = valueSerialization.serialize(valueRow);
- }
-
- return new ProducerRecord<>(
- topic,
- extractPartition(
- consumedRow,
- keySerialized,
- valueSerialized,
- context.getPartitionsForTopic(topic)),
- readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.TIMESTAMP),
- keySerialized,
- valueSerialized,
- readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.HEADERS));
- }
-
- @Override
- public void open(
- SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext)
- throws Exception {
- if (keySerialization != null) {
- keySerialization.open(context);
- }
- if (partitioner != null) {
- partitioner.open(
- sinkContext.getParallelInstanceId(),
- sinkContext.getNumberOfParallelInstances());
- }
- valueSerialization.open(context);
- }
-
- private Integer extractPartition(
- RowData consumedRow,
- @Nullable byte[] keySerialized,
- byte[] valueSerialized,
- int[] partitions) {
- if (partitioner != null) {
- return partitioner.partition(
- consumedRow, keySerialized, valueSerialized, topic, partitions);
- }
- return null;
- }
-
- static RowData createProjectedRow(
- RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) {
- final int arity = fieldGetters.length;
- final GenericRowData genericRowData = new GenericRowData(kind, arity);
- for (int fieldPos = 0; fieldPos < arity; fieldPos++) {
- genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow));
- }
- return genericRowData;
- }
-
- @SuppressWarnings("unchecked")
- private <T> T readMetadata(RowData consumedRow, KafkaDynamicSink.WritableMetadata metadata) {
- final int pos = metadataPositions[metadata.ordinal()];
- if (pos < 0) {
- return null;
- }
- return (T) metadata.converter.read(consumedRow, pos);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java
deleted file mode 100644
index a6cdbce..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.DescribedEnum;
-import org.apache.flink.configuration.description.Description;
-import org.apache.flink.configuration.description.InlineElement;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.table.factories.FactoryUtil;
-
-import java.time.Duration;
-import java.util.List;
-
-import static org.apache.flink.configuration.description.TextElement.text;
-import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX;
-
-/** Options for the Kafka connector. */
-@PublicEvolving
-public class KafkaConnectorOptions {
-
- // --------------------------------------------------------------------------------------------
- // Format options
- // --------------------------------------------------------------------------------------------
-
- public static final ConfigOption<String> KEY_FORMAT =
- ConfigOptions.key("key" + FORMAT_SUFFIX)
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Defines the format identifier for encoding key data. "
- + "The identifier is used to discover a suitable format factory.");
-
- public static final ConfigOption<String> VALUE_FORMAT =
- ConfigOptions.key("value" + FORMAT_SUFFIX)
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Defines the format identifier for encoding value data. "
- + "The identifier is used to discover a suitable format factory.");
-
- public static final ConfigOption<List<String>> KEY_FIELDS =
- ConfigOptions.key("key.fields")
- .stringType()
- .asList()
- .defaultValues()
- .withDescription(
- "Defines an explicit list of physical columns from the table schema "
- + "that configure the data type for the key format. By default, this list is "
- + "empty and thus a key is undefined.");
-
- public static final ConfigOption<ValueFieldsStrategy> VALUE_FIELDS_INCLUDE =
- ConfigOptions.key("value.fields-include")
- .enumType(ValueFieldsStrategy.class)
- .defaultValue(ValueFieldsStrategy.ALL)
- .withDescription(
- String.format(
- "Defines a strategy how to deal with key columns in the data type "
- + "of the value format. By default, '%s' physical columns of the table schema "
- + "will be included in the value format which means that the key columns "
- + "appear in the data type for both the key and value format.",
- ValueFieldsStrategy.ALL));
-
- public static final ConfigOption<String> KEY_FIELDS_PREFIX =
- ConfigOptions.key("key.fields-prefix")
- .stringType()
- .noDefaultValue()
- .withDescription(
- Description.builder()
- .text(
- "Defines a custom prefix for all fields of the key format to avoid "
- + "name clashes with fields of the value format. "
- + "By default, the prefix is empty.")
- .linebreak()
- .text(
- String.format(
- "If a custom prefix is defined, both the table schema and '%s' will work with prefixed names.",
- KEY_FIELDS.key()))
- .linebreak()
- .text(
- "When constructing the data type of the key format, the prefix "
- + "will be removed and the non-prefixed names will be used within the key format.")
- .linebreak()
- .text(
- String.format(
- "Please note that this option requires that '%s' must be '%s'.",
- VALUE_FIELDS_INCLUDE.key(),
- ValueFieldsStrategy.EXCEPT_KEY))
- .build());
-
- public static final ConfigOption<Integer> SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM;
-
- // --------------------------------------------------------------------------------------------
- // Kafka specific options
- // --------------------------------------------------------------------------------------------
-
- public static final ConfigOption<List<String>> TOPIC =
- ConfigOptions.key("topic")
- .stringType()
- .asList()
- .noDefaultValue()
- .withDescription(
- "Topic names from which the table is read. Either 'topic' or 'topic-pattern' must be set for source. "
- + "Option 'topic' is required for sink.");
-
- public static final ConfigOption<String> TOPIC_PATTERN =
- ConfigOptions.key("topic-pattern")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Optional topic pattern from which the table is read for source. Either 'topic' or 'topic-pattern' must be set.");
-
- public static final ConfigOption<String> PROPS_BOOTSTRAP_SERVERS =
- ConfigOptions.key("properties.bootstrap.servers")
- .stringType()
- .noDefaultValue()
- .withDescription("Required Kafka server connection string");
-
- public static final ConfigOption<String> PROPS_GROUP_ID =
- ConfigOptions.key("properties.group.id")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Required consumer group in Kafka consumer, no need for Kafka producer");
-
- // --------------------------------------------------------------------------------------------
- // Scan specific options
- // --------------------------------------------------------------------------------------------
-
- public static final ConfigOption<ScanStartupMode> SCAN_STARTUP_MODE =
- ConfigOptions.key("scan.startup.mode")
- .enumType(ScanStartupMode.class)
- .defaultValue(ScanStartupMode.GROUP_OFFSETS)
- .withDescription("Startup mode for Kafka consumer.");
-
- public static final ConfigOption<ScanBoundedMode> SCAN_BOUNDED_MODE =
- ConfigOptions.key("scan.bounded.mode")
- .enumType(ScanBoundedMode.class)
- .defaultValue(ScanBoundedMode.UNBOUNDED)
- .withDescription("Bounded mode for Kafka consumer.");
-
- public static final ConfigOption<String> SCAN_STARTUP_SPECIFIC_OFFSETS =
- ConfigOptions.key("scan.startup.specific-offsets")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Optional offsets used in case of \"specific-offsets\" startup mode");
-
- public static final ConfigOption<String> SCAN_BOUNDED_SPECIFIC_OFFSETS =
- ConfigOptions.key("scan.bounded.specific-offsets")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Optional offsets used in case of \"specific-offsets\" bounded mode");
-
- public static final ConfigOption<Long> SCAN_STARTUP_TIMESTAMP_MILLIS =
- ConfigOptions.key("scan.startup.timestamp-millis")
- .longType()
- .noDefaultValue()
- .withDescription(
- "Optional timestamp used in case of \"timestamp\" startup mode");
-
- public static final ConfigOption<Long> SCAN_BOUNDED_TIMESTAMP_MILLIS =
- ConfigOptions.key("scan.bounded.timestamp-millis")
- .longType()
- .noDefaultValue()
- .withDescription(
- "Optional timestamp used in case of \"timestamp\" bounded mode");
-
- public static final ConfigOption<Duration> SCAN_TOPIC_PARTITION_DISCOVERY =
- ConfigOptions.key("scan.topic-partition-discovery.interval")
- .durationType()
- .noDefaultValue()
- .withDescription(
- "Optional interval for consumer to discover dynamically created Kafka partitions periodically.");
-
- // --------------------------------------------------------------------------------------------
- // Sink specific options
- // --------------------------------------------------------------------------------------------
-
- public static final ConfigOption<String> SINK_PARTITIONER =
- ConfigOptions.key("sink.partitioner")
- .stringType()
- .defaultValue("default")
- .withDescription(
- Description.builder()
- .text(
- "Optional output partitioning from Flink's partitions into Kafka's partitions. Valid enumerations are")
- .list(
- text(
- "'default' (use kafka default partitioner to partition records)"),
- text(
- "'fixed' (each Flink partition ends up in at most one Kafka partition)"),
- text(
- "'round-robin' (a Flink partition is distributed to Kafka partitions round-robin when 'key.fields' is not specified)"),
- text(
- "custom class name (use custom FlinkKafkaPartitioner subclass)"))
- .build());
-
- // Disable this feature by default
- public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
- ConfigOptions.key("sink.buffer-flush.max-rows")
- .intType()
- .defaultValue(0)
- .withDescription(
- Description.builder()
- .text(
- "The max size of buffered records before flushing. "
- + "When the sink receives many updates on the same key, "
- + "the buffer will retain the last records of the same key. "
- + "This can help to reduce data shuffling and avoid possible tombstone messages to the Kafka topic.")
- .linebreak()
- .text("Can be set to '0' to disable it.")
- .linebreak()
- .text(
- "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' "
- + "must be set to be greater than zero to enable sink buffer flushing.")
- .build());
-
- // Disable this feature by default
- public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
- ConfigOptions.key("sink.buffer-flush.interval")
- .durationType()
- .defaultValue(Duration.ofSeconds(0))
- .withDescription(
- Description.builder()
- .text(
- "The flush interval millis. Over this time, asynchronous threads "
- + "will flush data. When the sink receives many updates on the same key, "
- + "the buffer will retain the last record of the same key.")
- .linebreak()
- .text("Can be set to '0' to disable it.")
- .linebreak()
- .text(
- "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' "
- + "must be set to be greater than zero to enable sink buffer flushing.")
- .build());
-
- public static final ConfigOption<DeliveryGuarantee> DELIVERY_GUARANTEE =
- ConfigOptions.key("sink.delivery-guarantee")
- .enumType(DeliveryGuarantee.class)
- .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE)
- .withDescription("Optional delivery guarantee when committing.");
-
- public static final ConfigOption<String> TRANSACTIONAL_ID_PREFIX =
- ConfigOptions.key("sink.transactional-id-prefix")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "If the delivery guarantee is configured as "
- + DeliveryGuarantee.EXACTLY_ONCE
- + " this value is used a prefix for the identifier of all opened Kafka transactions.");
-
- // --------------------------------------------------------------------------------------------
- // Enums
- // --------------------------------------------------------------------------------------------
-
- /** Strategies to derive the data type of a value format by considering a key format. */
- public enum ValueFieldsStrategy {
- ALL,
- EXCEPT_KEY
- }
-
- /** Startup mode for the Kafka consumer, see {@link #SCAN_STARTUP_MODE}. */
- public enum ScanStartupMode implements DescribedEnum {
- EARLIEST_OFFSET("earliest-offset", text("Start from the earliest offset possible.")),
- LATEST_OFFSET("latest-offset", text("Start from the latest offset.")),
- GROUP_OFFSETS(
- "group-offsets",
- text(
- "Start from committed offsets in ZooKeeper / Kafka brokers of a specific consumer group.")),
- TIMESTAMP("timestamp", text("Start from user-supplied timestamp for each partition.")),
- SPECIFIC_OFFSETS(
- "specific-offsets",
- text("Start from user-supplied specific offsets for each partition."));
-
- private final String value;
- private final InlineElement description;
-
- ScanStartupMode(String value, InlineElement description) {
- this.value = value;
- this.description = description;
- }
-
- @Override
- public String toString() {
- return value;
- }
-
- @Override
- public InlineElement getDescription() {
- return description;
- }
- }
-
- /** Bounded mode for the Kafka consumer, see {@link #SCAN_BOUNDED_MODE}. */
- public enum ScanBoundedMode implements DescribedEnum {
- UNBOUNDED("unbounded", text("Do not stop consuming")),
- LATEST_OFFSET(
- "latest-offset",
- text(
- "Bounded by latest offsets. This is evaluated at the start of consumption"
- + " from a given partition.")),
- GROUP_OFFSETS(
- "group-offsets",
- text(
- "Bounded by committed offsets in ZooKeeper / Kafka brokers of a specific"
- + " consumer group. This is evaluated at the start of consumption"
- + " from a given partition.")),
- TIMESTAMP("timestamp", text("Bounded by a user-supplied timestamp.")),
- SPECIFIC_OFFSETS(
- "specific-offsets",
- text(
- "Bounded by user-supplied specific offsets for each partition. If an offset"
- + " for a partition is not provided it will not consume from that"
- + " partition."));
- private final String value;
- private final InlineElement description;
-
- ScanBoundedMode(String value, InlineElement description) {
- this.value = value;
- this.description = description;
- }
-
- @Override
- public String toString() {
- return value;
- }
-
- @Override
- public InlineElement getDescription() {
- return description;
- }
- }
-
- private KafkaConnectorOptions() {}
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java
deleted file mode 100644
index ef70644..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java
+++ /dev/null
@@ -1,697 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode;
-import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode;
-import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ValueFieldsStrategy;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.DynamicTableFactory;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeRoot;
-import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
-import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.regex.Pattern;
-import java.util.stream.IntStream;
-
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT;
-import static org.apache.flink.table.factories.FactoryUtil.FORMAT;
-
-/** Utilities for {@link KafkaConnectorOptions}. */
-@Internal
-class KafkaConnectorOptionsUtil {
-
- private static final ConfigOption<String> SCHEMA_REGISTRY_SUBJECT =
- ConfigOptions.key("schema-registry.subject").stringType().noDefaultValue();
-
- // --------------------------------------------------------------------------------------------
- // Option enumerations
- // --------------------------------------------------------------------------------------------
-
- // Sink partitioner.
- public static final String SINK_PARTITIONER_VALUE_DEFAULT = "default";
- public static final String SINK_PARTITIONER_VALUE_FIXED = "fixed";
- public static final String SINK_PARTITIONER_VALUE_ROUND_ROBIN = "round-robin";
-
- // Prefix for Kafka specific properties.
- public static final String PROPERTIES_PREFIX = "properties.";
-
- // Other keywords.
- private static final String PARTITION = "partition";
- private static final String OFFSET = "offset";
- protected static final String AVRO_CONFLUENT = "avro-confluent";
- protected static final String DEBEZIUM_AVRO_CONFLUENT = "debezium-avro-confluent";
- private static final List<String> SCHEMA_REGISTRY_FORMATS =
- Arrays.asList(AVRO_CONFLUENT, DEBEZIUM_AVRO_CONFLUENT);
-
- // --------------------------------------------------------------------------------------------
- // Validation
- // --------------------------------------------------------------------------------------------
-
- public static void validateTableSourceOptions(ReadableConfig tableOptions) {
- validateSourceTopic(tableOptions);
- validateScanStartupMode(tableOptions);
- validateScanBoundedMode(tableOptions);
- }
-
- public static void validateTableSinkOptions(ReadableConfig tableOptions) {
- validateSinkTopic(tableOptions);
- validateSinkPartitioner(tableOptions);
- }
-
- public static void validateSourceTopic(ReadableConfig tableOptions) {
- Optional<List<String>> topic = tableOptions.getOptional(TOPIC);
- Optional<String> pattern = tableOptions.getOptional(TOPIC_PATTERN);
-
- if (topic.isPresent() && pattern.isPresent()) {
- throw new ValidationException(
- "Option 'topic' and 'topic-pattern' shouldn't be set together.");
- }
-
- if (!topic.isPresent() && !pattern.isPresent()) {
- throw new ValidationException("Either 'topic' or 'topic-pattern' must be set.");
- }
- }
-
- public static void validateSinkTopic(ReadableConfig tableOptions) {
- String errorMessageTemp =
- "Flink Kafka sink currently only supports single topic, but got %s: %s.";
- if (!isSingleTopic(tableOptions)) {
- if (tableOptions.getOptional(TOPIC_PATTERN).isPresent()) {
- throw new ValidationException(
- String.format(
- errorMessageTemp,
- "'topic-pattern'",
- tableOptions.get(TOPIC_PATTERN)));
- } else {
- throw new ValidationException(
- String.format(errorMessageTemp, "'topic'", tableOptions.get(TOPIC)));
- }
- }
- }
-
- private static void validateScanStartupMode(ReadableConfig tableOptions) {
- tableOptions
- .getOptional(SCAN_STARTUP_MODE)
- .ifPresent(
- mode -> {
- switch (mode) {
- case TIMESTAMP:
- if (!tableOptions
- .getOptional(SCAN_STARTUP_TIMESTAMP_MILLIS)
- .isPresent()) {
- throw new ValidationException(
- String.format(
- "'%s' is required in '%s' startup mode"
- + " but missing.",
- SCAN_STARTUP_TIMESTAMP_MILLIS.key(),
- ScanStartupMode.TIMESTAMP));
- }
-
- break;
- case SPECIFIC_OFFSETS:
- if (!tableOptions
- .getOptional(SCAN_STARTUP_SPECIFIC_OFFSETS)
- .isPresent()) {
- throw new ValidationException(
- String.format(
- "'%s' is required in '%s' startup mode"
- + " but missing.",
- SCAN_STARTUP_SPECIFIC_OFFSETS.key(),
- ScanStartupMode.SPECIFIC_OFFSETS));
- }
- if (!isSingleTopic(tableOptions)) {
- throw new ValidationException(
- "Currently Kafka source only supports specific offset for single topic.");
- }
- String specificOffsets =
- tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS);
- parseSpecificOffsets(
- specificOffsets, SCAN_STARTUP_SPECIFIC_OFFSETS.key());
-
- break;
- }
- });
- }
-
- private static void validateScanBoundedMode(ReadableConfig tableOptions) {
- tableOptions
- .getOptional(SCAN_BOUNDED_MODE)
- .ifPresent(
- mode -> {
- switch (mode) {
- case TIMESTAMP:
- if (!tableOptions
- .getOptional(SCAN_BOUNDED_TIMESTAMP_MILLIS)
- .isPresent()) {
- throw new ValidationException(
- String.format(
- "'%s' is required in '%s' bounded mode"
- + " but missing.",
- SCAN_BOUNDED_TIMESTAMP_MILLIS.key(),
- ScanBoundedMode.TIMESTAMP));
- }
-
- break;
- case SPECIFIC_OFFSETS:
- if (!tableOptions
- .getOptional(SCAN_BOUNDED_SPECIFIC_OFFSETS)
- .isPresent()) {
- throw new ValidationException(
- String.format(
- "'%s' is required in '%s' bounded mode"
- + " but missing.",
- SCAN_BOUNDED_SPECIFIC_OFFSETS.key(),
- ScanBoundedMode.SPECIFIC_OFFSETS));
- }
- if (!isSingleTopic(tableOptions)) {
- throw new ValidationException(
- "Currently Kafka source only supports specific offset for single topic.");
- }
- String specificOffsets =
- tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS);
- parseSpecificOffsets(
- specificOffsets, SCAN_BOUNDED_SPECIFIC_OFFSETS.key());
- break;
- }
- });
- }
-
- private static void validateSinkPartitioner(ReadableConfig tableOptions) {
- tableOptions
- .getOptional(SINK_PARTITIONER)
- .ifPresent(
- partitioner -> {
- if (partitioner.equals(SINK_PARTITIONER_VALUE_ROUND_ROBIN)
- && tableOptions.getOptional(KEY_FIELDS).isPresent()) {
- throw new ValidationException(
- "Currently 'round-robin' partitioner only works when option 'key.fields' is not specified.");
- } else if (partitioner.isEmpty()) {
- throw new ValidationException(
- String.format(
- "Option '%s' should be a non-empty string.",
- SINK_PARTITIONER.key()));
- }
- });
- }
-
- // --------------------------------------------------------------------------------------------
- // Utilities
- // --------------------------------------------------------------------------------------------
-
- public static List<String> getSourceTopics(ReadableConfig tableOptions) {
- return tableOptions.getOptional(TOPIC).orElse(null);
- }
-
- public static Pattern getSourceTopicPattern(ReadableConfig tableOptions) {
- return tableOptions.getOptional(TOPIC_PATTERN).map(Pattern::compile).orElse(null);
- }
-
- private static boolean isSingleTopic(ReadableConfig tableOptions) {
- // Option 'topic-pattern' is regarded as multi-topics.
- return tableOptions.getOptional(TOPIC).map(t -> t.size() == 1).orElse(false);
- }
-
- public static StartupOptions getStartupOptions(ReadableConfig tableOptions) {
- final Map<KafkaTopicPartition, Long> specificOffsets = new HashMap<>();
- final StartupMode startupMode =
- tableOptions
- .getOptional(SCAN_STARTUP_MODE)
- .map(KafkaConnectorOptionsUtil::fromOption)
- .orElse(StartupMode.GROUP_OFFSETS);
- if (startupMode == StartupMode.SPECIFIC_OFFSETS) {
- // It will be refactored after support specific offset for multiple topics in
- // FLINK-18602. We have already checked tableOptions.get(TOPIC) contains one topic in
- // validateScanStartupMode().
- buildSpecificOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets);
- }
-
- final StartupOptions options = new StartupOptions();
- options.startupMode = startupMode;
- options.specificOffsets = specificOffsets;
- if (startupMode == StartupMode.TIMESTAMP) {
- options.startupTimestampMillis = tableOptions.get(SCAN_STARTUP_TIMESTAMP_MILLIS);
- }
- return options;
- }
-
- public static BoundedOptions getBoundedOptions(ReadableConfig tableOptions) {
- final Map<KafkaTopicPartition, Long> specificOffsets = new HashMap<>();
- final BoundedMode boundedMode =
- KafkaConnectorOptionsUtil.fromOption(tableOptions.get(SCAN_BOUNDED_MODE));
- if (boundedMode == BoundedMode.SPECIFIC_OFFSETS) {
- buildBoundedOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets);
- }
-
- final BoundedOptions options = new BoundedOptions();
- options.boundedMode = boundedMode;
- options.specificOffsets = specificOffsets;
- if (boundedMode == BoundedMode.TIMESTAMP) {
- options.boundedTimestampMillis = tableOptions.get(SCAN_BOUNDED_TIMESTAMP_MILLIS);
- }
- return options;
- }
-
- private static void buildSpecificOffsets(
- ReadableConfig tableOptions,
- String topic,
- Map<KafkaTopicPartition, Long> specificOffsets) {
- String specificOffsetsStrOpt = tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS);
- final Map<Integer, Long> offsetMap =
- parseSpecificOffsets(specificOffsetsStrOpt, SCAN_STARTUP_SPECIFIC_OFFSETS.key());
- offsetMap.forEach(
- (partition, offset) -> {
- final KafkaTopicPartition topicPartition =
- new KafkaTopicPartition(topic, partition);
- specificOffsets.put(topicPartition, offset);
- });
- }
-
- public static void buildBoundedOffsets(
- ReadableConfig tableOptions,
- String topic,
- Map<KafkaTopicPartition, Long> specificOffsets) {
- String specificOffsetsEndOpt = tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS);
- final Map<Integer, Long> offsetMap =
- parseSpecificOffsets(specificOffsetsEndOpt, SCAN_BOUNDED_SPECIFIC_OFFSETS.key());
-
- offsetMap.forEach(
- (partition, offset) -> {
- final KafkaTopicPartition topicPartition =
- new KafkaTopicPartition(topic, partition);
- specificOffsets.put(topicPartition, offset);
- });
- }
-
- /**
- * Returns the {@link StartupMode} of Kafka Consumer by passed-in table-specific {@link
- * ScanStartupMode}.
- */
- private static StartupMode fromOption(ScanStartupMode scanStartupMode) {
- switch (scanStartupMode) {
- case EARLIEST_OFFSET:
- return StartupMode.EARLIEST;
- case LATEST_OFFSET:
- return StartupMode.LATEST;
- case GROUP_OFFSETS:
- return StartupMode.GROUP_OFFSETS;
- case SPECIFIC_OFFSETS:
- return StartupMode.SPECIFIC_OFFSETS;
- case TIMESTAMP:
- return StartupMode.TIMESTAMP;
-
- default:
- throw new TableException(
- "Unsupported startup mode. Validator should have checked that.");
- }
- }
-
- /**
- * Returns the {@link BoundedMode} of Kafka Consumer by passed-in table-specific {@link
- * ScanBoundedMode}.
- */
- private static BoundedMode fromOption(ScanBoundedMode scanBoundedMode) {
- switch (scanBoundedMode) {
- case UNBOUNDED:
- return BoundedMode.UNBOUNDED;
- case LATEST_OFFSET:
- return BoundedMode.LATEST;
- case GROUP_OFFSETS:
- return BoundedMode.GROUP_OFFSETS;
- case TIMESTAMP:
- return BoundedMode.TIMESTAMP;
- case SPECIFIC_OFFSETS:
- return BoundedMode.SPECIFIC_OFFSETS;
-
- default:
- throw new TableException(
- "Unsupported bounded mode. Validator should have checked that.");
- }
- }
-
- public static Properties getKafkaProperties(Map<String, String> tableOptions) {
- final Properties kafkaProperties = new Properties();
-
- if (hasKafkaClientProperties(tableOptions)) {
- tableOptions.keySet().stream()
- .filter(key -> key.startsWith(PROPERTIES_PREFIX))
- .forEach(
- key -> {
- final String value = tableOptions.get(key);
- final String subKey = key.substring((PROPERTIES_PREFIX).length());
- kafkaProperties.put(subKey, value);
- });
- }
- return kafkaProperties;
- }
-
- /**
- * The partitioner can be either "fixed", "round-robin" or a customized partitioner full class
- * name.
- */
- public static Optional<FlinkKafkaPartitioner<RowData>> getFlinkKafkaPartitioner(
- ReadableConfig tableOptions, ClassLoader classLoader) {
- return tableOptions
- .getOptional(SINK_PARTITIONER)
- .flatMap(
- (String partitioner) -> {
- switch (partitioner) {
- case SINK_PARTITIONER_VALUE_FIXED:
- return Optional.of(new FlinkFixedPartitioner<>());
- case SINK_PARTITIONER_VALUE_DEFAULT:
- case SINK_PARTITIONER_VALUE_ROUND_ROBIN:
- return Optional.empty();
- // Default fallback to full class name of the partitioner.
- default:
- return Optional.of(
- initializePartitioner(partitioner, classLoader));
- }
- });
- }
-
- /**
- * Parses specificOffsets String to Map.
- *
- * <p>specificOffsets String format was given as following:
- *
- * <pre>
- * scan.startup.specific-offsets = partition:0,offset:42;partition:1,offset:300
- * </pre>
- *
- * @return specificOffsets with Map format, key is partition, and value is offset
- */
- public static Map<Integer, Long> parseSpecificOffsets(
- String specificOffsetsStr, String optionKey) {
- final Map<Integer, Long> offsetMap = new HashMap<>();
- final String[] pairs = specificOffsetsStr.split(";");
- final String validationExceptionMessage =
- String.format(
- "Invalid properties '%s' should follow the format "
- + "'partition:0,offset:42;partition:1,offset:300', but is '%s'.",
- optionKey, specificOffsetsStr);
-
- if (pairs.length == 0) {
- throw new ValidationException(validationExceptionMessage);
- }
-
- for (String pair : pairs) {
- if (null == pair || pair.length() == 0 || !pair.contains(",")) {
- throw new ValidationException(validationExceptionMessage);
- }
-
- final String[] kv = pair.split(",");
- if (kv.length != 2
- || !kv[0].startsWith(PARTITION + ':')
- || !kv[1].startsWith(OFFSET + ':')) {
- throw new ValidationException(validationExceptionMessage);
- }
-
- String partitionValue = kv[0].substring(kv[0].indexOf(":") + 1);
- String offsetValue = kv[1].substring(kv[1].indexOf(":") + 1);
- try {
- final Integer partition = Integer.valueOf(partitionValue);
- final Long offset = Long.valueOf(offsetValue);
- offsetMap.put(partition, offset);
- } catch (NumberFormatException e) {
- throw new ValidationException(validationExceptionMessage, e);
- }
- }
- return offsetMap;
- }
-
- /**
- * Decides if the table options contains Kafka client properties that start with prefix
- * 'properties'.
- */
- private static boolean hasKafkaClientProperties(Map<String, String> tableOptions) {
- return tableOptions.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX));
- }
-
- /** Returns a class value with the given class name. */
- private static <T> FlinkKafkaPartitioner<T> initializePartitioner(
- String name, ClassLoader classLoader) {
- try {
- Class<?> clazz = Class.forName(name, true, classLoader);
- if (!FlinkKafkaPartitioner.class.isAssignableFrom(clazz)) {
- throw new ValidationException(
- String.format(
- "Sink partitioner class '%s' should extend from the required class %s",
- name, FlinkKafkaPartitioner.class.getName()));
- }
- @SuppressWarnings("unchecked")
- final FlinkKafkaPartitioner<T> kafkaPartitioner =
- InstantiationUtil.instantiate(name, FlinkKafkaPartitioner.class, classLoader);
-
- return kafkaPartitioner;
- } catch (ClassNotFoundException | FlinkException e) {
- throw new ValidationException(
- String.format("Could not find and instantiate partitioner class '%s'", name),
- e);
- }
- }
-
- /**
- * Creates an array of indices that determine which physical fields of the table schema to
- * include in the key format and the order that those fields have in the key format.
- *
- * <p>See {@link KafkaConnectorOptions#KEY_FORMAT}, {@link KafkaConnectorOptions#KEY_FIELDS},
- * and {@link KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information.
- */
- public static int[] createKeyFormatProjection(
- ReadableConfig options, DataType physicalDataType) {
- final LogicalType physicalType = physicalDataType.getLogicalType();
- Preconditions.checkArgument(
- physicalType.is(LogicalTypeRoot.ROW), "Row data type expected.");
- final Optional<String> optionalKeyFormat = options.getOptional(KEY_FORMAT);
- final Optional<List<String>> optionalKeyFields = options.getOptional(KEY_FIELDS);
-
- if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) {
- throw new ValidationException(
- String.format(
- "The option '%s' can only be declared if a key format is defined using '%s'.",
- KEY_FIELDS.key(), KEY_FORMAT.key()));
- } else if (optionalKeyFormat.isPresent()
- && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) {
- throw new ValidationException(
- String.format(
- "A key format '%s' requires the declaration of one or more of key fields using '%s'.",
- KEY_FORMAT.key(), KEY_FIELDS.key()));
- }
-
- if (!optionalKeyFormat.isPresent()) {
- return new int[0];
- }
-
- final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse("");
-
- final List<String> keyFields = optionalKeyFields.get();
- final List<String> physicalFields = LogicalTypeChecks.getFieldNames(physicalType);
- return keyFields.stream()
- .mapToInt(
- keyField -> {
- final int pos = physicalFields.indexOf(keyField);
- // check that field name exists
- if (pos < 0) {
- throw new ValidationException(
- String.format(
- "Could not find the field '%s' in the table schema for usage in the key format. "
- + "A key field must be a regular, physical column. "
- + "The following columns can be selected in the '%s' option:\n"
- + "%s",
- keyField, KEY_FIELDS.key(), physicalFields));
- }
- // check that field name is prefixed correctly
- if (!keyField.startsWith(keyPrefix)) {
- throw new ValidationException(
- String.format(
- "All fields in '%s' must be prefixed with '%s' when option '%s' "
- + "is set but field '%s' is not prefixed.",
- KEY_FIELDS.key(),
- keyPrefix,
- KEY_FIELDS_PREFIX.key(),
- keyField));
- }
- return pos;
- })
- .toArray();
- }
-
- /**
- * Creates an array of indices that determine which physical fields of the table schema to
- * include in the value format.
- *
- * <p>See {@link KafkaConnectorOptions#VALUE_FORMAT}, {@link
- * KafkaConnectorOptions#VALUE_FIELDS_INCLUDE}, and {@link
- * KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information.
- */
- public static int[] createValueFormatProjection(
- ReadableConfig options, DataType physicalDataType) {
- final LogicalType physicalType = physicalDataType.getLogicalType();
- Preconditions.checkArgument(
- physicalType.is(LogicalTypeRoot.ROW), "Row data type expected.");
- final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType);
- final IntStream physicalFields = IntStream.range(0, physicalFieldCount);
-
- final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse("");
-
- final ValueFieldsStrategy strategy = options.get(VALUE_FIELDS_INCLUDE);
- if (strategy == ValueFieldsStrategy.ALL) {
- if (keyPrefix.length() > 0) {
- throw new ValidationException(
- String.format(
- "A key prefix is not allowed when option '%s' is set to '%s'. "
- + "Set it to '%s' instead to avoid field overlaps.",
- VALUE_FIELDS_INCLUDE.key(),
- ValueFieldsStrategy.ALL,
- ValueFieldsStrategy.EXCEPT_KEY));
- }
- return physicalFields.toArray();
- } else if (strategy == ValueFieldsStrategy.EXCEPT_KEY) {
- final int[] keyProjection = createKeyFormatProjection(options, physicalDataType);
- return physicalFields
- .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos))
- .toArray();
- }
- throw new TableException("Unknown value fields strategy:" + strategy);
- }
-
- /**
- * Returns a new table context with a default schema registry subject value in the options if
- * the format is a schema registry format (e.g. 'avro-confluent') and the subject is not
- * defined.
- */
- public static DynamicTableFactory.Context autoCompleteSchemaRegistrySubject(
- DynamicTableFactory.Context context) {
- Map<String, String> tableOptions = context.getCatalogTable().getOptions();
- Map<String, String> newOptions = autoCompleteSchemaRegistrySubject(tableOptions);
- if (newOptions.size() > tableOptions.size()) {
- // build a new context
- return new FactoryUtil.DefaultDynamicTableContext(
- context.getObjectIdentifier(),
- context.getCatalogTable().copy(newOptions),
- context.getEnrichmentOptions(),
- context.getConfiguration(),
- context.getClassLoader(),
- context.isTemporary());
- } else {
- return context;
- }
- }
-
- private static Map<String, String> autoCompleteSchemaRegistrySubject(
- Map<String, String> options) {
- Configuration configuration = Configuration.fromMap(options);
- // the subject autoComplete should only be used in sink, check the topic first
- validateSinkTopic(configuration);
- final Optional<String> valueFormat = configuration.getOptional(VALUE_FORMAT);
- final Optional<String> keyFormat = configuration.getOptional(KEY_FORMAT);
- final Optional<String> format = configuration.getOptional(FORMAT);
- final String topic = configuration.get(TOPIC).get(0);
-
- if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) {
- autoCompleteSubject(configuration, format.get(), topic + "-value");
- } else if (valueFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) {
- autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value");
- }
-
- if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) {
- autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key");
- }
- return configuration.toMap();
- }
-
- private static void autoCompleteSubject(
- Configuration configuration, String format, String subject) {
- ConfigOption<String> subjectOption =
- ConfigOptions.key(format + "." + SCHEMA_REGISTRY_SUBJECT.key())
- .stringType()
- .noDefaultValue();
- if (!configuration.getOptional(subjectOption).isPresent()) {
- configuration.setString(subjectOption, subject);
- }
- }
-
- static void validateDeliveryGuarantee(ReadableConfig tableOptions) {
- if (tableOptions.get(DELIVERY_GUARANTEE) == DeliveryGuarantee.EXACTLY_ONCE
- && !tableOptions.getOptional(TRANSACTIONAL_ID_PREFIX).isPresent()) {
- throw new ValidationException(
- TRANSACTIONAL_ID_PREFIX.key()
- + " must be specified when using DeliveryGuarantee.EXACTLY_ONCE.");
- }
- }
-
- // --------------------------------------------------------------------------------------------
- // Inner classes
- // --------------------------------------------------------------------------------------------
-
- /** Kafka startup options. * */
- public static class StartupOptions {
- public StartupMode startupMode;
- public Map<KafkaTopicPartition, Long> specificOffsets;
- public long startupTimestampMillis;
- }
-
- /** Kafka bounded options. * */
- public static class BoundedOptions {
- public BoundedMode boundedMode;
- public Map<KafkaTopicPartition, Long> specificOffsets;
- public long boundedTimestampMillis;
- }
-
- private KafkaConnectorOptionsUtil() {}
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java
deleted file mode 100644
index 041846d..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java
+++ /dev/null
@@ -1,479 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.ProviderContext;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkV2Provider;
-import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata;
-import org.apache.flink.table.data.ArrayData;
-import org.apache.flink.table.data.MapData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.header.Header;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.stream.Stream;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** A version-agnostic Kafka {@link DynamicTableSink}. */
-@Internal
-public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetadata {
-
- private static final String UPSERT_KAFKA_TRANSFORMATION = "upsert-kafka";
-
- // --------------------------------------------------------------------------------------------
- // Mutable attributes
- // --------------------------------------------------------------------------------------------
-
- /** Metadata that is appended at the end of a physical sink row. */
- protected List<String> metadataKeys;
-
- // --------------------------------------------------------------------------------------------
- // Format attributes
- // --------------------------------------------------------------------------------------------
-
- /** Data type of consumed data type. */
- protected DataType consumedDataType;
-
- /** Data type to configure the formats. */
- protected final DataType physicalDataType;
-
- /** Optional format for encoding keys to Kafka. */
- protected final @Nullable EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat;
-
- /** Format for encoding values to Kafka. */
- protected final EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat;
-
- /** Indices that determine the key fields and the source position in the consumed row. */
- protected final int[] keyProjection;
-
- /** Indices that determine the value fields and the source position in the consumed row. */
- protected final int[] valueProjection;
-
- /** Prefix that needs to be removed from fields when constructing the physical data type. */
- protected final @Nullable String keyPrefix;
-
- // --------------------------------------------------------------------------------------------
- // Kafka-specific attributes
- // --------------------------------------------------------------------------------------------
-
- /** The defined delivery guarantee. */
- private final DeliveryGuarantee deliveryGuarantee;
-
- /**
- * If the {@link #deliveryGuarantee} is {@link DeliveryGuarantee#EXACTLY_ONCE} the value is the
- * prefix for all ids of opened Kafka transactions.
- */
- @Nullable private final String transactionalIdPrefix;
-
- /** The Kafka topic to write to. */
- protected final String topic;
-
- /** Properties for the Kafka producer. */
- protected final Properties properties;
-
- /** Partitioner to select Kafka partition for each item. */
- protected final @Nullable FlinkKafkaPartitioner<RowData> partitioner;
-
- /**
- * Flag to determine sink mode. In upsert mode sink transforms the delete/update-before message
- * to tombstone message.
- */
- protected final boolean upsertMode;
-
- /** Sink buffer flush config which only supported in upsert mode now. */
- protected final SinkBufferFlushMode flushMode;
-
- /** Parallelism of the physical Kafka producer. * */
- protected final @Nullable Integer parallelism;
-
- public KafkaDynamicSink(
- DataType consumedDataType,
- DataType physicalDataType,
- @Nullable EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat,
- EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- String topic,
- Properties properties,
- @Nullable FlinkKafkaPartitioner<RowData> partitioner,
- DeliveryGuarantee deliveryGuarantee,
- boolean upsertMode,
- SinkBufferFlushMode flushMode,
- @Nullable Integer parallelism,
- @Nullable String transactionalIdPrefix) {
- // Format attributes
- this.consumedDataType =
- checkNotNull(consumedDataType, "Consumed data type must not be null.");
- this.physicalDataType =
- checkNotNull(physicalDataType, "Physical data type must not be null.");
- this.keyEncodingFormat = keyEncodingFormat;
- this.valueEncodingFormat =
- checkNotNull(valueEncodingFormat, "Value encoding format must not be null.");
- this.keyProjection = checkNotNull(keyProjection, "Key projection must not be null.");
- this.valueProjection = checkNotNull(valueProjection, "Value projection must not be null.");
- this.keyPrefix = keyPrefix;
- this.transactionalIdPrefix = transactionalIdPrefix;
- // Mutable attributes
- this.metadataKeys = Collections.emptyList();
- // Kafka-specific attributes
- this.topic = checkNotNull(topic, "Topic must not be null.");
- this.properties = checkNotNull(properties, "Properties must not be null.");
- this.partitioner = partitioner;
- this.deliveryGuarantee =
- checkNotNull(deliveryGuarantee, "DeliveryGuarantee must not be null.");
- this.upsertMode = upsertMode;
- this.flushMode = checkNotNull(flushMode);
- if (flushMode.isEnabled() && !upsertMode) {
- throw new IllegalArgumentException(
- "Sink buffer flush is only supported in upsert-kafka.");
- }
- this.parallelism = parallelism;
- }
-
- @Override
- public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
- return valueEncodingFormat.getChangelogMode();
- }
-
- @Override
- public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
- final SerializationSchema<RowData> keySerialization =
- createSerialization(context, keyEncodingFormat, keyProjection, keyPrefix);
-
- final SerializationSchema<RowData> valueSerialization =
- createSerialization(context, valueEncodingFormat, valueProjection, null);
-
- final KafkaSinkBuilder<RowData> sinkBuilder = KafkaSink.builder();
- final List<LogicalType> physicalChildren = physicalDataType.getLogicalType().getChildren();
- if (transactionalIdPrefix != null) {
- sinkBuilder.setTransactionalIdPrefix(transactionalIdPrefix);
- }
- final KafkaSink<RowData> kafkaSink =
- sinkBuilder
- .setDeliveryGuarantee(deliveryGuarantee)
- .setBootstrapServers(
- properties.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG).toString())
- .setKafkaProducerConfig(properties)
- .setRecordSerializer(
- new DynamicKafkaRecordSerializationSchema(
- topic,
- partitioner,
- keySerialization,
- valueSerialization,
- getFieldGetters(physicalChildren, keyProjection),
- getFieldGetters(physicalChildren, valueProjection),
- hasMetadata(),
- getMetadataPositions(physicalChildren),
- upsertMode))
- .build();
- if (flushMode.isEnabled() && upsertMode) {
- return new DataStreamSinkProvider() {
- @Override
- public DataStreamSink<?> consumeDataStream(
- ProviderContext providerContext, DataStream<RowData> dataStream) {
- final boolean objectReuse =
- dataStream.getExecutionEnvironment().getConfig().isObjectReuseEnabled();
- final ReducingUpsertSink<?> sink =
- new ReducingUpsertSink<>(
- kafkaSink,
- physicalDataType,
- keyProjection,
- flushMode,
- objectReuse
- ? createRowDataTypeSerializer(
- context,
- dataStream.getExecutionConfig())
- ::copy
- : rowData -> rowData);
- final DataStreamSink<RowData> end = dataStream.sinkTo(sink);
- providerContext.generateUid(UPSERT_KAFKA_TRANSFORMATION).ifPresent(end::uid);
- if (parallelism != null) {
- end.setParallelism(parallelism);
- }
- return end;
- }
- };
- }
- return SinkV2Provider.of(kafkaSink, parallelism);
- }
-
- @Override
- public Map<String, DataType> listWritableMetadata() {
- final Map<String, DataType> metadataMap = new LinkedHashMap<>();
- Stream.of(WritableMetadata.values())
- .forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
- return metadataMap;
- }
-
- @Override
- public void applyWritableMetadata(List<String> metadataKeys, DataType consumedDataType) {
- this.metadataKeys = metadataKeys;
- this.consumedDataType = consumedDataType;
- }
-
- @Override
- public DynamicTableSink copy() {
- final KafkaDynamicSink copy =
- new KafkaDynamicSink(
- consumedDataType,
- physicalDataType,
- keyEncodingFormat,
- valueEncodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topic,
- properties,
- partitioner,
- deliveryGuarantee,
- upsertMode,
- flushMode,
- parallelism,
- transactionalIdPrefix);
- copy.metadataKeys = metadataKeys;
- return copy;
- }
-
- @Override
- public String asSummaryString() {
- return "Kafka table sink";
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- final KafkaDynamicSink that = (KafkaDynamicSink) o;
- return Objects.equals(metadataKeys, that.metadataKeys)
- && Objects.equals(consumedDataType, that.consumedDataType)
- && Objects.equals(physicalDataType, that.physicalDataType)
- && Objects.equals(keyEncodingFormat, that.keyEncodingFormat)
- && Objects.equals(valueEncodingFormat, that.valueEncodingFormat)
- && Arrays.equals(keyProjection, that.keyProjection)
- && Arrays.equals(valueProjection, that.valueProjection)
- && Objects.equals(keyPrefix, that.keyPrefix)
- && Objects.equals(topic, that.topic)
- && Objects.equals(properties, that.properties)
- && Objects.equals(partitioner, that.partitioner)
- && Objects.equals(deliveryGuarantee, that.deliveryGuarantee)
- && Objects.equals(upsertMode, that.upsertMode)
- && Objects.equals(flushMode, that.flushMode)
- && Objects.equals(transactionalIdPrefix, that.transactionalIdPrefix)
- && Objects.equals(parallelism, that.parallelism);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(
- metadataKeys,
- consumedDataType,
- physicalDataType,
- keyEncodingFormat,
- valueEncodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topic,
- properties,
- partitioner,
- deliveryGuarantee,
- upsertMode,
- flushMode,
- transactionalIdPrefix,
- parallelism);
- }
-
- // --------------------------------------------------------------------------------------------
-
- private TypeSerializer<RowData> createRowDataTypeSerializer(
- Context context, ExecutionConfig executionConfig) {
- final TypeInformation<RowData> typeInformation =
- context.createTypeInformation(consumedDataType);
- return typeInformation.createSerializer(executionConfig);
- }
-
- private int[] getMetadataPositions(List<LogicalType> physicalChildren) {
- return Stream.of(WritableMetadata.values())
- .mapToInt(
- m -> {
- final int pos = metadataKeys.indexOf(m.key);
- if (pos < 0) {
- return -1;
- }
- return physicalChildren.size() + pos;
- })
- .toArray();
- }
-
- private boolean hasMetadata() {
- return metadataKeys.size() > 0;
- }
-
- private RowData.FieldGetter[] getFieldGetters(
- List<LogicalType> physicalChildren, int[] keyProjection) {
- return Arrays.stream(keyProjection)
- .mapToObj(
- targetField ->
- RowData.createFieldGetter(
- physicalChildren.get(targetField), targetField))
- .toArray(RowData.FieldGetter[]::new);
- }
-
- private @Nullable SerializationSchema<RowData> createSerialization(
- DynamicTableSink.Context context,
- @Nullable EncodingFormat<SerializationSchema<RowData>> format,
- int[] projection,
- @Nullable String prefix) {
- if (format == null) {
- return null;
- }
- DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType);
- if (prefix != null) {
- physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix);
- }
- return format.createRuntimeEncoder(context, physicalFormatDataType);
- }
-
- // --------------------------------------------------------------------------------------------
- // Metadata handling
- // --------------------------------------------------------------------------------------------
-
- enum WritableMetadata {
- HEADERS(
- "headers",
- // key and value of the map are nullable to make handling easier in queries
- DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable())
- .nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(RowData row, int pos) {
- if (row.isNullAt(pos)) {
- return null;
- }
- final MapData map = row.getMap(pos);
- final ArrayData keyArray = map.keyArray();
- final ArrayData valueArray = map.valueArray();
- final List<Header> headers = new ArrayList<>();
- for (int i = 0; i < keyArray.size(); i++) {
- if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) {
- final String key = keyArray.getString(i).toString();
- final byte[] value = valueArray.getBinary(i);
- headers.add(new KafkaHeader(key, value));
- }
- }
- return headers;
- }
- }),
-
- TIMESTAMP(
- "timestamp",
- DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(RowData row, int pos) {
- if (row.isNullAt(pos)) {
- return null;
- }
- return row.getTimestamp(pos, 3).getMillisecond();
- }
- });
-
- final String key;
-
- final DataType dataType;
-
- final MetadataConverter converter;
-
- WritableMetadata(String key, DataType dataType, MetadataConverter converter) {
- this.key = key;
- this.dataType = dataType;
- this.converter = converter;
- }
- }
-
- interface MetadataConverter extends Serializable {
- Object read(RowData consumedRow, int pos);
- }
-
- // --------------------------------------------------------------------------------------------
-
- private static class KafkaHeader implements Header {
-
- private final String key;
-
- private final byte[] value;
-
- KafkaHeader(String key, byte[] value) {
- this.key = key;
- this.value = value;
- }
-
- @Override
- public String key() {
- return key;
- }
-
- @Override
- public byte[] value() {
- return value;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java
deleted file mode 100644
index c963da7..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java
+++ /dev/null
@@ -1,678 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.connector.kafka.source.enumerator.initializer.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.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema.MetadataConverter;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.ProviderContext;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.source.DataStreamScanProvider;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.ScanTableSource;
-import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
-import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown;
-import org.apache.flink.table.data.GenericMapData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.utils.DataTypeUtils;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.header.Header;
-
-import javax.annotation.Nullable;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-import java.util.stream.Stream;
-
-/** A version-agnostic Kafka {@link ScanTableSource}. */
-@Internal
-public class KafkaDynamicSource
- implements ScanTableSource, SupportsReadingMetadata, SupportsWatermarkPushDown {
-
- private static final String KAFKA_TRANSFORMATION = "kafka";
-
- // --------------------------------------------------------------------------------------------
- // Mutable attributes
- // --------------------------------------------------------------------------------------------
-
- /** Data type that describes the final output of the source. */
- protected DataType producedDataType;
-
- /** Metadata that is appended at the end of a physical source row. */
- protected List<String> metadataKeys;
-
- /** Watermark strategy that is used to generate per-partition watermark. */
- protected @Nullable WatermarkStrategy<RowData> watermarkStrategy;
-
- // --------------------------------------------------------------------------------------------
- // Format attributes
- // --------------------------------------------------------------------------------------------
-
- private static final String VALUE_METADATA_PREFIX = "value.";
-
- /** Data type to configure the formats. */
- protected final DataType physicalDataType;
-
- /** Optional format for decoding keys from Kafka. */
- protected final @Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat;
-
- /** Format for decoding values from Kafka. */
- protected final DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat;
-
- /** Indices that determine the key fields and the target position in the produced row. */
- protected final int[] keyProjection;
-
- /** Indices that determine the value fields and the target position in the produced row. */
- protected final int[] valueProjection;
-
- /** Prefix that needs to be removed from fields when constructing the physical data type. */
- protected final @Nullable String keyPrefix;
-
- // --------------------------------------------------------------------------------------------
- // Kafka-specific attributes
- // --------------------------------------------------------------------------------------------
-
- /** The Kafka topics to consume. */
- protected final List<String> topics;
-
- /** The Kafka topic pattern to consume. */
- protected final Pattern topicPattern;
-
- /** Properties for the Kafka consumer. */
- protected final Properties properties;
-
- /**
- * The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}).
- */
- protected final StartupMode startupMode;
-
- /**
- * Specific startup offsets; only relevant when startup mode is {@link
- * StartupMode#SPECIFIC_OFFSETS}.
- */
- protected final Map<KafkaTopicPartition, Long> specificStartupOffsets;
-
- /**
- * The start timestamp to locate partition offsets; only relevant when startup mode is {@link
- * StartupMode#TIMESTAMP}.
- */
- protected final long startupTimestampMillis;
-
- /** The bounded mode for the contained consumer (default is an unbounded data stream). */
- protected final BoundedMode boundedMode;
-
- /**
- * Specific end offsets; only relevant when bounded mode is {@link
- * BoundedMode#SPECIFIC_OFFSETS}.
- */
- protected final Map<KafkaTopicPartition, Long> specificBoundedOffsets;
-
- /**
- * The bounded timestamp to locate partition offsets; only relevant when bounded mode is {@link
- * BoundedMode#TIMESTAMP}.
- */
- protected final long boundedTimestampMillis;
-
- /** Flag to determine source mode. In upsert mode, it will keep the tombstone message. * */
- protected final boolean upsertMode;
-
- protected final String tableIdentifier;
-
- public KafkaDynamicSource(
- DataType physicalDataType,
- @Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- @Nullable List<String> topics,
- @Nullable Pattern topicPattern,
- Properties properties,
- StartupMode startupMode,
- Map<KafkaTopicPartition, Long> specificStartupOffsets,
- long startupTimestampMillis,
- BoundedMode boundedMode,
- Map<KafkaTopicPartition, Long> specificBoundedOffsets,
- long boundedTimestampMillis,
- boolean upsertMode,
- String tableIdentifier) {
- // Format attributes
- this.physicalDataType =
- Preconditions.checkNotNull(
- physicalDataType, "Physical data type must not be null.");
- this.keyDecodingFormat = keyDecodingFormat;
- this.valueDecodingFormat =
- Preconditions.checkNotNull(
- valueDecodingFormat, "Value decoding format must not be null.");
- this.keyProjection =
- Preconditions.checkNotNull(keyProjection, "Key projection must not be null.");
- this.valueProjection =
- Preconditions.checkNotNull(valueProjection, "Value projection must not be null.");
- this.keyPrefix = keyPrefix;
- // Mutable attributes
- this.producedDataType = physicalDataType;
- this.metadataKeys = Collections.emptyList();
- this.watermarkStrategy = null;
- // Kafka-specific attributes
- Preconditions.checkArgument(
- (topics != null && topicPattern == null)
- || (topics == null && topicPattern != null),
- "Either Topic or Topic Pattern must be set for source.");
- this.topics = topics;
- this.topicPattern = topicPattern;
- this.properties = Preconditions.checkNotNull(properties, "Properties must not be null.");
- this.startupMode =
- Preconditions.checkNotNull(startupMode, "Startup mode must not be null.");
- this.specificStartupOffsets =
- Preconditions.checkNotNull(
- specificStartupOffsets, "Specific offsets must not be null.");
- this.startupTimestampMillis = startupTimestampMillis;
- this.boundedMode =
- Preconditions.checkNotNull(boundedMode, "Bounded mode must not be null.");
- this.specificBoundedOffsets =
- Preconditions.checkNotNull(
- specificBoundedOffsets, "Specific bounded offsets must not be null.");
- this.boundedTimestampMillis = boundedTimestampMillis;
- this.upsertMode = upsertMode;
- this.tableIdentifier = tableIdentifier;
- }
-
- @Override
- public ChangelogMode getChangelogMode() {
- return valueDecodingFormat.getChangelogMode();
- }
-
- @Override
- public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
- final DeserializationSchema<RowData> keyDeserialization =
- createDeserialization(context, keyDecodingFormat, keyProjection, keyPrefix);
-
- final DeserializationSchema<RowData> valueDeserialization =
- createDeserialization(context, valueDecodingFormat, valueProjection, null);
-
- final TypeInformation<RowData> producedTypeInfo =
- context.createTypeInformation(producedDataType);
-
- final KafkaSource<RowData> kafkaSource =
- createKafkaSource(keyDeserialization, valueDeserialization, producedTypeInfo);
-
- return new DataStreamScanProvider() {
- @Override
- public DataStream<RowData> produceDataStream(
- ProviderContext providerContext, StreamExecutionEnvironment execEnv) {
- if (watermarkStrategy == null) {
- watermarkStrategy = WatermarkStrategy.noWatermarks();
- }
- DataStreamSource<RowData> sourceStream =
- execEnv.fromSource(
- kafkaSource, watermarkStrategy, "KafkaSource-" + tableIdentifier);
- providerContext.generateUid(KAFKA_TRANSFORMATION).ifPresent(sourceStream::uid);
- return sourceStream;
- }
-
- @Override
- public boolean isBounded() {
- return kafkaSource.getBoundedness() == Boundedness.BOUNDED;
- }
- };
- }
-
- @Override
- public Map<String, DataType> listReadableMetadata() {
- final Map<String, DataType> metadataMap = new LinkedHashMap<>();
-
- // according to convention, the order of the final row must be
- // PHYSICAL + FORMAT METADATA + CONNECTOR METADATA
- // where the format metadata has highest precedence
-
- // add value format metadata with prefix
- valueDecodingFormat
- .listReadableMetadata()
- .forEach((key, value) -> metadataMap.put(VALUE_METADATA_PREFIX + key, value));
-
- // add connector metadata
- Stream.of(ReadableMetadata.values())
- .forEachOrdered(m -> metadataMap.putIfAbsent(m.key, m.dataType));
-
- return metadataMap;
- }
-
- @Override
- public void applyReadableMetadata(List<String> metadataKeys, DataType producedDataType) {
- // separate connector and format metadata
- final List<String> formatMetadataKeys =
- metadataKeys.stream()
- .filter(k -> k.startsWith(VALUE_METADATA_PREFIX))
- .collect(Collectors.toList());
- final List<String> connectorMetadataKeys = new ArrayList<>(metadataKeys);
- connectorMetadataKeys.removeAll(formatMetadataKeys);
-
- // push down format metadata
- final Map<String, DataType> formatMetadata = valueDecodingFormat.listReadableMetadata();
- if (formatMetadata.size() > 0) {
- final List<String> requestedFormatMetadataKeys =
- formatMetadataKeys.stream()
- .map(k -> k.substring(VALUE_METADATA_PREFIX.length()))
- .collect(Collectors.toList());
- valueDecodingFormat.applyReadableMetadata(requestedFormatMetadataKeys);
- }
-
- this.metadataKeys = connectorMetadataKeys;
- this.producedDataType = producedDataType;
- }
-
- @Override
- public boolean supportsMetadataProjection() {
- return false;
- }
-
- @Override
- public void applyWatermark(WatermarkStrategy<RowData> watermarkStrategy) {
- this.watermarkStrategy = watermarkStrategy;
- }
-
- @Override
- public DynamicTableSource copy() {
- final KafkaDynamicSource copy =
- new KafkaDynamicSource(
- physicalDataType,
- keyDecodingFormat,
- valueDecodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topics,
- topicPattern,
- properties,
- startupMode,
- specificStartupOffsets,
- startupTimestampMillis,
- boundedMode,
- specificBoundedOffsets,
- boundedTimestampMillis,
- upsertMode,
- tableIdentifier);
- copy.producedDataType = producedDataType;
- copy.metadataKeys = metadataKeys;
- copy.watermarkStrategy = watermarkStrategy;
- return copy;
- }
-
- @Override
- public String asSummaryString() {
- return "Kafka table source";
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- final KafkaDynamicSource that = (KafkaDynamicSource) o;
- return Objects.equals(producedDataType, that.producedDataType)
- && Objects.equals(metadataKeys, that.metadataKeys)
- && Objects.equals(physicalDataType, that.physicalDataType)
- && Objects.equals(keyDecodingFormat, that.keyDecodingFormat)
- && Objects.equals(valueDecodingFormat, that.valueDecodingFormat)
- && Arrays.equals(keyProjection, that.keyProjection)
- && Arrays.equals(valueProjection, that.valueProjection)
- && Objects.equals(keyPrefix, that.keyPrefix)
- && Objects.equals(topics, that.topics)
- && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern))
- && Objects.equals(properties, that.properties)
- && startupMode == that.startupMode
- && Objects.equals(specificStartupOffsets, that.specificStartupOffsets)
- && startupTimestampMillis == that.startupTimestampMillis
- && boundedMode == that.boundedMode
- && Objects.equals(specificBoundedOffsets, that.specificBoundedOffsets)
- && boundedTimestampMillis == that.boundedTimestampMillis
- && Objects.equals(upsertMode, that.upsertMode)
- && Objects.equals(tableIdentifier, that.tableIdentifier)
- && Objects.equals(watermarkStrategy, that.watermarkStrategy);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(
- producedDataType,
- metadataKeys,
- physicalDataType,
- keyDecodingFormat,
- valueDecodingFormat,
- Arrays.hashCode(keyProjection),
- Arrays.hashCode(valueProjection),
- keyPrefix,
- topics,
- topicPattern,
- properties,
- startupMode,
- specificStartupOffsets,
- startupTimestampMillis,
- boundedMode,
- specificBoundedOffsets,
- boundedTimestampMillis,
- upsertMode,
- tableIdentifier,
- watermarkStrategy);
- }
-
- // --------------------------------------------------------------------------------------------
-
- protected KafkaSource<RowData> createKafkaSource(
- DeserializationSchema<RowData> keyDeserialization,
- DeserializationSchema<RowData> valueDeserialization,
- TypeInformation<RowData> producedTypeInfo) {
-
- final KafkaDeserializationSchema<RowData> kafkaDeserializer =
- createKafkaDeserializationSchema(
- keyDeserialization, valueDeserialization, producedTypeInfo);
-
- final KafkaSourceBuilder<RowData> kafkaSourceBuilder = KafkaSource.builder();
-
- if (topics != null) {
- kafkaSourceBuilder.setTopics(topics);
- } else {
- kafkaSourceBuilder.setTopicPattern(topicPattern);
- }
-
- switch (startupMode) {
- case EARLIEST:
- kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.earliest());
- break;
- case LATEST:
- kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.latest());
- break;
- case GROUP_OFFSETS:
- String offsetResetConfig =
- properties.getProperty(
- ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
- OffsetResetStrategy.NONE.name());
- OffsetResetStrategy offsetResetStrategy = getResetStrategy(offsetResetConfig);
- kafkaSourceBuilder.setStartingOffsets(
- OffsetsInitializer.committedOffsets(offsetResetStrategy));
- break;
- case SPECIFIC_OFFSETS:
- Map<TopicPartition, Long> offsets = new HashMap<>();
- specificStartupOffsets.forEach(
- (tp, offset) ->
- offsets.put(
- new TopicPartition(tp.getTopic(), tp.getPartition()),
- offset));
- kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.offsets(offsets));
- break;
- case TIMESTAMP:
- kafkaSourceBuilder.setStartingOffsets(
- OffsetsInitializer.timestamp(startupTimestampMillis));
- break;
- }
-
- switch (boundedMode) {
- case UNBOUNDED:
- kafkaSourceBuilder.setUnbounded(new NoStoppingOffsetsInitializer());
- break;
- case LATEST:
- kafkaSourceBuilder.setBounded(OffsetsInitializer.latest());
- break;
- case GROUP_OFFSETS:
- kafkaSourceBuilder.setBounded(OffsetsInitializer.committedOffsets());
- break;
- case SPECIFIC_OFFSETS:
- Map<TopicPartition, Long> offsets = new HashMap<>();
- specificBoundedOffsets.forEach(
- (tp, offset) ->
- offsets.put(
- new TopicPartition(tp.getTopic(), tp.getPartition()),
- offset));
- kafkaSourceBuilder.setBounded(OffsetsInitializer.offsets(offsets));
- break;
- case TIMESTAMP:
- kafkaSourceBuilder.setBounded(OffsetsInitializer.timestamp(boundedTimestampMillis));
- break;
- }
-
- kafkaSourceBuilder
- .setProperties(properties)
- .setDeserializer(KafkaRecordDeserializationSchema.of(kafkaDeserializer));
-
- return kafkaSourceBuilder.build();
- }
-
- private OffsetResetStrategy getResetStrategy(String offsetResetConfig) {
- return Arrays.stream(OffsetResetStrategy.values())
- .filter(ors -> ors.name().equals(offsetResetConfig.toUpperCase(Locale.ROOT)))
- .findAny()
- .orElseThrow(
- () ->
- new IllegalArgumentException(
- String.format(
- "%s can not be set to %s. Valid values: [%s]",
- ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
- offsetResetConfig,
- Arrays.stream(OffsetResetStrategy.values())
- .map(Enum::name)
- .map(String::toLowerCase)
- .collect(Collectors.joining(",")))));
- }
-
- private KafkaDeserializationSchema<RowData> createKafkaDeserializationSchema(
- DeserializationSchema<RowData> keyDeserialization,
- DeserializationSchema<RowData> valueDeserialization,
- TypeInformation<RowData> producedTypeInfo) {
- final MetadataConverter[] metadataConverters =
- metadataKeys.stream()
- .map(
- k ->
- Stream.of(ReadableMetadata.values())
- .filter(rm -> rm.key.equals(k))
- .findFirst()
- .orElseThrow(IllegalStateException::new))
- .map(m -> m.converter)
- .toArray(MetadataConverter[]::new);
-
- // check if connector metadata is used at all
- final boolean hasMetadata = metadataKeys.size() > 0;
-
- // adjust physical arity with value format's metadata
- final int adjustedPhysicalArity =
- DataType.getFieldDataTypes(producedDataType).size() - metadataKeys.size();
-
- // adjust value format projection to include value format's metadata columns at the end
- final int[] adjustedValueProjection =
- IntStream.concat(
- IntStream.of(valueProjection),
- IntStream.range(
- keyProjection.length + valueProjection.length,
- adjustedPhysicalArity))
- .toArray();
-
- return new DynamicKafkaDeserializationSchema(
- adjustedPhysicalArity,
- keyDeserialization,
- keyProjection,
- valueDeserialization,
- adjustedValueProjection,
- hasMetadata,
- metadataConverters,
- producedTypeInfo,
- upsertMode);
- }
-
- private @Nullable DeserializationSchema<RowData> createDeserialization(
- DynamicTableSource.Context context,
- @Nullable DecodingFormat<DeserializationSchema<RowData>> format,
- int[] projection,
- @Nullable String prefix) {
- if (format == null) {
- return null;
- }
- DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType);
- if (prefix != null) {
- physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix);
- }
- return format.createRuntimeDecoder(context, physicalFormatDataType);
- }
-
- // --------------------------------------------------------------------------------------------
- // Metadata handling
- // --------------------------------------------------------------------------------------------
-
- enum ReadableMetadata {
- TOPIC(
- "topic",
- DataTypes.STRING().notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return StringData.fromString(record.topic());
- }
- }),
-
- PARTITION(
- "partition",
- DataTypes.INT().notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return record.partition();
- }
- }),
-
- HEADERS(
- "headers",
- // key and value of the map are nullable to make handling easier in queries
- DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable())
- .notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- final Map<StringData, byte[]> map = new HashMap<>();
- for (Header header : record.headers()) {
- map.put(StringData.fromString(header.key()), header.value());
- }
- return new GenericMapData(map);
- }
- }),
-
- LEADER_EPOCH(
- "leader-epoch",
- DataTypes.INT().nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return record.leaderEpoch().orElse(null);
- }
- }),
-
- OFFSET(
- "offset",
- DataTypes.BIGINT().notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return record.offset();
- }
- }),
-
- TIMESTAMP(
- "timestamp",
- DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return TimestampData.fromEpochMillis(record.timestamp());
- }
- }),
-
- TIMESTAMP_TYPE(
- "timestamp-type",
- DataTypes.STRING().notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(ConsumerRecord<?, ?> record) {
- return StringData.fromString(record.timestampType().toString());
- }
- });
-
- final String key;
-
- final DataType dataType;
-
- final MetadataConverter converter;
-
- ReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
- this.key = key;
- this.dataType = dataType;
- this.converter = converter;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java
deleted file mode 100644
index 48c0091..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java
+++ /dev/null
@@ -1,449 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.catalog.ObjectIdentifier;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.format.Format;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.DeserializationFormatFactory;
-import org.apache.flink.table.factories.DynamicTableSinkFactory;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
-import org.apache.flink.table.factories.SerializationFormatFactory;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.types.RowKind;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.time.Duration;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_GROUP_ID;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_TOPIC_PARTITION_DISCOVERY;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.StartupOptions;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getFlinkKafkaPartitioner;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getStartupOptions;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSinkOptions;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSourceOptions;
-
-/**
- * Factory for creating configured instances of {@link KafkaDynamicSource} and {@link
- * KafkaDynamicSink}.
- */
-@Internal
-public class KafkaDynamicTableFactory
- implements DynamicTableSourceFactory, DynamicTableSinkFactory {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaDynamicTableFactory.class);
- private static final ConfigOption<String> SINK_SEMANTIC =
- ConfigOptions.key("sink.semantic")
- .stringType()
- .noDefaultValue()
- .withDescription("Optional semantic when committing.");
-
- public static final String IDENTIFIER = "kafka";
-
- @Override
- public String factoryIdentifier() {
- return IDENTIFIER;
- }
-
- @Override
- public Set<ConfigOption<?>> requiredOptions() {
- final Set<ConfigOption<?>> options = new HashSet<>();
- options.add(PROPS_BOOTSTRAP_SERVERS);
- return options;
- }
-
- @Override
- public Set<ConfigOption<?>> optionalOptions() {
- final Set<ConfigOption<?>> options = new HashSet<>();
- options.add(FactoryUtil.FORMAT);
- options.add(KEY_FORMAT);
- options.add(KEY_FIELDS);
- options.add(KEY_FIELDS_PREFIX);
- options.add(VALUE_FORMAT);
- options.add(VALUE_FIELDS_INCLUDE);
- options.add(TOPIC);
- options.add(TOPIC_PATTERN);
- options.add(PROPS_GROUP_ID);
- options.add(SCAN_STARTUP_MODE);
- options.add(SCAN_STARTUP_SPECIFIC_OFFSETS);
- options.add(SCAN_TOPIC_PARTITION_DISCOVERY);
- options.add(SCAN_STARTUP_TIMESTAMP_MILLIS);
- options.add(SINK_PARTITIONER);
- options.add(SINK_PARALLELISM);
- options.add(DELIVERY_GUARANTEE);
- options.add(TRANSACTIONAL_ID_PREFIX);
- options.add(SINK_SEMANTIC);
- options.add(SCAN_BOUNDED_MODE);
- options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS);
- options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS);
- return options;
- }
-
- @Override
- public Set<ConfigOption<?>> forwardOptions() {
- return Stream.of(
- PROPS_BOOTSTRAP_SERVERS,
- PROPS_GROUP_ID,
- TOPIC,
- TOPIC_PATTERN,
- SCAN_STARTUP_MODE,
- SCAN_STARTUP_SPECIFIC_OFFSETS,
- SCAN_TOPIC_PARTITION_DISCOVERY,
- SCAN_STARTUP_TIMESTAMP_MILLIS,
- SINK_PARTITIONER,
- SINK_PARALLELISM,
- TRANSACTIONAL_ID_PREFIX)
- .collect(Collectors.toSet());
- }
-
- @Override
- public DynamicTableSource createDynamicTableSource(Context context) {
- final TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
-
- final Optional<DecodingFormat<DeserializationSchema<RowData>>> keyDecodingFormat =
- getKeyDecodingFormat(helper);
-
- final DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat =
- getValueDecodingFormat(helper);
-
- helper.validateExcept(PROPERTIES_PREFIX);
-
- final ReadableConfig tableOptions = helper.getOptions();
-
- validateTableSourceOptions(tableOptions);
-
- validatePKConstraints(
- context.getObjectIdentifier(),
- context.getPrimaryKeyIndexes(),
- context.getCatalogTable().getOptions(),
- valueDecodingFormat);
-
- final StartupOptions startupOptions = getStartupOptions(tableOptions);
-
- final BoundedOptions boundedOptions = getBoundedOptions(tableOptions);
-
- final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions());
-
- // add topic-partition discovery
- final Optional<Long> partitionDiscoveryInterval =
- tableOptions.getOptional(SCAN_TOPIC_PARTITION_DISCOVERY).map(Duration::toMillis);
- properties.setProperty(
- KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(),
- partitionDiscoveryInterval.orElse(-1L).toString());
-
- final DataType physicalDataType = context.getPhysicalRowDataType();
-
- final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType);
-
- final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType);
-
- final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null);
-
- return createKafkaTableSource(
- physicalDataType,
- keyDecodingFormat.orElse(null),
- valueDecodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- getSourceTopics(tableOptions),
- getSourceTopicPattern(tableOptions),
- properties,
- startupOptions.startupMode,
- startupOptions.specificOffsets,
- startupOptions.startupTimestampMillis,
- boundedOptions.boundedMode,
- boundedOptions.specificOffsets,
- boundedOptions.boundedTimestampMillis,
- context.getObjectIdentifier().asSummaryString());
- }
-
- @Override
- public DynamicTableSink createDynamicTableSink(Context context) {
- final TableFactoryHelper helper =
- FactoryUtil.createTableFactoryHelper(
- this, autoCompleteSchemaRegistrySubject(context));
-
- final Optional<EncodingFormat<SerializationSchema<RowData>>> keyEncodingFormat =
- getKeyEncodingFormat(helper);
-
- final EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- getValueEncodingFormat(helper);
-
- helper.validateExcept(PROPERTIES_PREFIX);
-
- final ReadableConfig tableOptions = helper.getOptions();
-
- final DeliveryGuarantee deliveryGuarantee = validateDeprecatedSemantic(tableOptions);
- validateTableSinkOptions(tableOptions);
-
- KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions);
-
- validatePKConstraints(
- context.getObjectIdentifier(),
- context.getPrimaryKeyIndexes(),
- context.getCatalogTable().getOptions(),
- valueEncodingFormat);
-
- final DataType physicalDataType = context.getPhysicalRowDataType();
-
- final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType);
-
- final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType);
-
- final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null);
-
- final Integer parallelism = tableOptions.getOptional(SINK_PARALLELISM).orElse(null);
-
- return createKafkaTableSink(
- physicalDataType,
- keyEncodingFormat.orElse(null),
- valueEncodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- tableOptions.get(TOPIC).get(0),
- getKafkaProperties(context.getCatalogTable().getOptions()),
- getFlinkKafkaPartitioner(tableOptions, context.getClassLoader()).orElse(null),
- deliveryGuarantee,
- parallelism,
- tableOptions.get(TRANSACTIONAL_ID_PREFIX));
- }
-
- // --------------------------------------------------------------------------------------------
-
- private static Optional<DecodingFormat<DeserializationSchema<RowData>>> getKeyDecodingFormat(
- TableFactoryHelper helper) {
- final Optional<DecodingFormat<DeserializationSchema<RowData>>> keyDecodingFormat =
- helper.discoverOptionalDecodingFormat(
- DeserializationFormatFactory.class, KEY_FORMAT);
- keyDecodingFormat.ifPresent(
- format -> {
- if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) {
- throw new ValidationException(
- String.format(
- "A key format should only deal with INSERT-only records. "
- + "But %s has a changelog mode of %s.",
- helper.getOptions().get(KEY_FORMAT),
- format.getChangelogMode()));
- }
- });
- return keyDecodingFormat;
- }
-
- private static Optional<EncodingFormat<SerializationSchema<RowData>>> getKeyEncodingFormat(
- TableFactoryHelper helper) {
- final Optional<EncodingFormat<SerializationSchema<RowData>>> keyEncodingFormat =
- helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT);
- keyEncodingFormat.ifPresent(
- format -> {
- if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) {
- throw new ValidationException(
- String.format(
- "A key format should only deal with INSERT-only records. "
- + "But %s has a changelog mode of %s.",
- helper.getOptions().get(KEY_FORMAT),
- format.getChangelogMode()));
- }
- });
- return keyEncodingFormat;
- }
-
- private static DecodingFormat<DeserializationSchema<RowData>> getValueDecodingFormat(
- TableFactoryHelper helper) {
- return helper.discoverOptionalDecodingFormat(
- DeserializationFormatFactory.class, FactoryUtil.FORMAT)
- .orElseGet(
- () ->
- helper.discoverDecodingFormat(
- DeserializationFormatFactory.class, VALUE_FORMAT));
- }
-
- private static EncodingFormat<SerializationSchema<RowData>> getValueEncodingFormat(
- TableFactoryHelper helper) {
- return helper.discoverOptionalEncodingFormat(
- SerializationFormatFactory.class, FactoryUtil.FORMAT)
- .orElseGet(
- () ->
- helper.discoverEncodingFormat(
- SerializationFormatFactory.class, VALUE_FORMAT));
- }
-
- private static void validatePKConstraints(
- ObjectIdentifier tableName,
- int[] primaryKeyIndexes,
- Map<String, String> options,
- Format format) {
- if (primaryKeyIndexes.length > 0
- && format.getChangelogMode().containsOnly(RowKind.INSERT)) {
- Configuration configuration = Configuration.fromMap(options);
- String formatName =
- configuration
- .getOptional(FactoryUtil.FORMAT)
- .orElse(configuration.get(VALUE_FORMAT));
- throw new ValidationException(
- String.format(
- "The Kafka table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint"
- + " on the table, because it can't guarantee the semantic of primary key.",
- tableName.asSummaryString(), formatName));
- }
- }
-
- private static DeliveryGuarantee validateDeprecatedSemantic(ReadableConfig tableOptions) {
- if (tableOptions.getOptional(SINK_SEMANTIC).isPresent()) {
- LOG.warn(
- "{} is deprecated and will be removed. Please use {} instead.",
- SINK_SEMANTIC.key(),
- DELIVERY_GUARANTEE.key());
- return DeliveryGuarantee.valueOf(
- tableOptions.get(SINK_SEMANTIC).toUpperCase().replace("-", "_"));
- }
- return tableOptions.get(DELIVERY_GUARANTEE);
- }
-
- // --------------------------------------------------------------------------------------------
-
- protected KafkaDynamicSource createKafkaTableSource(
- DataType physicalDataType,
- @Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- @Nullable List<String> topics,
- @Nullable Pattern topicPattern,
- Properties properties,
- StartupMode startupMode,
- Map<KafkaTopicPartition, Long> specificStartupOffsets,
- long startupTimestampMillis,
- BoundedMode boundedMode,
- Map<KafkaTopicPartition, Long> specificEndOffsets,
- long endTimestampMillis,
- String tableIdentifier) {
- return new KafkaDynamicSource(
- physicalDataType,
- keyDecodingFormat,
- valueDecodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topics,
- topicPattern,
- properties,
- startupMode,
- specificStartupOffsets,
- startupTimestampMillis,
- boundedMode,
- specificEndOffsets,
- endTimestampMillis,
- false,
- tableIdentifier);
- }
-
- protected KafkaDynamicSink createKafkaTableSink(
- DataType physicalDataType,
- @Nullable EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat,
- EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- String topic,
- Properties properties,
- FlinkKafkaPartitioner<RowData> partitioner,
- DeliveryGuarantee deliveryGuarantee,
- Integer parallelism,
- @Nullable String transactionalIdPrefix) {
- return new KafkaDynamicSink(
- physicalDataType,
- physicalDataType,
- keyEncodingFormat,
- valueEncodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topic,
- properties,
- partitioner,
- deliveryGuarantee,
- false,
- SinkBufferFlushMode.DISABLED,
- parallelism,
- transactionalIdPrefix);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java
deleted file mode 100644
index da48e39..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.connector.sink.Sink;
-import org.apache.flink.api.connector.sink.SinkWriter;
-import org.apache.flink.api.connector.sink2.StatefulSink;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.util.function.SerializableFunction;
-
-import java.io.IOException;
-import java.util.Collection;
-
-/**
- * A wrapper of a {@link Sink}. It will buffer the data emitted by the wrapper {@link SinkWriter}
- * and only emit it when the buffer is full or a timer is triggered or a checkpoint happens.
- *
- * <p>The sink provides eventual consistency guarantees without the need of a two-phase protocol
- * because the updates are idempotent therefore duplicates have no effect.
- */
-class ReducingUpsertSink<WriterState> implements StatefulSink<RowData, WriterState> {
-
- private final StatefulSink<RowData, WriterState> wrappedSink;
- private final DataType physicalDataType;
- private final int[] keyProjection;
- private final SinkBufferFlushMode bufferFlushMode;
- private final SerializableFunction<RowData, RowData> valueCopyFunction;
-
- ReducingUpsertSink(
- StatefulSink<RowData, WriterState> wrappedSink,
- DataType physicalDataType,
- int[] keyProjection,
- SinkBufferFlushMode bufferFlushMode,
- SerializableFunction<RowData, RowData> valueCopyFunction) {
- this.wrappedSink = wrappedSink;
- this.physicalDataType = physicalDataType;
- this.keyProjection = keyProjection;
- this.bufferFlushMode = bufferFlushMode;
- this.valueCopyFunction = valueCopyFunction;
- }
-
- @Override
- public StatefulSinkWriter<RowData, WriterState> createWriter(InitContext context)
- throws IOException {
- final StatefulSinkWriter<RowData, WriterState> wrapperWriter =
- wrappedSink.createWriter(context);
- return new ReducingUpsertWriter<>(
- wrapperWriter,
- physicalDataType,
- keyProjection,
- bufferFlushMode,
- context.getProcessingTimeService(),
- valueCopyFunction);
- }
-
- @Override
- public StatefulSinkWriter<RowData, WriterState> restoreWriter(
- InitContext context, Collection<WriterState> recoveredState) throws IOException {
- final StatefulSinkWriter<RowData, WriterState> wrapperWriter =
- wrappedSink.restoreWriter(context, recoveredState);
- return new ReducingUpsertWriter<>(
- wrapperWriter,
- physicalDataType,
- keyProjection,
- bufferFlushMode,
- context.getProcessingTimeService(),
- valueCopyFunction);
- }
-
- @Override
- public SimpleVersionedSerializer<WriterState> getWriterStateSerializer() {
- return wrappedSink.getWriterStateSerializer();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java
deleted file mode 100644
index 5a87add..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.operators.ProcessingTimeService;
-import org.apache.flink.api.connector.sink2.SinkWriter;
-import org.apache.flink.api.connector.sink2.StatefulSink;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.types.RowKind;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.function.Function;
-
-import static org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaRecordSerializationSchema.createProjectedRow;
-import static org.apache.flink.types.RowKind.DELETE;
-import static org.apache.flink.types.RowKind.UPDATE_AFTER;
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-class ReducingUpsertWriter<WriterState>
- implements StatefulSink.StatefulSinkWriter<RowData, WriterState> {
-
- private final StatefulSink.StatefulSinkWriter<RowData, WriterState> wrappedWriter;
- private final WrappedContext wrappedContext = new WrappedContext();
- private final int batchMaxRowNums;
- private final Function<RowData, RowData> valueCopyFunction;
- private final Map<RowData, Tuple2<RowData, Long>> reduceBuffer = new HashMap<>();
- private final Function<RowData, RowData> keyExtractor;
- private final ProcessingTimeService timeService;
- private final long batchIntervalMs;
-
- private boolean closed = false;
- private long lastFlush = System.currentTimeMillis();
-
- ReducingUpsertWriter(
- StatefulSink.StatefulSinkWriter<RowData, WriterState> wrappedWriter,
- DataType physicalDataType,
- int[] keyProjection,
- SinkBufferFlushMode bufferFlushMode,
- ProcessingTimeService timeService,
- Function<RowData, RowData> valueCopyFunction) {
- checkArgument(bufferFlushMode != null && bufferFlushMode.isEnabled());
- this.wrappedWriter = checkNotNull(wrappedWriter);
- this.timeService = checkNotNull(timeService);
- this.batchMaxRowNums = bufferFlushMode.getBatchSize();
- this.batchIntervalMs = bufferFlushMode.getBatchIntervalMs();
- registerFlush();
- List<LogicalType> fields = physicalDataType.getLogicalType().getChildren();
- final RowData.FieldGetter[] keyFieldGetters =
- Arrays.stream(keyProjection)
- .mapToObj(
- targetField ->
- RowData.createFieldGetter(
- fields.get(targetField), targetField))
- .toArray(RowData.FieldGetter[]::new);
- this.keyExtractor = rowData -> createProjectedRow(rowData, RowKind.INSERT, keyFieldGetters);
- this.valueCopyFunction = valueCopyFunction;
- }
-
- @Override
- public void write(RowData element, Context context) throws IOException, InterruptedException {
- wrappedContext.setContext(context);
- addToBuffer(element, context.timestamp());
- }
-
- @Override
- public void flush(boolean endOfInput) throws IOException, InterruptedException {
- flush();
- }
-
- @Override
- public List<WriterState> snapshotState(long checkpointId) throws IOException {
- return wrappedWriter.snapshotState(checkpointId);
- }
-
- @Override
- public void close() throws Exception {
- if (!closed) {
- closed = true;
- wrappedWriter.close();
- }
- }
-
- private void addToBuffer(RowData row, Long timestamp) throws IOException, InterruptedException {
- RowData key = keyExtractor.apply(row);
- RowData value = valueCopyFunction.apply(row);
- reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp));
-
- if (reduceBuffer.size() >= batchMaxRowNums) {
- flush();
- }
- }
-
- private void registerFlush() {
- if (closed) {
- return;
- }
- timeService.registerTimer(
- lastFlush + batchIntervalMs,
- (t) -> {
- if (t >= lastFlush + batchIntervalMs) {
- flush();
- }
- registerFlush();
- });
- }
-
- private RowData changeFlag(RowData value) {
- switch (value.getRowKind()) {
- case INSERT:
- case UPDATE_AFTER:
- value.setRowKind(UPDATE_AFTER);
- break;
- case UPDATE_BEFORE:
- case DELETE:
- value.setRowKind(DELETE);
- }
- return value;
- }
-
- private void flush() throws IOException, InterruptedException {
- for (Tuple2<RowData, Long> value : reduceBuffer.values()) {
- wrappedContext.setTimestamp(value.f1);
- wrappedWriter.write(value.f0, wrappedContext);
- }
- lastFlush = System.currentTimeMillis();
- reduceBuffer.clear();
- }
-
- /**
- * Wrapper of {@link SinkWriter.Context}.
- *
- * <p>When records arrives, the {@link ReducingUpsertWriter} updates the current {@link
- * SinkWriter.Context} and memorize the timestamp with the records. When flushing, the {@link
- * ReducingUpsertWriter} will emit the records in the buffer with memorized timestamp.
- */
- private static class WrappedContext implements SinkWriter.Context {
- private long timestamp;
- private SinkWriter.Context context;
-
- @Override
- public long currentWatermark() {
- checkNotNull(context, "context must be set before retrieving it.");
- return context.currentWatermark();
- }
-
- @Override
- public Long timestamp() {
- checkNotNull(timestamp, "timestamp must to be set before retrieving it.");
- return timestamp;
- }
-
- public void setTimestamp(long timestamp) {
- this.timestamp = timestamp;
- }
-
- public void setContext(SinkWriter.Context context) {
- this.context = context;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/SinkBufferFlushMode.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/SinkBufferFlushMode.java
deleted file mode 100644
index 91a897a..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/SinkBufferFlushMode.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.table;
-
-import java.io.Serializable;
-import java.util.Objects;
-
-/** Sink buffer flush configuration. */
-public class SinkBufferFlushMode implements Serializable {
-
- private static final int DISABLED_BATCH_SIZE = 0;
- private static final long DISABLED_BATCH_INTERVAL = 0L;
-
- public static final SinkBufferFlushMode DISABLED =
- new SinkBufferFlushMode(DISABLED_BATCH_SIZE, DISABLED_BATCH_INTERVAL);
-
- private final int batchSize;
- private final long batchIntervalMs;
-
- public SinkBufferFlushMode(int batchSize, long batchIntervalMs) {
- this.batchSize = batchSize;
- this.batchIntervalMs = batchIntervalMs;
-
- // validation
- if (isEnabled()
- && !(batchSize > DISABLED_BATCH_SIZE
- && batchIntervalMs > DISABLED_BATCH_INTERVAL)) {
- throw new IllegalArgumentException(
- String.format(
- "batchSize and batchInterval must greater than zero if buffer flush is enabled,"
- + " but got batchSize=%s and batchIntervalMs=%s",
- batchSize, batchIntervalMs));
- }
- }
-
- public int getBatchSize() {
- return batchSize;
- }
-
- public long getBatchIntervalMs() {
- return batchIntervalMs;
- }
-
- public boolean isEnabled() {
- return !(batchSize == DISABLED_BATCH_SIZE && batchIntervalMs == DISABLED_BATCH_INTERVAL);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- SinkBufferFlushMode that = (SinkBufferFlushMode) o;
- return batchSize == that.batchSize && batchIntervalMs == that.batchIntervalMs;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(batchSize, batchIntervalMs);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java
deleted file mode 100644
index 254e1bf..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java
+++ /dev/null
@@ -1,406 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.catalog.ResolvedCatalogTable;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.format.Format;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.factories.DeserializationFormatFactory;
-import org.apache.flink.table.factories.DynamicTableSinkFactory;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.SerializationFormatFactory;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.types.RowKind;
-
-import java.time.Duration;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.Set;
-
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics;
-
-/** Upsert-Kafka factory. */
-public class UpsertKafkaDynamicTableFactory
- implements DynamicTableSourceFactory, DynamicTableSinkFactory {
-
- public static final String IDENTIFIER = "upsert-kafka";
-
- @Override
- public String factoryIdentifier() {
- return IDENTIFIER;
- }
-
- @Override
- public Set<ConfigOption<?>> requiredOptions() {
- final Set<ConfigOption<?>> options = new HashSet<>();
- options.add(PROPS_BOOTSTRAP_SERVERS);
- options.add(TOPIC);
- options.add(KEY_FORMAT);
- options.add(VALUE_FORMAT);
- return options;
- }
-
- @Override
- public Set<ConfigOption<?>> optionalOptions() {
- final Set<ConfigOption<?>> options = new HashSet<>();
- options.add(KEY_FIELDS_PREFIX);
- options.add(VALUE_FIELDS_INCLUDE);
- options.add(SINK_PARALLELISM);
- options.add(SINK_BUFFER_FLUSH_INTERVAL);
- options.add(SINK_BUFFER_FLUSH_MAX_ROWS);
- return options;
- }
-
- @Override
- public DynamicTableSource createDynamicTableSource(Context context) {
- FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
-
- ReadableConfig tableOptions = helper.getOptions();
- DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat =
- helper.discoverDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT);
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat =
- helper.discoverDecodingFormat(DeserializationFormatFactory.class, VALUE_FORMAT);
-
- // Validate the option data type.
- helper.validateExcept(PROPERTIES_PREFIX);
- validateSource(
- tableOptions,
- keyDecodingFormat,
- valueDecodingFormat,
- context.getPrimaryKeyIndexes());
-
- Tuple2<int[], int[]> keyValueProjections =
- createKeyValueProjections(context.getCatalogTable());
- String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null);
- Properties properties = getKafkaProperties(context.getCatalogTable().getOptions());
- // always use earliest to keep data integrity
- StartupMode earliest = StartupMode.EARLIEST;
-
- return new KafkaDynamicSource(
- context.getPhysicalRowDataType(),
- keyDecodingFormat,
- new DecodingFormatWrapper(valueDecodingFormat),
- keyValueProjections.f0,
- keyValueProjections.f1,
- keyPrefix,
- getSourceTopics(tableOptions),
- getSourceTopicPattern(tableOptions),
- properties,
- earliest,
- Collections.emptyMap(),
- 0,
- BoundedMode.UNBOUNDED,
- Collections.emptyMap(),
- 0,
- true,
- context.getObjectIdentifier().asSummaryString());
- }
-
- @Override
- public DynamicTableSink createDynamicTableSink(Context context) {
- FactoryUtil.TableFactoryHelper helper =
- FactoryUtil.createTableFactoryHelper(
- this, autoCompleteSchemaRegistrySubject(context));
-
- final ReadableConfig tableOptions = helper.getOptions();
-
- EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat =
- helper.discoverEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT);
- EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- helper.discoverEncodingFormat(SerializationFormatFactory.class, VALUE_FORMAT);
-
- // Validate the option data type.
- helper.validateExcept(PROPERTIES_PREFIX);
- validateSink(
- tableOptions,
- keyEncodingFormat,
- valueEncodingFormat,
- context.getPrimaryKeyIndexes());
-
- Tuple2<int[], int[]> keyValueProjections =
- createKeyValueProjections(context.getCatalogTable());
- final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null);
- final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions());
-
- Integer parallelism = tableOptions.get(SINK_PARALLELISM);
-
- int batchSize = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS);
- Duration batchInterval = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL);
- SinkBufferFlushMode flushMode =
- new SinkBufferFlushMode(batchSize, batchInterval.toMillis());
-
- // use {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}.
- // it will use hash partition if key is set else in round-robin behaviour.
- return new KafkaDynamicSink(
- context.getPhysicalRowDataType(),
- context.getPhysicalRowDataType(),
- keyEncodingFormat,
- new EncodingFormatWrapper(valueEncodingFormat),
- keyValueProjections.f0,
- keyValueProjections.f1,
- keyPrefix,
- tableOptions.get(TOPIC).get(0),
- properties,
- null,
- DeliveryGuarantee.AT_LEAST_ONCE,
- true,
- flushMode,
- parallelism,
- tableOptions.get(TRANSACTIONAL_ID_PREFIX));
- }
-
- private Tuple2<int[], int[]> createKeyValueProjections(ResolvedCatalogTable catalogTable) {
- ResolvedSchema schema = catalogTable.getResolvedSchema();
- // primary key should validated earlier
- List<String> keyFields = schema.getPrimaryKey().get().getColumns();
- DataType physicalDataType = schema.toPhysicalRowDataType();
-
- Configuration tableOptions = Configuration.fromMap(catalogTable.getOptions());
- // upsert-kafka will set key.fields to primary key fields by default
- tableOptions.set(KEY_FIELDS, keyFields);
-
- int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType);
- int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType);
-
- return Tuple2.of(keyProjection, valueProjection);
- }
-
- // --------------------------------------------------------------------------------------------
- // Validation
- // --------------------------------------------------------------------------------------------
-
- private static void validateSource(
- ReadableConfig tableOptions,
- Format keyFormat,
- Format valueFormat,
- int[] primaryKeyIndexes) {
- validateTopic(tableOptions);
- validateFormat(keyFormat, valueFormat, tableOptions);
- validatePKConstraints(primaryKeyIndexes);
- }
-
- private static void validateSink(
- ReadableConfig tableOptions,
- Format keyFormat,
- Format valueFormat,
- int[] primaryKeyIndexes) {
- validateTopic(tableOptions);
- validateFormat(keyFormat, valueFormat, tableOptions);
- validatePKConstraints(primaryKeyIndexes);
- validateSinkBufferFlush(tableOptions);
- }
-
- private static void validateTopic(ReadableConfig tableOptions) {
- List<String> topic = tableOptions.get(TOPIC);
- if (topic.size() > 1) {
- throw new ValidationException(
- "The 'upsert-kafka' connector doesn't support topic list now. "
- + "Please use single topic as the value of the parameter 'topic'.");
- }
- }
-
- private static void validateFormat(
- Format keyFormat, Format valueFormat, ReadableConfig tableOptions) {
- if (!keyFormat.getChangelogMode().containsOnly(RowKind.INSERT)) {
- String identifier = tableOptions.get(KEY_FORMAT);
- throw new ValidationException(
- String.format(
- "'upsert-kafka' connector doesn't support '%s' as key format, "
- + "because '%s' is not in insert-only mode.",
- identifier, identifier));
- }
- if (!valueFormat.getChangelogMode().containsOnly(RowKind.INSERT)) {
- String identifier = tableOptions.get(VALUE_FORMAT);
- throw new ValidationException(
- String.format(
- "'upsert-kafka' connector doesn't support '%s' as value format, "
- + "because '%s' is not in insert-only mode.",
- identifier, identifier));
- }
- }
-
- private static void validatePKConstraints(int[] schema) {
- if (schema.length == 0) {
- throw new ValidationException(
- "'upsert-kafka' tables require to define a PRIMARY KEY constraint. "
- + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. "
- + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys.");
- }
- }
-
- private static void validateSinkBufferFlush(ReadableConfig tableOptions) {
- int flushMaxRows = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS);
- long flushIntervalMs = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis();
- if (flushMaxRows > 0 && flushIntervalMs > 0) {
- // flush is enabled
- return;
- }
- if (flushMaxRows <= 0 && flushIntervalMs <= 0) {
- // flush is disabled
- return;
- }
- // one of them is set which is not allowed
- throw new ValidationException(
- String.format(
- "'%s' and '%s' must be set to be greater than zero together to enable sink buffer flushing.",
- SINK_BUFFER_FLUSH_MAX_ROWS.key(), SINK_BUFFER_FLUSH_INTERVAL.key()));
- }
-
- // --------------------------------------------------------------------------------------------
- // Format wrapper
- // --------------------------------------------------------------------------------------------
-
- /**
- * It is used to wrap the decoding format and expose the desired changelog mode. It's only works
- * for insert-only format.
- */
- protected static class DecodingFormatWrapper
- implements DecodingFormat<DeserializationSchema<RowData>> {
- private final DecodingFormat<DeserializationSchema<RowData>> innerDecodingFormat;
-
- private static final ChangelogMode SOURCE_CHANGELOG_MODE =
- ChangelogMode.newBuilder()
- .addContainedKind(RowKind.UPDATE_AFTER)
- .addContainedKind(RowKind.DELETE)
- .build();
-
- public DecodingFormatWrapper(
- DecodingFormat<DeserializationSchema<RowData>> innerDecodingFormat) {
- this.innerDecodingFormat = innerDecodingFormat;
- }
-
- @Override
- public DeserializationSchema<RowData> createRuntimeDecoder(
- DynamicTableSource.Context context, DataType producedDataType) {
- return innerDecodingFormat.createRuntimeDecoder(context, producedDataType);
- }
-
- @Override
- public ChangelogMode getChangelogMode() {
- return SOURCE_CHANGELOG_MODE;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == this) {
- return true;
- }
-
- if (obj == null || getClass() != obj.getClass()) {
- return false;
- }
-
- DecodingFormatWrapper that = (DecodingFormatWrapper) obj;
- return Objects.equals(innerDecodingFormat, that.innerDecodingFormat);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(innerDecodingFormat);
- }
- }
-
- /**
- * It is used to wrap the encoding format and expose the desired changelog mode. It's only works
- * for insert-only format.
- */
- protected static class EncodingFormatWrapper
- implements EncodingFormat<SerializationSchema<RowData>> {
- private final EncodingFormat<SerializationSchema<RowData>> innerEncodingFormat;
-
- public static final ChangelogMode SINK_CHANGELOG_MODE =
- ChangelogMode.newBuilder()
- .addContainedKind(RowKind.INSERT)
- .addContainedKind(RowKind.UPDATE_AFTER)
- .addContainedKind(RowKind.DELETE)
- .build();
-
- public EncodingFormatWrapper(
- EncodingFormat<SerializationSchema<RowData>> innerEncodingFormat) {
- this.innerEncodingFormat = innerEncodingFormat;
- }
-
- @Override
- public SerializationSchema<RowData> createRuntimeEncoder(
- DynamicTableSink.Context context, DataType consumedDataType) {
- return innerEncodingFormat.createRuntimeEncoder(context, consumedDataType);
- }
-
- @Override
- public ChangelogMode getChangelogMode() {
- return SINK_CHANGELOG_MODE;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == this) {
- return true;
- }
-
- if (obj == null || getClass() != obj.getClass()) {
- return false;
- }
-
- EncodingFormatWrapper that = (EncodingFormatWrapper) obj;
- return Objects.equals(innerEncodingFormat, that.innerEncodingFormat);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(innerEncodingFormat);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
deleted file mode 100644
index e2b428e..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.util.serialization;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass;
-
-/**
- * DeserializationSchema that deserializes a JSON String into an ObjectNode.
- *
- * <p>Key fields can be accessed by calling objectNode.get("key").get(<name>).as(<type>)
- *
- * <p>Value fields can be accessed by calling objectNode.get("value").get(<name>).as(<type>)
- *
- * <p>Metadata fields can be accessed by calling
- * objectNode.get("metadata").get(<name>).as(<type>) and include the "offset" (long), "topic"
- * (String) and "partition" (int).
- */
-@PublicEvolving
-public class JSONKeyValueDeserializationSchema implements KafkaDeserializationSchema<ObjectNode> {
-
- private static final long serialVersionUID = 1509391548173891955L;
-
- private final boolean includeMetadata;
- private ObjectMapper mapper;
-
- public JSONKeyValueDeserializationSchema(boolean includeMetadata) {
- this.includeMetadata = includeMetadata;
- }
-
- @Override
- public void open(DeserializationSchema.InitializationContext context) throws Exception {
- mapper = JacksonMapperFactory.createObjectMapper();
- }
-
- @Override
- public ObjectNode deserialize(ConsumerRecord<byte[], byte[]> record) throws Exception {
- ObjectNode node = mapper.createObjectNode();
- if (record.key() != null) {
- node.set("key", mapper.readValue(record.key(), JsonNode.class));
- }
- if (record.value() != null) {
- node.set("value", mapper.readValue(record.value(), JsonNode.class));
- }
- if (includeMetadata) {
- node.putObject("metadata")
- .put("offset", record.offset())
- .put("topic", record.topic())
- .put("partition", record.partition());
- }
- return node;
- }
-
- @Override
- public boolean isEndOfStream(ObjectNode nextElement) {
- return false;
- }
-
- @Override
- public TypeInformation<ObjectNode> getProducedType() {
- return getForClass(ObjectNode.class);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
deleted file mode 100644
index d3150b9..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.util.serialization;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.IOException;
-
-/**
- * The deserialization schema describes how to turn the byte key / value messages delivered by
- * certain data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
- * processed by Flink.
- *
- * @param <T> The type created by the keyed deserialization schema.
- * @deprecated Use {@link KafkaDeserializationSchema}.
- */
-@Deprecated
-@PublicEvolving
-public interface KeyedDeserializationSchema<T> extends KafkaDeserializationSchema<T> {
- /**
- * Deserializes the byte message.
- *
- * @param messageKey the key as a byte array (null if no key has been set).
- * @param message The message, as a byte array (null if the message was empty or deleted).
- * @param partition The partition the message has originated from.
- * @param offset the offset of the message in the original source (for example the Kafka
- * offset).
- * @return The deserialized message as an object (null if the message cannot be deserialized).
- */
- T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset)
- throws IOException;
-
- @Override
- default T deserialize(ConsumerRecord<byte[], byte[]> record) throws IOException {
- return deserialize(
- record.key(), record.value(), record.topic(), record.partition(), record.offset());
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
deleted file mode 100644
index b777419..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.util.serialization;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema;
-
-import java.io.Serializable;
-
-/**
- * The serialization schema describes how to turn a data object into a different serialized
- * representation. Most data sinks (for example Apache Kafka) require the data to be handed to them
- * in a specific format (for example as byte strings).
- *
- * @param <T> The type to be serialized.
- * @deprecated Use {@link KafkaSerializationSchema}.
- */
-@Deprecated
-@PublicEvolving
-public interface KeyedSerializationSchema<T> extends Serializable {
-
- /**
- * Serializes the key of the incoming element to a byte array This method might return null if
- * no key is available.
- *
- * @param element The incoming element to be serialized
- * @return the key of the element as a byte array
- */
- byte[] serializeKey(T element);
-
- /**
- * Serializes the value of the incoming element to a byte array.
- *
- * @param element The incoming element to be serialized
- * @return the value of the element as a byte array
- */
- byte[] serializeValue(T element);
-
- /**
- * Optional method to determine the target topic for the element.
- *
- * @param element Incoming element to determine the target topic from
- * @return null or the target topic
- */
- String getTargetTopic(T element);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
deleted file mode 100644
index 1c3eaa6..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * 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.util.serialization;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.memory.DataInputDeserializer;
-import org.apache.flink.core.memory.DataOutputSerializer;
-import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.io.IOException;
-
-/**
- * A serialization and deserialization schema for Key Value Pairs that uses Flink's serialization
- * stack to transform typed from and to byte arrays.
- *
- * @param <K> The key type to be serialized.
- * @param <V> The value type to be serialized.
- */
-@PublicEvolving
-public class TypeInformationKeyValueSerializationSchema<K, V>
- implements KafkaDeserializationSchema<Tuple2<K, V>>,
- KeyedSerializationSchema<Tuple2<K, V>> {
-
- private static final long serialVersionUID = -5359448468131559102L;
-
- /** The serializer for the key. */
- private final TypeSerializer<K> keySerializer;
-
- /** The serializer for the value. */
- private final TypeSerializer<V> valueSerializer;
-
- /** reusable input deserialization buffer. */
- private final DataInputDeserializer inputDeserializer;
-
- /** reusable output serialization buffer for the key. */
- private transient DataOutputSerializer keyOutputSerializer;
-
- /** reusable output serialization buffer for the value. */
- private transient DataOutputSerializer valueOutputSerializer;
-
- /**
- * The type information, to be returned by {@link #getProducedType()}. It is transient, because
- * it is not serializable. Note that this means that the type information is not available at
- * runtime, but only prior to the first serialization / deserialization
- */
- private final transient TypeInformation<Tuple2<K, V>> typeInfo;
-
- // ------------------------------------------------------------------------
-
- /**
- * Creates a new de-/serialization schema for the given types.
- *
- * @param keyTypeInfo The type information for the key type de-/serialized by this schema.
- * @param valueTypeInfo The type information for the value type de-/serialized by this schema.
- * @param ec The execution config, which is used to parametrize the type serializers.
- */
- public TypeInformationKeyValueSerializationSchema(
- TypeInformation<K> keyTypeInfo, TypeInformation<V> valueTypeInfo, ExecutionConfig ec) {
- this.typeInfo = new TupleTypeInfo<>(keyTypeInfo, valueTypeInfo);
- this.keySerializer = keyTypeInfo.createSerializer(ec);
- this.valueSerializer = valueTypeInfo.createSerializer(ec);
- this.inputDeserializer = new DataInputDeserializer();
- }
-
- /**
- * Creates a new de-/serialization schema for the given types. This constructor accepts the
- * types as classes and internally constructs the type information from the classes.
- *
- * <p>If the types are parametrized and cannot be fully defined via classes, use the constructor
- * that accepts {@link TypeInformation} instead.
- *
- * @param keyClass The class of the key de-/serialized by this schema.
- * @param valueClass The class of the value de-/serialized by this schema.
- * @param config The execution config, which is used to parametrize the type serializers.
- */
- public TypeInformationKeyValueSerializationSchema(
- Class<K> keyClass, Class<V> valueClass, ExecutionConfig config) {
- this(
- TypeExtractor.createTypeInfo(keyClass),
- TypeExtractor.createTypeInfo(valueClass),
- config);
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public Tuple2<K, V> deserialize(ConsumerRecord<byte[], byte[]> record) throws Exception {
- K key = null;
- V value = null;
-
- if (record.key() != null) {
- inputDeserializer.setBuffer(record.key());
- key = keySerializer.deserialize(inputDeserializer);
- }
- if (record.value() != null) {
- inputDeserializer.setBuffer(record.value());
- value = valueSerializer.deserialize(inputDeserializer);
- }
- return new Tuple2<>(key, value);
- }
-
- /**
- * This schema never considers an element to signal end-of-stream, so this method returns always
- * false.
- *
- * @param nextElement The element to test for the end-of-stream signal.
- * @return Returns false.
- */
- @Override
- public boolean isEndOfStream(Tuple2<K, V> nextElement) {
- return false;
- }
-
- @Override
- public byte[] serializeKey(Tuple2<K, V> element) {
- if (element.f0 == null) {
- return null;
- } else {
- // key is not null. serialize it:
- if (keyOutputSerializer == null) {
- keyOutputSerializer = new DataOutputSerializer(16);
- }
- try {
- keySerializer.serialize(element.f0, keyOutputSerializer);
- } catch (IOException e) {
- throw new RuntimeException("Unable to serialize record", e);
- }
- // check if key byte array size changed
- byte[] res = keyOutputSerializer.getByteArray();
- if (res.length != keyOutputSerializer.length()) {
- byte[] n = new byte[keyOutputSerializer.length()];
- System.arraycopy(res, 0, n, 0, keyOutputSerializer.length());
- res = n;
- }
- keyOutputSerializer.clear();
- return res;
- }
- }
-
- @Override
- public byte[] serializeValue(Tuple2<K, V> element) {
- // if the value is null, its serialized value is null as well.
- if (element.f1 == null) {
- return null;
- }
-
- if (valueOutputSerializer == null) {
- valueOutputSerializer = new DataOutputSerializer(16);
- }
-
- try {
- valueSerializer.serialize(element.f1, valueOutputSerializer);
- } catch (IOException e) {
- throw new RuntimeException("Unable to serialize record", e);
- }
-
- byte[] res = valueOutputSerializer.getByteArray();
- if (res.length != valueOutputSerializer.length()) {
- byte[] n = new byte[valueOutputSerializer.length()];
- System.arraycopy(res, 0, n, 0, valueOutputSerializer.length());
- res = n;
- }
- valueOutputSerializer.clear();
- return res;
- }
-
- @Override
- public String getTargetTopic(Tuple2<K, V> element) {
- return null; // we are never overriding the topic
- }
-
- @Override
- public TypeInformation<Tuple2<K, V>> getProducedType() {
- if (typeInfo != null) {
- return typeInfo;
- } else {
- throw new IllegalStateException(
- "The type information is not available after this class has been serialized and distributed.");
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
deleted file mode 100644
index 9b8bf8e..0000000
--- a/flink-connectors/flink-connector-kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ /dev/null
@@ -1,17 +0,0 @@
-# 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.
-
-org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicTableFactory
-org.apache.flink.streaming.connectors.kafka.table.UpsertKafkaDynamicTableFactory
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
deleted file mode 100644
index 7b73b67..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.architecture;
-
-import org.apache.flink.architecture.common.ImportOptions;
-
-import com.tngtech.archunit.core.importer.ImportOption;
-import com.tngtech.archunit.junit.AnalyzeClasses;
-import com.tngtech.archunit.junit.ArchTest;
-import com.tngtech.archunit.junit.ArchTests;
-
-/** Architecture tests for test code. */
-@AnalyzeClasses(
- packages = {
- "org.apache.flink.connector.kafka",
- "org.apache.flink.streaming.connectors.kafka"
- },
- importOptions = {
- ImportOption.OnlyIncludeTests.class,
- ImportOptions.ExcludeScalaImportOption.class,
- ImportOptions.ExcludeShadedImportOption.class
- })
-public class TestCodeArchitectureTest {
-
- @ArchTest
- public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java
deleted file mode 100644
index 51770f0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.MethodSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-import java.time.Duration;
-import java.util.List;
-import java.util.Properties;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-@Testcontainers
-@ExtendWith(TestLoggerExtension.class)
-class FlinkKafkaInternalProducerITCase {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(FlinkKafkaInternalProducerITCase.class);
-
- @Container
- private static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper();
-
- private static final String TRANSACTION_PREFIX = "test-transaction-";
-
- @Test
- void testInitTransactionId() {
- final String topic = "test-init-transactions";
- try (FlinkKafkaInternalProducer<String, String> reuse =
- new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) {
- int numTransactions = 20;
- for (int i = 1; i <= numTransactions; i++) {
- reuse.initTransactionId(TRANSACTION_PREFIX + i);
- reuse.beginTransaction();
- reuse.send(new ProducerRecord<>(topic, "test-value-" + i));
- if (i % 2 == 0) {
- reuse.commitTransaction();
- } else {
- reuse.flush();
- reuse.abortTransaction();
- }
- assertNumTransactions(i);
- assertThat(readRecords(topic).count()).isEqualTo(i / 2);
- }
- }
- }
-
- @ParameterizedTest
- @MethodSource("provideTransactionsFinalizer")
- void testResetInnerTransactionIfFinalizingTransactionFailed(
- Consumer<FlinkKafkaInternalProducer<?, ?>> transactionFinalizer) {
- final String topic = "reset-producer-internal-state";
- try (FlinkKafkaInternalProducer<String, String> fenced =
- new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) {
- fenced.initTransactions();
- fenced.beginTransaction();
- fenced.send(new ProducerRecord<>(topic, "test-value"));
- // Start a second producer that fences the first one
- try (FlinkKafkaInternalProducer<String, String> producer =
- new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) {
- producer.initTransactions();
- producer.beginTransaction();
- producer.send(new ProducerRecord<>(topic, "test-value"));
- producer.commitTransaction();
- }
- assertThatThrownBy(() -> transactionFinalizer.accept(fenced))
- .isInstanceOf(ProducerFencedException.class);
- // Internal transaction should be reset and setting a new transactional id is possible
- fenced.setTransactionId("dummy2");
- }
- }
-
- private static Properties getProperties() {
- Properties properties = new Properties();
- properties.put(
- CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
- KAFKA_CONTAINER.getBootstrapServers());
- properties.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true");
- properties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
- properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
- properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
- properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
- return properties;
- }
-
- private static List<Consumer<FlinkKafkaInternalProducer<?, ?>>> provideTransactionsFinalizer() {
- return Lists.newArrayList(
- FlinkKafkaInternalProducer::commitTransaction,
- FlinkKafkaInternalProducer::abortTransaction);
- }
-
- private void assertNumTransactions(int numTransactions) {
- List<KafkaTransactionLog.TransactionRecord> transactions =
- new KafkaTransactionLog(getProperties())
- .getTransactions(id -> id.startsWith(TRANSACTION_PREFIX));
- assertThat(
- transactions.stream()
- .map(KafkaTransactionLog.TransactionRecord::getTransactionId)
- .collect(Collectors.toSet()))
- .hasSize(numTransactions);
- }
-
- private ConsumerRecords<String, String> readRecords(String topic) {
- Properties properties = getProperties();
- properties.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
- KafkaConsumer<String, String> consumer = new KafkaConsumer<>(properties);
- consumer.assign(
- consumer.partitionsFor(topic).stream()
- .map(partitionInfo -> new TopicPartition(topic, partitionInfo.partition()))
- .collect(Collectors.toSet()));
- consumer.seekToBeginning(consumer.assignment());
- return consumer.poll(Duration.ofMillis(1000));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java
deleted file mode 100644
index 9f1e812..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/**
- * Tests for serializing and deserialzing {@link KafkaCommittable} with {@link
- * KafkaCommittableSerializer}.
- */
-public class KafkaCommittableSerializerTest extends TestLogger {
-
- private static final KafkaCommittableSerializer SERIALIZER = new KafkaCommittableSerializer();
-
- @Test
- public void testCommittableSerDe() throws IOException {
- final String transactionalId = "test-id";
- final short epoch = 5;
- final KafkaCommittable committable = new KafkaCommittable(1L, epoch, transactionalId, null);
- final byte[] serialized = SERIALIZER.serialize(committable);
- assertThat(SERIALIZER.deserialize(1, serialized)).isEqualTo(committable);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java
deleted file mode 100644
index 8def81a..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.Collections;
-import java.util.Properties;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link KafkaCommitter}. */
-@ExtendWith({TestLoggerExtension.class})
-public class KafkaCommitterTest {
-
- private static final int PRODUCER_ID = 0;
- private static final short EPOCH = 0;
- private static final String TRANSACTIONAL_ID = "transactionalId";
-
- /** Causes a network error by inactive broker and tests that a retry will happen. */
- @Test
- public void testRetryCommittableOnRetriableError() throws IOException, InterruptedException {
- Properties properties = getProperties();
- try (final KafkaCommitter committer = new KafkaCommitter(properties);
- FlinkKafkaInternalProducer<Object, Object> producer =
- new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID);
- Recyclable<FlinkKafkaInternalProducer<Object, Object>> recyclable =
- new Recyclable<>(producer, p -> {})) {
- final MockCommitRequest<KafkaCommittable> request =
- new MockCommitRequest<>(
- new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable));
-
- producer.resumeTransaction(PRODUCER_ID, EPOCH);
- committer.commit(Collections.singletonList(request));
-
- assertThat(request.getNumberOfRetries()).isEqualTo(1);
- assertThat(recyclable.isRecycled()).isFalse();
- // FLINK-25531: force the producer to close immediately, else it would take 1 hour
- producer.close(Duration.ZERO);
- }
- }
-
- @Test
- public void testFailJobOnUnknownFatalError() throws IOException, InterruptedException {
- Properties properties = getProperties();
- try (final KafkaCommitter committer = new KafkaCommitter(properties);
- FlinkKafkaInternalProducer<Object, Object> producer =
- new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID);
- Recyclable<FlinkKafkaInternalProducer<Object, Object>> recyclable =
- new Recyclable<>(producer, p -> {})) {
- // will fail because transaction not started
- final MockCommitRequest<KafkaCommittable> request =
- new MockCommitRequest<>(
- new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable));
- committer.commit(Collections.singletonList(request));
- assertThat(request.getFailedWithUnknownReason())
- .isInstanceOf(IllegalStateException.class);
- assertThat(request.getFailedWithUnknownReason().getMessage())
- .contains("Transaction was not started");
- assertThat(recyclable.isRecycled()).isTrue();
- }
- }
-
- @Test
- public void testKafkaCommitterClosesProducer() throws IOException, InterruptedException {
- Properties properties = getProperties();
- FlinkKafkaInternalProducer<Object, Object> producer =
- new FlinkKafkaInternalProducer(properties, TRANSACTIONAL_ID) {
- @Override
- public void commitTransaction() throws ProducerFencedException {}
-
- @Override
- public void flush() {}
-
- @Override
- public void close() {}
- };
- try (final KafkaCommitter committer = new KafkaCommitter(properties);
- Recyclable<FlinkKafkaInternalProducer<Object, Object>> recyclable =
- new Recyclable<>(producer, p -> {})) {
- final MockCommitRequest<KafkaCommittable> request =
- new MockCommitRequest<>(
- new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable));
-
- committer.commit(Collections.singletonList(request));
- assertThat(recyclable.isRecycled()).isTrue();
- }
- }
-
- Properties getProperties() {
- Properties properties = new Properties();
- properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "http://localhost:1");
- // Low timeout will fail commitTransaction quicker
- properties.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "100");
- properties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
- properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
- return properties;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java
deleted file mode 100644
index 614624e..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.connector.testutils.formats.DummyInitializationContext;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.Configurable;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Function;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link KafkaRecordSerializationSchemaBuilder}. */
-public class KafkaRecordSerializationSchemaBuilderTest extends TestLogger {
-
- private static final String DEFAULT_TOPIC = "test";
-
- private static Map<String, ?> configurableConfiguration;
- private static Map<String, ?> configuration;
- private static boolean isKeySerializer;
-
- @Before
- public void setUp() {
- configurableConfiguration = new HashMap<>();
- configuration = new HashMap<>();
- isKeySerializer = false;
- }
-
- @Test
- public void testDoNotAllowMultipleKeySerializer() {
- assertOnlyOneSerializerAllowed(keySerializationSetter());
- }
-
- @Test
- public void testDoNotAllowMultipleValueSerializer() {
- assertOnlyOneSerializerAllowed(valueSerializationSetter());
- }
-
- @Test
- public void testDoNotAllowMultipleTopicSelector() {
- assertThatThrownBy(
- () ->
- KafkaRecordSerializationSchema.builder()
- .setTopicSelector(e -> DEFAULT_TOPIC)
- .setTopic(DEFAULT_TOPIC))
- .isInstanceOf(IllegalStateException.class);
- assertThatThrownBy(
- () ->
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setTopicSelector(e -> DEFAULT_TOPIC))
- .isInstanceOf(IllegalStateException.class);
- }
-
- @Test
- public void testExpectTopicSelector() {
- assertThatThrownBy(
- KafkaRecordSerializationSchema.builder()
- .setValueSerializationSchema(new SimpleStringSchema())
- ::build)
- .isInstanceOf(IllegalStateException.class);
- }
-
- @Test
- public void testExpectValueSerializer() {
- assertThatThrownBy(KafkaRecordSerializationSchema.builder().setTopic(DEFAULT_TOPIC)::build)
- .isInstanceOf(IllegalStateException.class);
- }
-
- @Test
- public void testSerializeRecordWithTopicSelector() {
- final TopicSelector<String> topicSelector =
- (e) -> {
- if (e.equals("a")) {
- return "topic-a";
- }
- return "topic-b";
- };
- final KafkaRecordSerializationSchemaBuilder<String> builder =
- KafkaRecordSerializationSchema.builder().setTopicSelector(topicSelector);
- final SerializationSchema<String> serializationSchema = new SimpleStringSchema();
- final KafkaRecordSerializationSchema<String> schema =
- builder.setValueSerializationSchema(serializationSchema).build();
- final ProducerRecord<byte[], byte[]> record = schema.serialize("a", null, null);
- assertThat(record.topic()).isEqualTo("topic-a");
- assertThat(record.key()).isNull();
- assertThat(record.value()).isEqualTo(serializationSchema.serialize("a"));
-
- final ProducerRecord<byte[], byte[]> record2 = schema.serialize("b", null, null);
- assertThat(record2.topic()).isEqualTo("topic-b");
- assertThat(record2.key()).isNull();
- assertThat(record2.value()).isEqualTo(serializationSchema.serialize("b"));
- }
-
- @Test
- public void testSerializeRecordWithPartitioner() throws Exception {
- AtomicBoolean opened = new AtomicBoolean(false);
- final int partition = 5;
- final FlinkKafkaPartitioner<Object> partitioner =
- new ConstantPartitioner<>(opened, partition);
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setValueSerializationSchema(new SimpleStringSchema())
- .setPartitioner(partitioner)
- .build();
- final KafkaRecordSerializationSchema.KafkaSinkContext sinkContext = new TestSinkContext();
- schema.open(null, sinkContext);
- final ProducerRecord<byte[], byte[]> record = schema.serialize("a", sinkContext, null);
- assertThat(record.partition()).isEqualTo(partition);
- assertThat(opened.get()).isTrue();
- }
-
- @Test
- public void testSerializeRecordWithKey() {
- final SerializationSchema<String> serializationSchema = new SimpleStringSchema();
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setValueSerializationSchema(serializationSchema)
- .setKeySerializationSchema(serializationSchema)
- .build();
- final ProducerRecord<byte[], byte[]> record = schema.serialize("a", null, null);
- assertThat(serializationSchema.serialize("a"))
- .isEqualTo(record.key())
- .isEqualTo(record.value());
- }
-
- @Test
- public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception {
- final Map<String, String> config = ImmutableMap.of("simpleKey", "simpleValue");
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- // Use StringSerializer as dummy Serializer, since ValueSerializer is
- // mandatory.
- .setKafkaValueSerializer(StringSerializer.class, config)
- .setKafkaKeySerializer(SimpleStringSerializer.class, config)
- .build();
- open(schema);
- assertThat(config).isEqualTo(configuration);
- assertThat(isKeySerializer).isTrue();
- assertThat(configurableConfiguration).isEmpty();
- }
-
- @Test
- public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exception {
- final Map<String, String> config = ImmutableMap.of("simpleKey", "simpleValue");
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setKafkaValueSerializer(SimpleStringSerializer.class, config)
- .build();
- open(schema);
- assertThat(config).isEqualTo(configuration);
- assertThat(isKeySerializer).isFalse();
- assertThat(configurableConfiguration).isEmpty();
- }
-
- @Test
- public void testSerializeRecordWithKafkaSerializer() throws Exception {
- final Map<String, String> config = ImmutableMap.of("configKey", "configValue");
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setKafkaValueSerializer(ConfigurableStringSerializer.class, config)
- .build();
- open(schema);
- assertThat(config).isEqualTo(configurableConfiguration);
- assertThat(configuration).isEmpty();
- final Deserializer<String> deserializer = new StringDeserializer();
- final ProducerRecord<byte[], byte[]> record = schema.serialize("a", null, null);
- assertThat(deserializer.deserialize(DEFAULT_TOPIC, record.value())).isEqualTo("a");
- }
-
- @Test
- public void testSerializeRecordWithTimestamp() {
- final SerializationSchema<String> serializationSchema = new SimpleStringSchema();
- final KafkaRecordSerializationSchema<String> schema =
- KafkaRecordSerializationSchema.builder()
- .setTopic(DEFAULT_TOPIC)
- .setValueSerializationSchema(serializationSchema)
- .setKeySerializationSchema(serializationSchema)
- .build();
- final ProducerRecord<byte[], byte[]> recordWithTimestamp =
- schema.serialize("a", null, 100L);
- assertThat((long) recordWithTimestamp.timestamp()).isEqualTo(100L);
-
- final ProducerRecord<byte[], byte[]> recordWithTimestampZero =
- schema.serialize("a", null, 0L);
- assertThat((long) recordWithTimestampZero.timestamp()).isEqualTo(0L);
-
- final ProducerRecord<byte[], byte[]> recordWithoutTimestamp =
- schema.serialize("a", null, null);
- assertThat(recordWithoutTimestamp.timestamp()).isNull();
-
- final ProducerRecord<byte[], byte[]> recordWithInvalidTimestamp =
- schema.serialize("a", null, -100L);
- assertThat(recordWithInvalidTimestamp.timestamp()).isNull();
- }
-
- private static void assertOnlyOneSerializerAllowed(
- List<
- Function<
- KafkaRecordSerializationSchemaBuilder<String>,
- KafkaRecordSerializationSchemaBuilder<String>>>
- serializers) {
- for (final Function<
- KafkaRecordSerializationSchemaBuilder<String>,
- KafkaRecordSerializationSchemaBuilder<String>>
- setter : serializers) {
- final KafkaRecordSerializationSchemaBuilder<String> builder =
- KafkaRecordSerializationSchema.<String>builder().setTopic(DEFAULT_TOPIC);
- setter.apply(builder);
- for (final Function<
- KafkaRecordSerializationSchemaBuilder<String>,
- KafkaRecordSerializationSchemaBuilder<String>>
- updater : serializers) {
- assertThatThrownBy(() -> updater.apply(builder))
- .isInstanceOf(IllegalStateException.class);
- }
- }
- }
-
- private static List<
- Function<
- KafkaRecordSerializationSchemaBuilder<String>,
- KafkaRecordSerializationSchemaBuilder<String>>>
- valueSerializationSetter() {
- return ImmutableList.of(
- (b) -> b.setKafkaValueSerializer(StringSerializer.class),
- (b) -> b.setValueSerializationSchema(new SimpleStringSchema()),
- (b) ->
- b.setKafkaValueSerializer(
- ConfigurableStringSerializer.class, Collections.emptyMap()));
- }
-
- private static List<
- Function<
- KafkaRecordSerializationSchemaBuilder<String>,
- KafkaRecordSerializationSchemaBuilder<String>>>
- keySerializationSetter() {
- return ImmutableList.of(
- (b) -> b.setKafkaKeySerializer(StringSerializer.class),
- (b) -> b.setKeySerializationSchema(new SimpleStringSchema()),
- (b) ->
- b.setKafkaKeySerializer(
- ConfigurableStringSerializer.class, Collections.emptyMap()));
- }
-
- /**
- * Serializer based on Kafka's serialization stack. This is the special case that implements
- * {@link Configurable}
- *
- * <p>This class must be public to make it instantiable by the tests.
- */
- public static class ConfigurableStringSerializer extends StringSerializer
- implements Configurable {
- @Override
- public void configure(Map<String, ?> configs) {
- configurableConfiguration = configs;
- }
- }
-
- /**
- * Serializer based on Kafka's serialization stack.
- *
- * <p>This class must be public to make it instantiable by the tests.
- */
- public static class SimpleStringSerializer extends StringSerializer {
- @Override
- public void configure(Map<String, ?> configs, boolean isKey) {
- configuration = configs;
- isKeySerializer = isKey;
- }
- }
-
- private static class TestSinkContext
- implements KafkaRecordSerializationSchema.KafkaSinkContext {
- @Override
- public int getParallelInstanceId() {
- return 0;
- }
-
- @Override
- public int getNumberOfParallelInstances() {
- return 0;
- }
-
- @Override
- public int[] getPartitionsForTopic(String topic) {
- return new int[0];
- }
- }
-
- private static class ConstantPartitioner<T> extends FlinkKafkaPartitioner<T> {
-
- private final AtomicBoolean opened;
- private final int partition;
-
- ConstantPartitioner(AtomicBoolean opened, int partition) {
- this.opened = opened;
- this.partition = partition;
- }
-
- @Override
- public void open(int parallelInstanceId, int parallelInstances) {
- opened.set(true);
- }
-
- @Override
- public int partition(
- T record, byte[] key, byte[] value, String targetTopic, int[] partitions) {
- return partition;
- }
- }
-
- private void open(KafkaRecordSerializationSchema<String> schema) throws Exception {
- schema.open(new DummyInitializationContext(), null);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java
deleted file mode 100644
index eeecc84..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.junit.jupiter.api.Test;
-
-import java.util.Arrays;
-import java.util.Properties;
-import java.util.function.Consumer;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link KafkaSinkBuilder}. */
-public class KafkaSinkBuilderTest extends TestLogger {
-
- private static final String[] DEFAULT_KEYS =
- new String[] {
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG,
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG
- };
-
- @Test
- public void testPropertyHandling() {
- validateProducerConfig(
- getBasicBuilder(),
- p -> {
- Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k));
- });
-
- validateProducerConfig(
- getBasicBuilder().setProperty("k1", "v1"),
- p -> {
- Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k));
- p.containsKey("k1");
- });
-
- Properties testConf = new Properties();
- testConf.put("k1", "v1");
- testConf.put("k2", "v2");
-
- validateProducerConfig(
- getBasicBuilder().setKafkaProducerConfig(testConf),
- p -> {
- Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k));
- testConf.forEach((k, v) -> assertThat(p.get(k)).isEqualTo(v));
- });
-
- validateProducerConfig(
- getBasicBuilder()
- .setProperty("k1", "incorrect")
- .setKafkaProducerConfig(testConf)
- .setProperty("k2", "correct"),
- p -> {
- Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k));
- assertThat(p).containsEntry("k1", "v1").containsEntry("k2", "correct");
- });
- }
-
- @Test
- public void testBootstrapServerSetting() {
- Properties testConf1 = new Properties();
- testConf1.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "testServer");
-
- validateProducerConfig(
- getNoServerBuilder().setKafkaProducerConfig(testConf1),
- p -> assertThat(p).containsKeys(DEFAULT_KEYS));
- }
-
- private void validateProducerConfig(
- KafkaSinkBuilder<?> builder, Consumer<Properties> validator) {
- validator.accept(builder.build().getKafkaProducerConfig());
- }
-
- private KafkaSinkBuilder<String> getBasicBuilder() {
- return new KafkaSinkBuilder<String>()
- .setBootstrapServers("testServer")
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic("topic")
- .setValueSerializationSchema(new SimpleStringSchema())
- .build());
- }
-
- private KafkaSinkBuilder<String> getNoServerBuilder() {
- return new KafkaSinkBuilder<String>()
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic("topic")
- .setValueSerializationSchema(new SimpleStringSchema())
- .build());
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java
deleted file mode 100644
index 942902f..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java
+++ /dev/null
@@ -1,691 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
-import org.apache.flink.configuration.CheckpointingOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.configuration.StateBackendOptions;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory;
-import org.apache.flink.connector.kafka.testutils.KafkaUtil;
-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.SinkTestSuiteBase;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.CheckpointConfig;
-import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.test.util.TestUtils;
-import org.apache.flink.testutils.junit.SharedObjects;
-import org.apache.flink.testutils.junit.SharedReference;
-import org.apache.flink.util.DockerImageVersions;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.guava30.com.google.common.base.Joiner;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.CreateTopicsResult;
-import org.apache.kafka.clients.admin.DeleteTopicsResult;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.jupiter.api.Nested;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-import org.testcontainers.utility.DockerImageName;
-
-import javax.annotation.Nullable;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-import java.util.stream.LongStream;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Tests for using KafkaSink writing to a Kafka cluster. */
-public class KafkaSinkITCase extends TestLogger {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class);
- private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- private static final Network NETWORK = Network.newNetwork();
- private static final int ZK_TIMEOUT_MILLIS = 30000;
- private static final short TOPIC_REPLICATION_FACTOR = 1;
- private static AdminClient admin;
-
- private String topic;
- private SharedReference<AtomicLong> emittedRecordsCount;
- private SharedReference<AtomicLong> emittedRecordsWithCheckpoint;
- private SharedReference<AtomicBoolean> failed;
- private SharedReference<AtomicLong> lastCheckpointedRecord;
-
- @ClassRule
- public static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG)
- .withEmbeddedZookeeper()
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS);
-
- @Rule public final SharedObjects sharedObjects = SharedObjects.create();
-
- @Rule public final TemporaryFolder temp = new TemporaryFolder();
-
- @BeforeClass
- public static void setupAdmin() {
- Map<String, Object> properties = new HashMap<>();
- properties.put(
- CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
- KAFKA_CONTAINER.getBootstrapServers());
- admin = AdminClient.create(properties);
- }
-
- @AfterClass
- public static void teardownAdmin() {
- admin.close();
- }
-
- @Before
- public void setUp() throws ExecutionException, InterruptedException, TimeoutException {
- emittedRecordsCount = sharedObjects.add(new AtomicLong());
- emittedRecordsWithCheckpoint = sharedObjects.add(new AtomicLong());
- failed = sharedObjects.add(new AtomicBoolean(false));
- lastCheckpointedRecord = sharedObjects.add(new AtomicLong(0));
- topic = UUID.randomUUID().toString();
- createTestTopic(topic, 1, TOPIC_REPLICATION_FACTOR);
- }
-
- @After
- public void tearDown() throws ExecutionException, InterruptedException, TimeoutException {
- deleteTestTopic(topic);
- }
-
- /** Integration test based on connector testing framework. */
- @Nested
- class IntegrationTests extends SinkTestSuiteBase<String> {
- // Defines test environment on Flink MiniCluster
- @SuppressWarnings("unused")
- @TestEnv
- MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
-
- // Defines external system
- @TestExternalSystem
- DefaultContainerizedExternalSystem<KafkaContainer> kafka =
- DefaultContainerizedExternalSystem.builder()
- .fromContainer(
- new KafkaContainer(
- DockerImageName.parse(DockerImageVersions.KAFKA)))
- .build();
-
- @SuppressWarnings("unused")
- @TestSemantics
- CheckpointingMode[] semantics =
- new CheckpointingMode[] {
- CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE
- };
-
- @SuppressWarnings("unused")
- @TestContext
- KafkaSinkExternalContextFactory sinkContext =
- new KafkaSinkExternalContextFactory(kafka.getContainer(), Collections.emptyList());
- }
-
- @Test
- public void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws Exception {
- writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE, emittedRecordsCount);
- }
-
- @Test
- public void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception {
- writeRecordsToKafka(DeliveryGuarantee.NONE, emittedRecordsCount);
- }
-
- @Test
- public void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws Exception {
- writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, emittedRecordsWithCheckpoint);
- }
-
- @Test
- public void testRecoveryWithAtLeastOnceGuarantee() throws Exception {
- testRecoveryWithAssertion(
- DeliveryGuarantee.AT_LEAST_ONCE,
- 1,
- (records) -> assertThat(records).contains(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L));
- }
-
- @Test
- public void testRecoveryWithExactlyOnceGuarantee() throws Exception {
- testRecoveryWithAssertion(
- DeliveryGuarantee.EXACTLY_ONCE,
- 1,
- (records) ->
- assertThat(records)
- .contains(
- (LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
- .boxed()
- .toArray(Long[]::new))));
- }
-
- @Test
- public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception {
- testRecoveryWithAssertion(
- DeliveryGuarantee.EXACTLY_ONCE,
- 2,
- (records) ->
- assertThat(records)
- .contains(
- LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
- .boxed()
- .toArray(Long[]::new)));
- }
-
- @Test
- public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception {
- // Run a first job failing during the async phase of a checkpoint to leave some
- // lingering transactions
- final Configuration config = new Configuration();
- config.setString(StateBackendOptions.STATE_BACKEND, "filesystem");
- final File checkpointDir = temp.newFolder();
- config.setString(
- CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString());
- config.set(
- ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT,
- CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
- config.set(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, 2);
- try {
- executeWithMapper(new FailAsyncCheckpointMapper(1), config, "firstPrefix");
- } catch (Exception e) {
- assertThat(e.getCause().getCause().getMessage())
- .contains("Exceeded checkpoint tolerable failure");
- }
- final File completedCheckpoint = TestUtils.getMostRecentCompletedCheckpoint(checkpointDir);
-
- config.set(SavepointConfigOptions.SAVEPOINT_PATH, completedCheckpoint.toURI().toString());
-
- // Run a second job which aborts all lingering transactions and new consumer should
- // immediately see the newly written records
- failed.get().set(true);
- executeWithMapper(
- new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, "newPrefix");
- final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
- drainAllRecordsFromTopic(topic, true);
- assertThat(deserializeValues(collectedRecords))
- .contains(
- LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
- .boxed()
- .toArray(Long[]::new));
- }
-
- @Test
- public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception {
- // Run a first job opening 5 transactions one per subtask and fail in async checkpoint phase
- final Configuration config = new Configuration();
- config.set(CoreOptions.DEFAULT_PARALLELISM, 5);
- try {
- executeWithMapper(new FailAsyncCheckpointMapper(0), config, null);
- } catch (Exception e) {
- assertThat(e.getCause().getCause().getMessage())
- .contains("Exceeded checkpoint tolerable failure");
- }
- assertThat(deserializeValues(drainAllRecordsFromTopic(topic, true))).isEmpty();
-
- // Second job aborts all transactions from previous runs with higher parallelism
- config.set(CoreOptions.DEFAULT_PARALLELISM, 1);
- failed.get().set(true);
- executeWithMapper(
- new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, null);
- final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
- drainAllRecordsFromTopic(topic, true);
- assertThat(deserializeValues(collectedRecords))
- .contains(
- LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
- .boxed()
- .toArray(Long[]::new));
- }
-
- private void executeWithMapper(
- MapFunction<Long, Long> mapper,
- Configuration config,
- @Nullable String transactionalIdPrefix)
- throws Exception {
- final StreamExecutionEnvironment env = new LocalStreamEnvironment(config);
- env.enableCheckpointing(100L);
- env.setRestartStrategy(RestartStrategies.noRestart());
- final DataStreamSource<Long> source = env.fromSequence(1, 10);
- final DataStream<Long> stream = source.map(mapper);
- final KafkaSinkBuilder<Long> builder =
- new KafkaSinkBuilder<Long>()
- .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)
- .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers())
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(topic)
- .setValueSerializationSchema(new RecordSerializer())
- .build());
- if (transactionalIdPrefix == null) {
- transactionalIdPrefix = "kafka-sink";
- }
- builder.setTransactionalIdPrefix(transactionalIdPrefix);
- stream.sinkTo(builder.build());
- env.execute();
- checkProducerLeak();
- }
-
- private void testRecoveryWithAssertion(
- DeliveryGuarantee guarantee,
- int maxConcurrentCheckpoints,
- java.util.function.Consumer<List<Long>> recordsAssertion)
- throws Exception {
- final StreamExecutionEnvironment env = new LocalStreamEnvironment();
- env.enableCheckpointing(300L);
- env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints);
- DataStreamSource<Long> source = env.fromSequence(1, 10);
- DataStream<Long> stream =
- source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord));
-
- stream.sinkTo(
- new KafkaSinkBuilder<Long>()
- .setDeliveryGuarantee(guarantee)
- .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers())
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(topic)
- .setValueSerializationSchema(new RecordSerializer())
- .build())
- .setTransactionalIdPrefix("kafka-sink")
- .build());
- env.execute();
-
- final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
- drainAllRecordsFromTopic(topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE);
- recordsAssertion.accept(deserializeValues(collectedRecords));
- checkProducerLeak();
- }
-
- private void writeRecordsToKafka(
- DeliveryGuarantee deliveryGuarantee, SharedReference<AtomicLong> expectedRecords)
- throws Exception {
- final StreamExecutionEnvironment env = new LocalStreamEnvironment();
- env.enableCheckpointing(100L);
- final DataStream<Long> source =
- env.addSource(
- new InfiniteIntegerSource(
- emittedRecordsCount, emittedRecordsWithCheckpoint));
- source.sinkTo(
- new KafkaSinkBuilder<Long>()
- .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers())
- .setDeliveryGuarantee(deliveryGuarantee)
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(topic)
- .setValueSerializationSchema(new RecordSerializer())
- .build())
- .setTransactionalIdPrefix("kafka-sink")
- .build());
- env.execute();
-
- final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
- drainAllRecordsFromTopic(
- topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE);
- final long recordsCount = expectedRecords.get().get();
- assertThat(recordsCount).isEqualTo(collectedRecords.size());
- assertThat(deserializeValues(collectedRecords))
- .contains(LongStream.range(1, recordsCount + 1).boxed().toArray(Long[]::new));
- checkProducerLeak();
- }
-
- private static List<Long> deserializeValues(List<ConsumerRecord<byte[], byte[]>> records) {
- return records.stream()
- .map(
- record -> {
- final ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);
- final byte[] value = record.value();
- buffer.put(value, 0, value.length);
- buffer.flip();
- return buffer.getLong();
- })
- .collect(Collectors.toList());
- }
-
- private static Properties getKafkaClientConfiguration() {
- final Properties standardProps = new Properties();
- standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers());
- standardProps.put("group.id", UUID.randomUUID().toString());
- standardProps.put("enable.auto.commit", false);
- standardProps.put("auto.offset.reset", "earliest");
- standardProps.put("max.partition.fetch.bytes", 256);
- standardProps.put("zookeeper.session.timeout.ms", ZK_TIMEOUT_MILLIS);
- standardProps.put("zookeeper.connection.timeout.ms", ZK_TIMEOUT_MILLIS);
- return standardProps;
- }
-
- private void createTestTopic(String topic, int numPartitions, short replicationFactor)
- throws ExecutionException, InterruptedException, TimeoutException {
- final CreateTopicsResult result =
- admin.createTopics(
- Collections.singletonList(
- new NewTopic(topic, numPartitions, replicationFactor)));
- result.all().get();
- }
-
- private void deleteTestTopic(String topic)
- throws ExecutionException, InterruptedException, TimeoutException {
- final DeleteTopicsResult result = admin.deleteTopics(Collections.singletonList(topic));
- result.all().get();
- }
-
- private List<ConsumerRecord<byte[], byte[]>> drainAllRecordsFromTopic(
- String topic, boolean committed) {
- Properties properties = getKafkaClientConfiguration();
- return KafkaUtil.drainAllRecordsFromTopic(topic, properties, committed);
- }
-
- private static class RecordSerializer implements SerializationSchema<Long> {
-
- @Override
- public byte[] serialize(Long element) {
- final ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);
- buffer.putLong(element);
- return buffer.array();
- }
- }
-
- private static class FailAsyncCheckpointMapper
- implements MapFunction<Long, Long>, CheckpointedFunction {
- private static final ListStateDescriptor<Integer> stateDescriptor =
- new ListStateDescriptor<>("test-state", new SlowSerializer());
- private int failAfterCheckpoint;
-
- private ListState<Integer> state;
-
- public FailAsyncCheckpointMapper(int failAfterCheckpoint) {
- this.failAfterCheckpoint = failAfterCheckpoint;
- }
-
- @Override
- public Long map(Long value) throws Exception {
- Thread.sleep(100);
- return value;
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext context) throws Exception {
- state.clear();
- if (failAfterCheckpoint <= 0) {
- // Trigger a failure in the serializer
- state.add(-1);
- } else {
- state.add(1);
- }
- failAfterCheckpoint--;
- }
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {
- state = context.getOperatorStateStore().getListState(stateDescriptor);
- }
- }
-
- private static class SlowSerializer extends TypeSerializerSingleton<Integer> {
-
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public Integer createInstance() {
- return 1;
- }
-
- @Override
- public Integer copy(Integer from) {
- return from;
- }
-
- @Override
- public Integer copy(Integer from, Integer reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return 0;
- }
-
- @Override
- public void serialize(Integer record, DataOutputView target) throws IOException {
- if (record != -1) {
- return;
- }
- throw new RuntimeException("Expected failure during async checkpoint phase");
- }
-
- @Override
- public Integer deserialize(DataInputView source) throws IOException {
- return 1;
- }
-
- @Override
- public Integer deserialize(Integer reuse, DataInputView source) throws IOException {
- return 1;
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {}
-
- @Override
- public TypeSerializerSnapshot<Integer> snapshotConfiguration() {
- return new SlowSerializerSnapshot();
- }
- }
-
- /** Snapshot used in {@link FailAsyncCheckpointMapper}. */
- public static class SlowSerializerSnapshot extends SimpleTypeSerializerSnapshot<Integer> {
- public SlowSerializerSnapshot() {
- super(SlowSerializer::new);
- }
- }
-
- /** Fails after a checkpoint is taken and the next record was emitted. */
- private static class FailingCheckpointMapper
- implements MapFunction<Long, Long>, CheckpointListener, CheckpointedFunction {
-
- private final SharedReference<AtomicBoolean> failed;
- private final SharedReference<AtomicLong> lastCheckpointedRecord;
-
- private volatile long lastSeenRecord;
- private volatile long checkpointedRecord;
- private volatile long lastCheckpointId = 0;
- private final AtomicInteger emittedBetweenCheckpoint = new AtomicInteger(0);
-
- FailingCheckpointMapper(
- SharedReference<AtomicBoolean> failed,
- SharedReference<AtomicLong> lastCheckpointedRecord) {
- this.failed = failed;
- this.lastCheckpointedRecord = lastCheckpointedRecord;
- }
-
- @Override
- public Long map(Long value) throws Exception {
- lastSeenRecord = value;
- if (lastCheckpointId >= 1
- && emittedBetweenCheckpoint.get() > 0
- && !failed.get().get()) {
- failed.get().set(true);
- throw new RuntimeException("Planned exception.");
- }
- // Delay execution to ensure that at-least one checkpoint is triggered before finish
- Thread.sleep(50);
- emittedBetweenCheckpoint.incrementAndGet();
- return value;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- LOG.info("notifyCheckpointComplete {} @ {}", checkpointedRecord, checkpointId);
- lastCheckpointId = checkpointId;
- emittedBetweenCheckpoint.set(0);
- lastCheckpointedRecord.get().set(checkpointedRecord);
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext context) throws Exception {
- LOG.info("snapshotState {} @ {}", lastSeenRecord, context.getCheckpointId());
- checkpointedRecord = lastSeenRecord;
- }
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {}
- }
-
- private void checkProducerLeak() throws InterruptedException {
- List<Map.Entry<Thread, StackTraceElement[]>> leaks = null;
- for (int tries = 0; tries < 10; tries++) {
- leaks =
- Thread.getAllStackTraces().entrySet().stream()
- .filter(this::findAliveKafkaThread)
- .collect(Collectors.toList());
- if (leaks.isEmpty()) {
- return;
- }
- Thread.sleep(1000);
- }
-
- for (Map.Entry<Thread, StackTraceElement[]> leak : leaks) {
- leak.getKey().stop();
- }
- fail(
- "Detected producer leaks:\n"
- + leaks.stream().map(this::format).collect(Collectors.joining("\n\n")));
- }
-
- private String format(Map.Entry<Thread, StackTraceElement[]> leak) {
- return leak.getKey().getName() + ":\n" + Joiner.on("\n").join(leak.getValue());
- }
-
- private boolean findAliveKafkaThread(Map.Entry<Thread, StackTraceElement[]> threadStackTrace) {
- return threadStackTrace.getKey().getState() != Thread.State.TERMINATED
- && threadStackTrace.getKey().getName().contains("kafka-producer-network-thread");
- }
-
- /**
- * Exposes information about how man records have been emitted overall and finishes after
- * receiving the checkpoint completed event.
- */
- private static final class InfiniteIntegerSource
- implements SourceFunction<Long>, CheckpointListener, CheckpointedFunction {
-
- private final SharedReference<AtomicLong> emittedRecordsCount;
- private final SharedReference<AtomicLong> emittedRecordsWithCheckpoint;
-
- private volatile boolean running = true;
- private volatile long temp;
- private Object lock;
-
- InfiniteIntegerSource(
- SharedReference<AtomicLong> emittedRecordsCount,
- SharedReference<AtomicLong> emittedRecordsWithCheckpoint) {
- this.emittedRecordsCount = emittedRecordsCount;
- this.emittedRecordsWithCheckpoint = emittedRecordsWithCheckpoint;
- }
-
- @Override
- public void run(SourceContext<Long> ctx) throws Exception {
- lock = ctx.getCheckpointLock();
- while (running) {
- synchronized (lock) {
- ctx.collect(emittedRecordsCount.get().addAndGet(1));
- Thread.sleep(1);
- }
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- emittedRecordsWithCheckpoint.get().set(temp);
- running = false;
- LOG.info("notifyCheckpointCompleted {}", checkpointId);
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext context) throws Exception {
- temp = emittedRecordsCount.get().get();
- LOG.info(
- "snapshotState, {}, {}",
- context.getCheckpointId(),
- emittedRecordsCount.get().get());
- }
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {}
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLog.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLog.java
deleted file mode 100644
index 440d785..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLog.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME;
-
-/**
- * This class is responsible to provide the format of the used transationalIds and in case of an
- * application restart query the open transactions and decide which must be aborted.
- */
-class KafkaTransactionLog {
-
- private static final int SUPPORTED_KAFKA_SCHEMA_VERSION = 0;
- private final Properties consumerConfig;
-
- /**
- * Constructor creating a KafkaTransactionLog.
- *
- * @param kafkaConfig used to configure the {@link KafkaConsumer} to query the topic containing
- * the transaction information
- */
- KafkaTransactionLog(Properties kafkaConfig) {
- this.consumerConfig = new Properties();
- consumerConfig.putAll(checkNotNull(kafkaConfig, "kafkaConfig"));
- consumerConfig.put("key.deserializer", ByteArrayDeserializer.class.getName());
- consumerConfig.put("value.deserializer", ByteArrayDeserializer.class.getName());
- consumerConfig.put("enable.auto.commit", false);
- }
-
- public List<TransactionRecord> getTransactions() {
- return getTransactions(id -> true);
- }
-
- /** Gets all {@link TransactionRecord} matching the given id filter. */
- public List<TransactionRecord> getTransactions(Predicate<String> transactionIdFilter)
- throws KafkaException {
- return drainAllRecordsFromTopic(TRANSACTION_STATE_TOPIC_NAME, consumerConfig, true).stream()
- .map(r -> parseTransaction(r, transactionIdFilter))
- .filter(Optional::isPresent)
- .map(Optional::get)
- .collect(Collectors.toList());
- }
-
- private Optional<TransactionRecord> parseTransaction(
- ConsumerRecord<byte[], byte[]> consumerRecord, Predicate<String> transactionIdFilter) {
- final ByteBuffer keyBuffer = ByteBuffer.wrap(consumerRecord.key());
- checkKafkaSchemaVersionMatches(keyBuffer);
- // Ignore 2 bytes because Kafka's internal representation
- keyBuffer.getShort();
- final String transactionalId = StandardCharsets.UTF_8.decode(keyBuffer).toString();
-
- if (!transactionIdFilter.test(transactionalId)) {
- return Optional.empty();
- }
-
- final ByteBuffer valueBuffer = ByteBuffer.wrap(consumerRecord.value());
- checkKafkaSchemaVersionMatches(valueBuffer);
- final TransactionState state = TransactionState.fromByte(readTransactionState(valueBuffer));
-
- return Optional.of(new TransactionRecord(transactionalId, state));
- }
-
- private static byte readTransactionState(ByteBuffer buffer) {
- // producerId
- buffer.getLong();
- // epoch
- buffer.getShort();
- // transactionTimeout
- buffer.getInt();
- // statusKey
- return buffer.get();
- }
-
- public static class TransactionRecord {
- private final String transactionId;
- private final TransactionState state;
-
- public TransactionRecord(String transactionId, TransactionState state) {
- this.transactionId = checkNotNull(transactionId);
- this.state = checkNotNull(state);
- }
-
- public String getTransactionId() {
- return transactionId;
- }
-
- public TransactionState getState() {
- return state;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- TransactionRecord that = (TransactionRecord) o;
- return transactionId.equals(that.transactionId) && state == that.state;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(transactionId, state);
- }
-
- @Override
- public String toString() {
- return "TransactionRecord{"
- + "transactionId='"
- + transactionId
- + '\''
- + ", state="
- + state
- + '}';
- }
- }
-
- public enum TransactionState {
- Empty(Byte.parseByte("0"), false),
- Ongoing(Byte.parseByte("1"), false),
- PrepareCommit(Byte.parseByte("2"), false),
- PrepareAbort(Byte.parseByte("3"), false),
- CompleteCommit(Byte.parseByte("4"), true),
- CompleteAbort(Byte.parseByte("5"), true),
- Dead(Byte.parseByte("6"), true),
- PrepareEpochFence(Byte.parseByte("7"), false);
-
- private static final Map<Byte, TransactionState> BYTE_TO_STATE =
- Arrays.stream(TransactionState.values())
- .collect(Collectors.toMap(e -> e.state, e -> e));
-
- private final byte state;
-
- private boolean terminal;
-
- TransactionState(byte state, boolean terminal) {
- this.state = state;
- this.terminal = terminal;
- }
-
- public boolean isTerminal() {
- return terminal;
- }
-
- static TransactionState fromByte(byte state) {
- final TransactionState transactionState = BYTE_TO_STATE.get(state);
- if (transactionState == null) {
- throw new IllegalArgumentException(
- String.format("The given state %s is not supported.", state));
- }
- return transactionState;
- }
- }
-
- private static void checkKafkaSchemaVersionMatches(ByteBuffer buffer) {
- final short version = buffer.getShort();
- if (version != SUPPORTED_KAFKA_SCHEMA_VERSION) {
- throw new IllegalStateException(
- String.format(
- "Kafka has changed the schema version from %s to %s",
- SUPPORTED_KAFKA_SCHEMA_VERSION, version));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java
deleted file mode 100644
index 84d732d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionRecord;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
-import org.junit.After;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.function.Consumer;
-
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.CompleteAbort;
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.CompleteCommit;
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Empty;
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Ongoing;
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareAbort;
-import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareCommit;
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link KafkaTransactionLog} to retrieve abortable Kafka transactions. */
-public class KafkaTransactionLogITCase extends TestLogger {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class);
- private static final String TOPIC_NAME = "kafkaTransactionLogTest";
- private static final String TRANSACTIONAL_ID_PREFIX = "kafka-log";
-
- @ClassRule
- public static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper();
-
- private final List<Producer<byte[], Integer>> openProducers = new ArrayList<>();
-
- @After
- public void tearDown() {
- openProducers.forEach(Producer::close);
- }
-
- @Test
- public void testGetTransactions() {
- committedTransaction(1);
- abortedTransaction(2);
- lingeringTransaction(3);
- lingeringTransaction(4);
-
- final KafkaTransactionLog transactionLog =
- new KafkaTransactionLog(getKafkaClientConfiguration());
- final List<TransactionRecord> transactions = transactionLog.getTransactions();
- assertThat(transactions)
- .containsExactlyInAnyOrder(
- new TransactionRecord(buildTransactionalId(1), Empty),
- new TransactionRecord(buildTransactionalId(1), Ongoing),
- new TransactionRecord(buildTransactionalId(1), PrepareCommit),
- new TransactionRecord(buildTransactionalId(1), CompleteCommit),
- new TransactionRecord(buildTransactionalId(2), Empty),
- new TransactionRecord(buildTransactionalId(2), Ongoing),
- new TransactionRecord(buildTransactionalId(2), PrepareAbort),
- new TransactionRecord(buildTransactionalId(2), CompleteAbort),
- new TransactionRecord(buildTransactionalId(3), Empty),
- new TransactionRecord(buildTransactionalId(3), Ongoing),
- new TransactionRecord(buildTransactionalId(4), Empty),
- new TransactionRecord(buildTransactionalId(4), Ongoing));
- }
-
- private void committedTransaction(long id) {
- submitTransaction(
- id,
- producer -> {
- producer.initTransactions();
- producer.beginTransaction();
- producer.send(new ProducerRecord<>(TOPIC_NAME, 0, null, null, 1));
- producer.flush();
- producer.commitTransaction();
- producer.flush();
- });
- }
-
- private void lingeringTransaction(long id) {
- submitTransaction(
- id,
- producer -> {
- producer.initTransactions();
- producer.beginTransaction();
- producer.send(new ProducerRecord<>(TOPIC_NAME, 0, null, null, 1));
- producer.flush();
- });
- }
-
- private void abortedTransaction(long id) {
- submitTransaction(
- id,
- producer -> {
- producer.initTransactions();
- producer.beginTransaction();
- producer.send(new ProducerRecord<>(TOPIC_NAME, 0, null, null, 1));
- producer.flush();
- producer.abortTransaction();
- producer.flush();
- });
- }
-
- private void submitTransaction(long id, Consumer<Producer<byte[], Integer>> producerAction) {
- Producer<byte[], Integer> producer = createProducer(buildTransactionalId(id));
- openProducers.add(producer);
- producerAction.accept(producer);
- // don't close here for lingering transactions
- }
-
- private static String buildTransactionalId(long id) {
- return TRANSACTIONAL_ID_PREFIX + id;
- }
-
- private static Producer<byte[], Integer> createProducer(String transactionalId) {
- final Properties producerProperties = getKafkaClientConfiguration();
- producerProperties.put(
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
- producerProperties.put(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName());
- producerProperties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
- return new KafkaProducer<>(producerProperties);
- }
-
- private static Properties getKafkaClientConfiguration() {
- final Properties standardProps = new Properties();
- standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers());
- standardProps.put("group.id", "flink-tests");
- standardProps.put("enable.auto.commit", false);
- standardProps.put("auto.id.reset", "earliest");
- standardProps.put("max.partition.fetch.bytes", 256);
- return standardProps;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java
deleted file mode 100644
index c9d226d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java
+++ /dev/null
@@ -1,673 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.api.common.operators.ProcessingTimeService;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.connector.sink2.SinkWriter;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.base.sink.writer.TestSinkInitContext;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
-import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
-import org.apache.flink.metrics.testutils.MetricListener;
-import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
-import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
-import org.apache.flink.util.TestLoggerExtension;
-import org.apache.flink.util.UserCodeClassLoader;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.TestInfo;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.EnumSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Optional;
-import java.util.OptionalLong;
-import java.util.PriorityQueue;
-import java.util.Properties;
-import java.util.concurrent.ScheduledFuture;
-import java.util.function.Consumer;
-import java.util.stream.IntStream;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode;
-
-/** Tests for the standalone KafkaWriter. */
-@ExtendWith(TestLoggerExtension.class)
-public class KafkaWriterITCase {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaWriterITCase.class);
- private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- private static final Network NETWORK = Network.newNetwork();
- private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total";
- private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext();
- private String topic;
-
- private MetricListener metricListener;
- private TriggerTimeService timeService;
-
- private static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG)
- .withEmbeddedZookeeper()
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS);
-
- @BeforeAll
- public static void beforeAll() {
- KAFKA_CONTAINER.start();
- }
-
- @AfterAll
- public static void afterAll() {
- KAFKA_CONTAINER.stop();
- }
-
- @BeforeEach
- public void setUp(TestInfo testInfo) {
- metricListener = new MetricListener();
- timeService = new TriggerTimeService();
- topic = testInfo.getDisplayName().replaceAll("\\W", "");
- }
-
- @ParameterizedTest
- @EnumSource(DeliveryGuarantee.class)
- public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception {
- try (final KafkaWriter<Integer> ignored =
- createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) {
- assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME).isPresent()).isTrue();
- }
- }
-
- @ParameterizedTest
- @EnumSource(DeliveryGuarantee.class)
- public void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception {
- assertKafkaMetricNotPresent(guarantee, "flink.disable-metrics", "true");
- assertKafkaMetricNotPresent(guarantee, "register.producer.metrics", "false");
- }
-
- @Test
- public void testIncreasingRecordBasedCounters() throws Exception {
- final OperatorIOMetricGroup operatorIOMetricGroup =
- UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup();
- final InternalSinkWriterMetricGroup metricGroup =
- InternalSinkWriterMetricGroup.mock(
- metricListener.getMetricGroup(), operatorIOMetricGroup);
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) {
- final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter();
- final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter();
- final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter();
- final Counter numRecordsSendErrors = metricGroup.getNumRecordsSendErrorsCounter();
- assertThat(numBytesOut.getCount()).isEqualTo(0L);
- assertThat(numRecordsOut.getCount()).isEqualTo(0);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0);
- assertThat(numRecordsSendErrors.getCount()).isEqualTo(0);
-
- // elements for which the serializer returns null should be silently skipped
- writer.write(null, SINK_WRITER_CONTEXT);
- timeService.trigger();
- assertThat(numBytesOut.getCount()).isEqualTo(0L);
- assertThat(numRecordsOut.getCount()).isEqualTo(0);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0);
- assertThat(numRecordsSendErrors.getCount()).isEqualTo(0);
-
- // but elements for which a non-null producer record is returned should count
- writer.write(1, SINK_WRITER_CONTEXT);
- timeService.trigger();
- assertThat(numRecordsOut.getCount()).isEqualTo(1);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0);
- assertThat(numRecordsSendErrors.getCount()).isEqualTo(0);
- assertThat(numBytesOut.getCount()).isGreaterThan(0L);
- }
- }
-
- @Test
- public void testCurrentSendTimeMetric() throws Exception {
- final InternalSinkWriterMetricGroup metricGroup =
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup());
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(),
- DeliveryGuarantee.AT_LEAST_ONCE,
- metricGroup)) {
- final Optional<Gauge<Long>> currentSendTime =
- metricListener.getGauge("currentSendTime");
- assertThat(currentSendTime.isPresent()).isTrue();
- assertThat(currentSendTime.get().getValue()).isEqualTo(0L);
- IntStream.range(0, 100)
- .forEach(
- (run) -> {
- try {
- writer.write(1, SINK_WRITER_CONTEXT);
- // Manually flush the records to generate a sendTime
- if (run % 10 == 0) {
- writer.flush(false);
- }
- } catch (IOException | InterruptedException e) {
- throw new RuntimeException("Failed writing Kafka record.");
- }
- });
- assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);
- }
- }
-
- @Test
- void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception {
- Properties properties = getKafkaClientConfiguration();
-
- SinkInitContext sinkInitContext =
- new SinkInitContext(
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()),
- timeService,
- null);
- final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext);
- final Counter numRecordsOutErrors =
- sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L);
-
- triggerProducerException(writer, properties);
-
- // test flush
- assertThatCode(() -> writer.flush(false))
- .hasRootCauseExactlyInstanceOf(ProducerFencedException.class);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
-
- assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT))
- .as("the exception is not thrown again")
- .doesNotThrowAnyException();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
- }
-
- @Test
- void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception {
- Properties properties = getKafkaClientConfiguration();
-
- SinkInitContext sinkInitContext =
- new SinkInitContext(
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()),
- timeService,
- null);
- final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext);
- final Counter numRecordsOutErrors =
- sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L);
-
- triggerProducerException(writer, properties);
- // to ensure that the exceptional send request has completed
- writer.getCurrentProducer().flush();
-
- assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT))
- .hasRootCauseExactlyInstanceOf(ProducerFencedException.class);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
-
- assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT))
- .as("the exception is not thrown again")
- .doesNotThrowAnyException();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
- }
-
- @Test
- void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception {
- Properties properties = getKafkaClientConfiguration();
-
- SinkInitContext sinkInitContext =
- new SinkInitContext(
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()),
- timeService,
- null);
- final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext);
- final Counter numRecordsOutErrors =
- sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L);
-
- triggerProducerException(writer, properties);
- // to ensure that the exceptional send request has completed
- writer.getCurrentProducer().flush();
-
- while (sinkInitContext.getMailboxExecutor().tryYield()) {
- // execute all mails
- }
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
-
- assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT))
- .as("the exception is not thrown again")
- .doesNotThrowAnyException();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
- }
-
- @Test
- void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception {
- Properties properties = getKafkaClientConfiguration();
-
- SinkInitContext sinkInitContext =
- new SinkInitContext(
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()),
- timeService,
- null);
- final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext);
- final Counter numRecordsOutErrors =
- sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter();
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L);
-
- triggerProducerException(writer, properties);
- // to ensure that the exceptional send request has completed
- writer.getCurrentProducer().flush();
-
- // test flush
- assertThatCode(writer::close)
- .as("flush should throw the exception from the WriterCallback")
- .hasRootCauseExactlyInstanceOf(ProducerFencedException.class);
- assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L);
- }
-
- private void triggerProducerException(KafkaWriter<Integer> writer, Properties properties)
- throws IOException {
- final String transactionalId = writer.getCurrentProducer().getTransactionalId();
-
- try (FlinkKafkaInternalProducer<byte[], byte[]> producer =
- new FlinkKafkaInternalProducer<>(properties, transactionalId)) {
- producer.initTransactions();
- producer.beginTransaction();
- producer.send(new ProducerRecord<byte[], byte[]>(topic, "1".getBytes()));
- producer.commitTransaction();
- }
-
- writer.write(1, SINK_WRITER_CONTEXT);
- }
-
- @Test
- public void testMetadataPublisher() throws Exception {
- List<String> metadataList = new ArrayList<>();
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(),
- DeliveryGuarantee.AT_LEAST_ONCE,
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()),
- meta -> metadataList.add(meta.toString()))) {
- List<String> expected = new ArrayList<>();
- for (int i = 0; i < 100; i++) {
- writer.write(1, SINK_WRITER_CONTEXT);
- expected.add("testMetadataPublisher-0@" + i);
- }
- writer.flush(false);
- assertThat(metadataList).usingRecursiveComparison().isEqualTo(expected);
- }
- }
-
- /** Test that producer is not accidentally recreated or pool is used. */
- @Test
- void testLingeringTransaction() throws Exception {
- final KafkaWriter<Integer> failedWriter =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE);
-
- // create two lingering transactions
- failedWriter.flush(false);
- failedWriter.prepareCommit();
- failedWriter.snapshotState(1);
- failedWriter.flush(false);
- failedWriter.prepareCommit();
- failedWriter.snapshotState(2);
-
- try (final KafkaWriter<Integer> recoveredWriter =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) {
- recoveredWriter.write(1, SINK_WRITER_CONTEXT);
-
- recoveredWriter.flush(false);
- Collection<KafkaCommittable> committables = recoveredWriter.prepareCommit();
- recoveredWriter.snapshotState(1);
- assertThat(committables).hasSize(1);
- final KafkaCommittable committable = committables.stream().findFirst().get();
- assertThat(committable.getProducer().isPresent()).isTrue();
-
- committable.getProducer().get().getObject().commitTransaction();
-
- List<ConsumerRecord<byte[], byte[]>> records =
- drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true);
- assertThat(records).hasSize(1);
- }
-
- failedWriter.close();
- }
-
- /** Test that producer is not accidentally recreated or pool is used. */
- @ParameterizedTest
- @EnumSource(
- value = DeliveryGuarantee.class,
- names = "EXACTLY_ONCE",
- mode = EnumSource.Mode.EXCLUDE)
- void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception {
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) {
- assertThat(writer.getProducerPool()).hasSize(0);
-
- FlinkKafkaInternalProducer<byte[], byte[]> firstProducer = writer.getCurrentProducer();
- writer.flush(false);
- Collection<KafkaCommittable> committables = writer.prepareCommit();
- writer.snapshotState(0);
- assertThat(committables).hasSize(0);
-
- assertThat(writer.getCurrentProducer() == firstProducer)
- .as("Expected same producer")
- .isTrue();
- assertThat(writer.getProducerPool()).hasSize(0);
- }
- }
-
- /** Test that producers are reused when committed. */
- @Test
- void usePoolForTransactional() throws Exception {
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(
- getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) {
- assertThat(writer.getProducerPool()).hasSize(0);
-
- writer.flush(false);
- Collection<KafkaCommittable> committables0 = writer.prepareCommit();
- writer.snapshotState(1);
- assertThat(committables0).hasSize(1);
- final KafkaCommittable committable = committables0.stream().findFirst().get();
- assertThat(committable.getProducer().isPresent()).isTrue();
-
- FlinkKafkaInternalProducer<?, ?> firstProducer =
- committable.getProducer().get().getObject();
- assertThat(firstProducer != writer.getCurrentProducer())
- .as("Expected different producer")
- .isTrue();
-
- // recycle first producer, KafkaCommitter would commit it and then return it
- assertThat(writer.getProducerPool()).hasSize(0);
- firstProducer.commitTransaction();
- committable.getProducer().get().close();
- assertThat(writer.getProducerPool()).hasSize(1);
-
- writer.flush(false);
- Collection<KafkaCommittable> committables1 = writer.prepareCommit();
- writer.snapshotState(2);
- assertThat(committables1).hasSize(1);
- final KafkaCommittable committable1 = committables1.stream().findFirst().get();
- assertThat(committable1.getProducer().isPresent()).isTrue();
-
- assertThat(firstProducer == writer.getCurrentProducer())
- .as("Expected recycled producer")
- .isTrue();
- }
- }
-
- /**
- * Tests that open transactions are automatically aborted on close such that successive writes
- * succeed.
- */
- @Test
- void testAbortOnClose() throws Exception {
- Properties properties = getKafkaClientConfiguration();
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) {
- writer.write(1, SINK_WRITER_CONTEXT);
- assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0);
- }
-
- try (final KafkaWriter<Integer> writer =
- createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) {
- writer.write(2, SINK_WRITER_CONTEXT);
- writer.flush(false);
- Collection<KafkaCommittable> committables = writer.prepareCommit();
- writer.snapshotState(1L);
-
- // manually commit here, which would only succeed if the first transaction was aborted
- assertThat(committables).hasSize(1);
- final KafkaCommittable committable = committables.stream().findFirst().get();
- String transactionalId = committable.getTransactionalId();
- try (FlinkKafkaInternalProducer<byte[], byte[]> producer =
- new FlinkKafkaInternalProducer<>(properties, transactionalId)) {
- producer.resumeTransaction(committable.getProducerId(), committable.getEpoch());
- producer.commitTransaction();
- }
-
- assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1);
- }
- }
-
- private void assertKafkaMetricNotPresent(
- DeliveryGuarantee guarantee, String configKey, String configValue) throws Exception {
- final Properties config = getKafkaClientConfiguration();
- config.put(configKey, configValue);
- try (final KafkaWriter<Integer> ignored =
- createWriterWithConfiguration(config, guarantee)) {
- assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent();
- }
- }
-
- private KafkaWriter<Integer> createWriterWithConfiguration(
- Properties config, DeliveryGuarantee guarantee) {
- return createWriterWithConfiguration(
- config,
- guarantee,
- InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()));
- }
-
- private KafkaWriter<Integer> createWriterWithConfiguration(
- Properties config,
- DeliveryGuarantee guarantee,
- SinkWriterMetricGroup sinkWriterMetricGroup) {
- return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null);
- }
-
- private KafkaWriter<Integer> createWriterWithConfiguration(
- Properties config,
- DeliveryGuarantee guarantee,
- SinkWriterMetricGroup sinkWriterMetricGroup,
- @Nullable Consumer<RecordMetadata> metadataConsumer) {
- return new KafkaWriter<>(
- guarantee,
- config,
- "test-prefix",
- new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer),
- new DummyRecordSerializer(),
- new DummySchemaContext(),
- ImmutableList.of());
- }
-
- private KafkaWriter<Integer> createWriterWithConfiguration(
- Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) {
- return new KafkaWriter<>(
- guarantee,
- config,
- "test-prefix",
- sinkInitContext,
- new DummyRecordSerializer(),
- new DummySchemaContext(),
- ImmutableList.of());
- }
-
- private static Properties getKafkaClientConfiguration() {
- final Properties standardProps = new Properties();
- standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers());
- standardProps.put("group.id", "kafkaWriter-tests");
- standardProps.put("enable.auto.commit", false);
- standardProps.put("key.serializer", ByteArraySerializer.class.getName());
- standardProps.put("value.serializer", ByteArraySerializer.class.getName());
- standardProps.put("auto.offset.reset", "earliest");
- return standardProps;
- }
-
- private static class SinkInitContext extends TestSinkInitContext {
-
- private final SinkWriterMetricGroup metricGroup;
- private final ProcessingTimeService timeService;
- @Nullable private final Consumer<RecordMetadata> metadataConsumer;
-
- SinkInitContext(
- SinkWriterMetricGroup metricGroup,
- ProcessingTimeService timeService,
- @Nullable Consumer<RecordMetadata> metadataConsumer) {
- this.metricGroup = metricGroup;
- this.timeService = timeService;
- this.metadataConsumer = metadataConsumer;
- }
-
- @Override
- public UserCodeClassLoader getUserCodeClassLoader() {
- throw new UnsupportedOperationException("Not implemented.");
- }
-
- @Override
- public ProcessingTimeService getProcessingTimeService() {
- return timeService;
- }
-
- @Override
- public int getSubtaskId() {
- return 0;
- }
-
- @Override
- public int getNumberOfParallelSubtasks() {
- return 1;
- }
-
- @Override
- public int getAttemptNumber() {
- return 0;
- }
-
- @Override
- public SinkWriterMetricGroup metricGroup() {
- return metricGroup;
- }
-
- @Override
- public OptionalLong getRestoredCheckpointId() {
- return OptionalLong.empty();
- }
-
- @Override
- public SerializationSchema.InitializationContext
- asSerializationSchemaInitializationContext() {
- return null;
- }
-
- @Override
- public <MetaT> Optional<Consumer<MetaT>> metadataConsumer() {
- return Optional.ofNullable((Consumer<MetaT>) metadataConsumer);
- }
- }
-
- private class DummyRecordSerializer implements KafkaRecordSerializationSchema<Integer> {
- @Override
- public ProducerRecord<byte[], byte[]> serialize(
- Integer element, KafkaSinkContext context, Long timestamp) {
- if (element == null) {
- // in general, serializers should be allowed to skip invalid elements
- return null;
- }
- return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array());
- }
- }
-
- private static class DummySchemaContext implements SerializationSchema.InitializationContext {
-
- @Override
- public MetricGroup getMetricGroup() {
- throw new UnsupportedOperationException("Not implemented.");
- }
-
- @Override
- public UserCodeClassLoader getUserCodeClassLoader() {
- throw new UnsupportedOperationException("Not implemented.");
- }
- }
-
- private static class DummySinkWriterContext implements SinkWriter.Context {
- @Override
- public long currentWatermark() {
- return 0;
- }
-
- @Override
- public Long timestamp() {
- return null;
- }
- }
-
- private static class TriggerTimeService implements ProcessingTimeService {
-
- private final PriorityQueue<Tuple2<Long, ProcessingTimeCallback>> registeredCallbacks =
- new PriorityQueue<>(Comparator.comparingLong(o -> o.f0));
-
- @Override
- public long getCurrentProcessingTime() {
- return 0;
- }
-
- @Override
- public ScheduledFuture<?> registerTimer(
- long time, ProcessingTimeCallback processingTimerCallback) {
- registeredCallbacks.add(new Tuple2<>(time, processingTimerCallback));
- return null;
- }
-
- public void trigger() throws Exception {
- final Tuple2<Long, ProcessingTimeCallback> registered = registeredCallbacks.poll();
- if (registered == null) {
- LOG.warn("Triggered time service but no callback was registered.");
- return;
- }
- registered.f1.onProcessingTime(registered.f0);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java
deleted file mode 100644
index 3df0ea8..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/**
- * Tests for serializing and deserialzing {@link KafkaWriterState} with {@link
- * KafkaWriterStateSerializer}.
- */
-public class KafkaWriterStateSerializerTest extends TestLogger {
-
- private static final KafkaWriterStateSerializer SERIALIZER = new KafkaWriterStateSerializer();
-
- @Test
- public void testStateSerDe() throws IOException {
- final KafkaWriterState state = new KafkaWriterState("idPrefix");
- final byte[] serialized = SERIALIZER.serialize(state);
- assertThat(SERIALIZER.deserialize(1, serialized)).isEqualTo(state);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java
deleted file mode 100644
index 982d58e..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link TransactionalIdFactory}. */
-public class TransactionIdFactoryTest extends TestLogger {
-
- @Test
- public void testBuildTransactionalId() {
- final String expected = "prefix-0-2";
- assertThat(TransactionalIdFactory.buildTransactionalId("prefix", 0, 2L))
- .isEqualTo(expected);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java
deleted file mode 100644
index 1b65c9d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
-
-import org.junit.Test;
-
-import java.util.List;
-import java.util.Map;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link TransactionsToAbortChecker}. */
-public class TransactionToAbortCheckerTest extends TestLogger {
-
- public static final String ABORT = "abort";
-
- @Test
- public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() {
- final TransactionsToAbortChecker checker =
- new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L, 2, 3L), 0);
-
- // abort recovered subtasksId with equal or higher checkpoint offset
- final Map<Integer, Map<Long, String>> openTransactions =
- ImmutableMap.of(
- 0, ImmutableMap.of(2L, ABORT, 1L, ABORT),
- 2, ImmutableMap.of(3L, ABORT, 4L, ABORT),
- 3, ImmutableMap.of(3L, "keep", 4L, "keep"));
-
- final List<String> transactionsToAbort = checker.getTransactionsToAbort(openTransactions);
- assertThat(transactionsToAbort).hasSize(4);
- assertThatAbortCorrectTransaction(transactionsToAbort);
- }
-
- @Test
- public void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() {
- final TransactionsToAbortChecker checker =
- new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L), 0);
-
- // abort recovered subtasksId with equal or higher checkpoint offset
- final Map<Integer, Map<Long, String>> openTransactions =
- ImmutableMap.of(
- 0, ImmutableMap.of(2L, ABORT, 1L, ABORT),
- 2, ImmutableMap.of(1L, ABORT),
- 3, ImmutableMap.of(1L, "keep"),
- 4, ImmutableMap.of(1L, ABORT),
- 5, ImmutableMap.of(1L, "keep"));
-
- final List<String> transactionsToAbort = checker.getTransactionsToAbort(openTransactions);
- assertThat(transactionsToAbort).hasSize(4);
- assertThatAbortCorrectTransaction(transactionsToAbort);
- }
-
- private static void assertThatAbortCorrectTransaction(List<String> abortedTransactions) {
- assertThat(abortedTransactions.stream().allMatch(t -> t.equals(ABORT))).isTrue();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaDataReader.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaDataReader.java
deleted file mode 100644
index 0847cd4..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaDataReader.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.sink.testutils;
-
-import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.WakeupException;
-
-import java.time.Duration;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Properties;
-
-/** Kafka dataStream data reader. */
-public class KafkaDataReader implements ExternalSystemDataReader<String> {
- private final KafkaConsumer<String, String> consumer;
-
- public KafkaDataReader(Properties properties, Collection<TopicPartition> partitions) {
- this.consumer = new KafkaConsumer<>(properties);
- consumer.assign(partitions);
- consumer.seekToBeginning(partitions);
- }
-
- @Override
- public List<String> poll(Duration timeout) {
- List<String> result = new LinkedList<>();
- ConsumerRecords<String, String> consumerRecords;
- try {
- consumerRecords = consumer.poll(timeout);
- } catch (WakeupException we) {
- return Collections.emptyList();
- }
- Iterator<ConsumerRecord<String, String>> iterator = consumerRecords.iterator();
- while (iterator.hasNext()) {
- result.add(iterator.next().value());
- }
- return result;
- }
-
- @Override
- public void close() throws Exception {
- if (consumer != null) {
- consumer.close();
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContext.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContext.java
deleted file mode 100644
index ae26048..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContext.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * 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.sink.testutils;
-
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.sink2.Sink;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder;
-import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
-import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
-import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
-import org.apache.flink.streaming.api.CheckpointingMode;
-
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.AdminClientConfig;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.TopicPartitionInfo;
-import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.streaming.api.CheckpointingMode.EXACTLY_ONCE;
-
-/** A Kafka external context that will create only one topic and use partitions in that topic. */
-public class KafkaSinkExternalContext implements DataStreamSinkV2ExternalContext<String> {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkExternalContext.class);
-
- private static final String TOPIC_NAME_PREFIX = "kafka-single-topic";
- private static final int RANDOM_STRING_MAX_LENGTH = 50;
- private static final int NUM_RECORDS_UPPER_BOUND = 500;
- private static final int NUM_RECORDS_LOWER_BOUND = 100;
- private static final int DEFAULT_TRANSACTION_TIMEOUT_IN_MS = 900000;
-
- protected String bootstrapServers;
- protected final String topicName;
-
- private final List<ExternalSystemDataReader<String>> readers = new ArrayList<>();
-
- protected int numSplits = 0;
-
- private List<URL> connectorJarPaths;
-
- protected final AdminClient kafkaAdminClient;
-
- public KafkaSinkExternalContext(String bootstrapServers, List<URL> connectorJarPaths) {
- this.bootstrapServers = bootstrapServers;
- this.connectorJarPaths = connectorJarPaths;
- this.topicName =
- TOPIC_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
- kafkaAdminClient = createAdminClient();
- }
-
- private void createTopic(String topicName, int numPartitions, short replicationFactor) {
- LOG.debug(
- "Creating new Kafka topic {} with {} partitions and {} replicas",
- topicName,
- numPartitions,
- replicationFactor);
- NewTopic newTopic = new NewTopic(topicName, numPartitions, replicationFactor);
- try {
- kafkaAdminClient.createTopics(Collections.singletonList(newTopic)).all().get();
- } catch (Exception e) {
- throw new RuntimeException(String.format("Cannot create topic '%s'", topicName), e);
- }
- }
-
- private void deleteTopic(String topicName) {
- LOG.debug("Deleting Kafka topic {}", topicName);
- try {
- kafkaAdminClient.deleteTopics(Collections.singletonList(topicName)).all().get();
- } catch (Exception e) {
- if (ExceptionUtils.getRootCause(e) instanceof UnknownTopicOrPartitionException) {
- throw new RuntimeException(
- String.format("Cannot delete unknown Kafka topic '%s'", topicName), e);
- }
- }
- }
-
- private AdminClient createAdminClient() {
- final Properties config = new Properties();
- config.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- return AdminClient.create(config);
- }
-
- @Override
- public Sink<String> createSink(TestingSinkSettings sinkSettings) {
- if (!topicExists(topicName)) {
- createTopic(topicName, 4, (short) 1);
- }
-
- KafkaSinkBuilder<String> builder = KafkaSink.builder();
- final Properties properties = new Properties();
- properties.put(
- ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, DEFAULT_TRANSACTION_TIMEOUT_IN_MS);
- builder.setBootstrapServers(bootstrapServers)
- .setDeliveryGuarantee(toDeliveryGuarantee(sinkSettings.getCheckpointingMode()))
- .setTransactionalIdPrefix("testingFramework")
- .setKafkaProducerConfig(properties)
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(topicName)
- .setValueSerializationSchema(new SimpleStringSchema())
- .build());
- return builder.build();
- }
-
- @Override
- public ExternalSystemDataReader<String> createSinkDataReader(TestingSinkSettings sinkSettings) {
- LOG.info("Fetching information for topic: {}", topicName);
- final Map<String, TopicDescription> topicMetadata =
- getTopicMetadata(Arrays.asList(topicName));
-
- Set<TopicPartition> subscribedPartitions = new HashSet<>();
- for (TopicDescription topic : topicMetadata.values()) {
- for (TopicPartitionInfo partition : topic.partitions()) {
- subscribedPartitions.add(new TopicPartition(topic.name(), partition.partition()));
- }
- }
-
- Properties properties = new Properties();
- properties.setProperty(
- ConsumerConfig.GROUP_ID_CONFIG,
- "flink-kafka-test" + subscribedPartitions.hashCode());
- properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- properties.setProperty(
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
- StringDeserializer.class.getCanonicalName());
- properties.setProperty(
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
- StringDeserializer.class.getCanonicalName());
- if (EXACTLY_ONCE.equals(sinkSettings.getCheckpointingMode())) {
- // default is read_uncommitted
- properties.setProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
- }
- properties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
- readers.add(new KafkaDataReader(properties, subscribedPartitions));
- return readers.get(readers.size() - 1);
- }
-
- @Override
- public List<String> generateTestData(TestingSinkSettings sinkSettings, long seed) {
- Random random = new Random(seed);
- List<String> randomStringRecords = new ArrayList<>();
- int recordNum =
- random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND)
- + NUM_RECORDS_LOWER_BOUND;
- for (int i = 0; i < recordNum; i++) {
- int stringLength = random.nextInt(RANDOM_STRING_MAX_LENGTH) + 1;
- randomStringRecords.add(RandomStringUtils.random(stringLength, true, true));
- }
- return randomStringRecords;
- }
-
- protected Map<String, TopicDescription> getTopicMetadata(List<String> topics) {
- try {
- return kafkaAdminClient.describeTopics(topics).allTopicNames().get();
- } catch (Exception e) {
- throw new RuntimeException(
- String.format("Failed to get metadata for topics %s.", topics), e);
- }
- }
-
- private boolean topicExists(String topic) {
- try {
- kafkaAdminClient.describeTopics(Arrays.asList(topic)).allTopicNames().get();
- return true;
- } catch (Exception e) {
- return false;
- }
- }
-
- @Override
- public void close() {
- if (numSplits != 0) {
- deleteTopic(topicName);
- }
- readers.stream()
- .filter(Objects::nonNull)
- .forEach(
- reader -> {
- try {
- reader.close();
- } catch (Exception e) {
- if (kafkaAdminClient != null) {
- kafkaAdminClient.close();
- }
- throw new RuntimeException("Cannot close split writer", e);
- }
- });
- readers.clear();
- if (kafkaAdminClient != null) {
- kafkaAdminClient.close();
- }
- }
-
- @Override
- public String toString() {
- return "Single-topic Kafka";
- }
-
- @Override
- public List<URL> getConnectorJarPaths() {
- return connectorJarPaths;
- }
-
- @Override
- public TypeInformation<String> getProducedType() {
- return TypeInformation.of(String.class);
- }
-
- private DeliveryGuarantee toDeliveryGuarantee(CheckpointingMode checkpointingMode) {
- switch (checkpointingMode) {
- case EXACTLY_ONCE:
- return DeliveryGuarantee.EXACTLY_ONCE;
- case AT_LEAST_ONCE:
- return DeliveryGuarantee.AT_LEAST_ONCE;
- default:
- throw new IllegalArgumentException(
- String.format(
- "Only exactly-once and al-least-once checkpointing mode are supported, but actual is %s.",
- checkpointingMode));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContextFactory.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContextFactory.java
deleted file mode 100644
index b795854..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/testutils/KafkaSinkExternalContextFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.sink.testutils;
-
-import org.apache.flink.connector.testframe.external.ExternalContextFactory;
-
-import org.testcontainers.containers.KafkaContainer;
-
-import java.net.URL;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/** Kafka sink external context factory. */
-public class KafkaSinkExternalContextFactory
- implements ExternalContextFactory<KafkaSinkExternalContext> {
-
- private final KafkaContainer kafkaContainer;
- private final List<URL> connectorJars;
-
- public KafkaSinkExternalContextFactory(KafkaContainer kafkaContainer, List<URL> connectorJars) {
- this.kafkaContainer = kafkaContainer;
- this.connectorJars = connectorJars;
- }
-
- private String getBootstrapServer() {
- final String internalEndpoints =
- kafkaContainer.getNetworkAliases().stream()
- .map(host -> String.join(":", host, Integer.toString(9092)))
- .collect(Collectors.joining(","));
- return String.join(",", kafkaContainer.getBootstrapServers(), internalEndpoints);
- }
-
- @Override
- public KafkaSinkExternalContext createExternalContext(String testName) {
- return new KafkaSinkExternalContext(getBootstrapServer(), connectorJars);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java
deleted file mode 100644
index 236e961..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * 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.configuration.ConfigOptions;
-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.reader.deserializer.KafkaRecordDeserializationSchema;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link KafkaSourceBuilder}. */
-@ExtendWith(TestLoggerExtension.class)
-public class KafkaSourceBuilderTest {
-
- @Test
- public void testBuildSourceWithGroupId() {
- final KafkaSource<String> kafkaSource = getBasicBuilder().setGroupId("groupId").build();
- // Commit on checkpoint should be enabled by default
- assertThat(
- kafkaSource
- .getConfiguration()
- .get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT))
- .isTrue();
- // Auto commit should be disabled by default
- assertThat(
- kafkaSource
- .getConfiguration()
- .get(
- ConfigOptions.key(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
- .booleanType()
- .noDefaultValue()))
- .isFalse();
- }
-
- @Test
- public void testBuildSourceWithoutGroupId() {
- final KafkaSource<String> kafkaSource = getBasicBuilder().build();
- // Commit on checkpoint and auto commit should be disabled because group.id is not specified
- assertThat(
- kafkaSource
- .getConfiguration()
- .get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT))
- .isFalse();
- assertThat(
- kafkaSource
- .getConfiguration()
- .get(
- ConfigOptions.key(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
- .booleanType()
- .noDefaultValue()))
- .isFalse();
- }
-
- @Test
- public void testEnableCommitOnCheckpointWithoutGroupId() {
- assertThatThrownBy(
- () ->
- getBasicBuilder()
- .setProperty(
- KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT
- .key(),
- "true")
- .build())
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Property group.id is required when offset commit is enabled");
- }
-
- @Test
- public void testEnableAutoCommitWithoutGroupId() {
- assertThatThrownBy(
- () ->
- getBasicBuilder()
- .setProperty(
- ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
- .build())
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Property group.id is required when offset commit is enabled");
- }
-
- @Test
- public void testDisableOffsetCommitWithoutGroupId() {
- getBasicBuilder()
- .setProperty(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false")
- .build();
- getBasicBuilder().setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false").build();
- }
-
- @Test
- public void testUsingCommittedOffsetsInitializerWithoutGroupId() {
- // Using OffsetsInitializer#committedOffsets as starting offsets
- assertThatThrownBy(
- () ->
- getBasicBuilder()
- .setStartingOffsets(OffsetsInitializer.committedOffsets())
- .build())
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Property group.id is required when using committed offset for offsets initializer");
-
- // Using OffsetsInitializer#committedOffsets as stopping offsets
- assertThatThrownBy(
- () ->
- getBasicBuilder()
- .setBounded(OffsetsInitializer.committedOffsets())
- .build())
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Property group.id is required when using committed offset for offsets initializer");
-
- // Using OffsetsInitializer#offsets to manually specify committed offset as starting offset
- assertThatThrownBy(
- () -> {
- final Map<TopicPartition, Long> offsetMap = new HashMap<>();
- offsetMap.put(
- new TopicPartition("topic", 0),
- KafkaPartitionSplit.COMMITTED_OFFSET);
- getBasicBuilder()
- .setStartingOffsets(OffsetsInitializer.offsets(offsetMap))
- .build();
- })
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Property group.id is required because partition topic-0 is initialized with committed offset");
- }
-
- @Test
- public void testSettingCustomKafkaSubscriber() {
- ExampleCustomSubscriber exampleCustomSubscriber = new ExampleCustomSubscriber();
- KafkaSourceBuilder<String> customKafkaSubscriberBuilder =
- new KafkaSourceBuilder<String>()
- .setBootstrapServers("testServer")
- .setKafkaSubscriber(exampleCustomSubscriber)
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(
- StringDeserializer.class));
-
- assertThat(customKafkaSubscriberBuilder.build().getKafkaSubscriber())
- .isEqualTo(exampleCustomSubscriber);
-
- assertThatThrownBy(() -> customKafkaSubscriberBuilder.setTopics("topic"))
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Cannot use topics for consumption because a ExampleCustomSubscriber is already set for consumption.");
-
- assertThatThrownBy(
- () -> customKafkaSubscriberBuilder.setTopicPattern(Pattern.compile(".+")))
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Cannot use topic pattern for consumption because a ExampleCustomSubscriber is already set for consumption.");
-
- assertThatThrownBy(
- () ->
- customKafkaSubscriberBuilder.setPartitions(
- Collections.singleton(new TopicPartition("topic", 0))))
- .isInstanceOf(IllegalStateException.class)
- .hasMessageContaining(
- "Cannot use partitions for consumption because a ExampleCustomSubscriber is already set for consumption.");
- }
-
- private KafkaSourceBuilder<String> getBasicBuilder() {
- return new KafkaSourceBuilder<String>()
- .setBootstrapServers("testServer")
- .setTopics("topic")
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class));
- }
-
- private static class ExampleCustomSubscriber implements KafkaSubscriber {
-
- @Override
- public Set<TopicPartition> getSubscribedTopicPartitions(AdminClient adminClient) {
- return Collections.singleton(new TopicPartition("topic", 0));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java
deleted file mode 100644
index e37aefe..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java
+++ /dev/null
@@ -1,532 +0,0 @@
-/*
- * 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.api.common.JobExecutionResult;
-import org.apache.flink.api.common.accumulators.ListAccumulator;
-import org.apache.flink.api.common.eventtime.Watermark;
-import org.apache.flink.api.common.eventtime.WatermarkGenerator;
-import org.apache.flink.api.common.eventtime.WatermarkOutput;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.configuration.Configuration;
-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.KafkaSourceExternalContextFactory;
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-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.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.ProcessFunction;
-import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.CloseableIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.DockerImageVersions;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.serialization.IntegerDeserializer;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Nested;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.TestInstance;
-import org.junit.jupiter.api.TestInstance.Lifecycle;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION;
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Unite test class for {@link KafkaSource}. */
-public class KafkaSourceITCase {
- private static final String TOPIC1 = "topic1";
- private static final String TOPIC2 = "topic2";
-
- @Nested
- @TestInstance(Lifecycle.PER_CLASS)
- class KafkaSpecificTests {
- @BeforeAll
- public void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- KafkaSourceTestEnv.setupTopic(
- TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopicWithoutTimestamp);
- KafkaSourceTestEnv.setupTopic(
- TOPIC2, true, true, KafkaSourceTestEnv::getRecordsForTopicWithoutTimestamp);
- }
-
- @AfterAll
- public void tearDown() throws Exception {
- KafkaSourceTestEnv.tearDown();
- }
-
- @ParameterizedTest(name = "Object reuse in deserializer = {arguments}")
- @ValueSource(booleans = {false, true})
- public void testTimestamp(boolean enableObjectReuse) throws Throwable {
- final String topic =
- "testTimestamp-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE);
- final long currentTimestamp = System.currentTimeMillis();
- KafkaSourceTestEnv.createTestTopic(topic, 1, 1);
- KafkaSourceTestEnv.produceToKafka(
- Arrays.asList(
- new ProducerRecord<>(topic, 0, currentTimestamp + 1L, "key0", 0),
- new ProducerRecord<>(topic, 0, currentTimestamp + 2L, "key1", 1),
- new ProducerRecord<>(topic, 0, currentTimestamp + 3L, "key2", 2)));
-
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setGroupId("testTimestampAndWatermark")
- .setTopics(topic)
- .setDeserializer(
- new TestingKafkaRecordDeserializationSchema(enableObjectReuse))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- DataStream<PartitionAndValue> stream =
- env.fromSource(source, WatermarkStrategy.noWatermarks(), "testTimestamp");
-
- // Verify that the timestamp and watermark are working fine.
- stream.transform(
- "timestampVerifier",
- TypeInformation.of(PartitionAndValue.class),
- new WatermarkVerifyingOperator(v -> v));
- stream.addSink(new DiscardingSink<>());
- JobExecutionResult result = env.execute();
-
- assertThat(result.<List<Long>>getAccumulatorResult("timestamp"))
- .containsExactly(
- currentTimestamp + 1L, currentTimestamp + 2L, currentTimestamp + 3L);
- }
-
- @ParameterizedTest(name = "Object reuse in deserializer = {arguments}")
- @ValueSource(booleans = {false, true})
- public void testBasicRead(boolean enableObjectReuse) throws Exception {
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setGroupId("testBasicRead")
- .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- .setDeserializer(
- new TestingKafkaRecordDeserializationSchema(enableObjectReuse))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- DataStream<PartitionAndValue> stream =
- env.fromSource(source, WatermarkStrategy.noWatermarks(), "testBasicRead");
- executeAndVerify(env, stream);
- }
-
- @Test
- public void testValueOnlyDeserializer() throws Exception {
- KafkaSource<Integer> source =
- KafkaSource.<Integer>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setGroupId("testValueOnlyDeserializer")
- .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(
- IntegerDeserializer.class))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
-
- try (CloseableIterator<Integer> resultIterator =
- env.fromSource(
- source,
- WatermarkStrategy.noWatermarks(),
- "testValueOnlyDeserializer")
- .executeAndCollect()) {
- AtomicInteger actualSum = new AtomicInteger();
- resultIterator.forEachRemaining(actualSum::addAndGet);
-
- // Calculate the actual sum of values
- // Values in a partition should start from partition ID, and end with
- // (NUM_RECORDS_PER_PARTITION - 1)
- // e.g. Values in partition 5 should be {5, 6, 7, 8, 9}
- int expectedSum = 0;
- for (int partition = 0;
- partition < KafkaSourceTestEnv.NUM_PARTITIONS;
- partition++) {
- for (int value = partition;
- value < KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION;
- value++) {
- expectedSum += value;
- }
- }
-
- // Since we have two topics, the expected sum value should be doubled
- expectedSum *= 2;
-
- assertThat(actualSum.get()).isEqualTo(expectedSum);
- }
- }
-
- @ParameterizedTest(name = "Object reuse in deserializer = {arguments}")
- @ValueSource(booleans = {false, true})
- public void testRedundantParallelism(boolean enableObjectReuse) throws Exception {
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setGroupId("testRedundantParallelism")
- .setTopics(Collections.singletonList(TOPIC1))
- .setDeserializer(
- new TestingKafkaRecordDeserializationSchema(enableObjectReuse))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- // Here we use (NUM_PARTITION + 1) as the parallelism, so one SourceReader will not be
- // assigned with any splits. The redundant SourceReader should also be signaled with a
- // NoMoreSplitsEvent and eventually spins to FINISHED state.
- env.setParallelism(KafkaSourceTestEnv.NUM_PARTITIONS + 1);
- DataStream<PartitionAndValue> stream =
- env.fromSource(
- source, WatermarkStrategy.noWatermarks(), "testRedundantParallelism");
- executeAndVerify(env, stream);
- }
-
- @ParameterizedTest(name = "Object reuse in deserializer = {arguments}")
- @ValueSource(booleans = {false, true})
- public void testBasicReadWithoutGroupId(boolean enableObjectReuse) throws Exception {
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setTopics(Arrays.asList(TOPIC1, TOPIC2))
- .setDeserializer(
- new TestingKafkaRecordDeserializationSchema(enableObjectReuse))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- DataStream<PartitionAndValue> stream =
- env.fromSource(
- source,
- WatermarkStrategy.noWatermarks(),
- "testBasicReadWithoutGroupId");
- executeAndVerify(env, stream);
- }
-
- @Test
- public void testPerPartitionWatermark() throws Throwable {
- String watermarkTopic = "watermarkTestTopic-" + UUID.randomUUID();
- KafkaSourceTestEnv.createTestTopic(watermarkTopic, 2, 1);
- List<ProducerRecord<String, Integer>> records =
- Arrays.asList(
- new ProducerRecord<>(watermarkTopic, 0, 100L, null, 100),
- new ProducerRecord<>(watermarkTopic, 0, 200L, null, 200),
- new ProducerRecord<>(watermarkTopic, 0, 300L, null, 300),
- new ProducerRecord<>(watermarkTopic, 1, 150L, null, 150),
- new ProducerRecord<>(watermarkTopic, 1, 250L, null, 250),
- new ProducerRecord<>(watermarkTopic, 1, 350L, null, 350));
- KafkaSourceTestEnv.produceToKafka(records);
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setTopics(watermarkTopic)
- .setGroupId("watermark-test")
- .setDeserializer(new TestingKafkaRecordDeserializationSchema(false))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.fromSource(
- source,
- WatermarkStrategy.forGenerator(
- (context) -> new OnEventWatermarkGenerator()),
- "testPerPartitionWatermark")
- .process(
- new ProcessFunction<PartitionAndValue, Long>() {
- @Override
- public void processElement(
- PartitionAndValue value,
- ProcessFunction<PartitionAndValue, Long>.Context ctx,
- Collector<Long> out) {
- assertThat(ctx.timestamp())
- .as(
- "Event time should never behind watermark "
- + "because of per-split watermark multiplexing logic")
- .isGreaterThanOrEqualTo(
- ctx.timerService().currentWatermark());
- }
- });
- env.execute();
- }
-
- @Test
- public void testConsumingEmptyTopic() throws Throwable {
- String emptyTopic = "emptyTopic-" + UUID.randomUUID();
- KafkaSourceTestEnv.createTestTopic(emptyTopic, 3, 1);
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setTopics(emptyTopic)
- .setGroupId("empty-topic-test")
- .setDeserializer(new TestingKafkaRecordDeserializationSchema(false))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- try (CloseableIterator<PartitionAndValue> iterator =
- env.fromSource(
- source,
- WatermarkStrategy.noWatermarks(),
- "testConsumingEmptyTopic")
- .executeAndCollect()) {
- assertThat(iterator.hasNext()).isFalse();
- }
- }
-
- @Test
- public void testConsumingTopicWithEmptyPartitions() throws Throwable {
- String topicWithEmptyPartitions = "topicWithEmptyPartitions-" + UUID.randomUUID();
- KafkaSourceTestEnv.createTestTopic(
- topicWithEmptyPartitions, KafkaSourceTestEnv.NUM_PARTITIONS, 1);
- List<ProducerRecord<String, Integer>> records =
- KafkaSourceTestEnv.getRecordsForTopicWithoutTimestamp(topicWithEmptyPartitions);
- // Only keep records in partition 5
- int partitionWithRecords = 5;
- records.removeIf(record -> record.partition() != partitionWithRecords);
- KafkaSourceTestEnv.produceToKafka(records);
- KafkaSourceTestEnv.setupEarliestOffsets(
- Collections.singletonList(
- new TopicPartition(topicWithEmptyPartitions, partitionWithRecords)));
-
- KafkaSource<PartitionAndValue> source =
- KafkaSource.<PartitionAndValue>builder()
- .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
- .setTopics(topicWithEmptyPartitions)
- .setGroupId("topic-with-empty-partition-test")
- .setDeserializer(new TestingKafkaRecordDeserializationSchema(false))
- .setStartingOffsets(OffsetsInitializer.earliest())
- .setBounded(OffsetsInitializer.latest())
- .build();
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(2);
- executeAndVerify(
- env,
- env.fromSource(
- source,
- WatermarkStrategy.noWatermarks(),
- "testConsumingTopicWithEmptyPartitions"));
- }
- }
-
- /** Integration test based on connector testing framework. */
- @Nested
- class IntegrationTests extends SourceTestSuiteBase<String> {
- @TestSemantics
- CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
-
- // Defines test environment on Flink MiniCluster
- @SuppressWarnings("unused")
- @TestEnv
- MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
-
- // Defines external system
- @TestExternalSystem
- DefaultContainerizedExternalSystem<KafkaContainer> kafka =
- DefaultContainerizedExternalSystem.builder()
- .fromContainer(
- new KafkaContainer(
- DockerImageName.parse(DockerImageVersions.KAFKA)))
- .build();
-
- // Defines 2 External context Factories, so test cases will be invoked twice using these two
- // kinds of external contexts.
- @SuppressWarnings("unused")
- @TestContext
- KafkaSourceExternalContextFactory singleTopic =
- new KafkaSourceExternalContextFactory(
- kafka.getContainer(), Collections.emptyList(), PARTITION);
-
- @SuppressWarnings("unused")
- @TestContext
- KafkaSourceExternalContextFactory multipleTopic =
- new KafkaSourceExternalContextFactory(
- kafka.getContainer(), Collections.emptyList(), TOPIC);
- }
-
- // -----------------
-
- private static class PartitionAndValue implements Serializable {
- private static final long serialVersionUID = 4813439951036021779L;
- private String tp;
- private int value;
-
- public PartitionAndValue() {}
-
- private PartitionAndValue(TopicPartition tp, int value) {
- this.tp = tp.toString();
- this.value = value;
- }
- }
-
- private static class TestingKafkaRecordDeserializationSchema
- implements KafkaRecordDeserializationSchema<PartitionAndValue> {
- private static final long serialVersionUID = -3765473065594331694L;
- private transient Deserializer<Integer> deserializer;
- private final boolean enableObjectReuse;
- private final PartitionAndValue reuse = new PartitionAndValue();
-
- public TestingKafkaRecordDeserializationSchema(boolean enableObjectReuse) {
- this.enableObjectReuse = enableObjectReuse;
- }
-
- @Override
- public void deserialize(
- ConsumerRecord<byte[], byte[]> record, Collector<PartitionAndValue> collector)
- throws IOException {
- if (deserializer == null) {
- deserializer = new IntegerDeserializer();
- }
-
- if (enableObjectReuse) {
- reuse.tp = new TopicPartition(record.topic(), record.partition()).toString();
- reuse.value = deserializer.deserialize(record.topic(), record.value());
- collector.collect(reuse);
- } else {
- collector.collect(
- new PartitionAndValue(
- new TopicPartition(record.topic(), record.partition()),
- deserializer.deserialize(record.topic(), record.value())));
- }
- }
-
- @Override
- public TypeInformation<PartitionAndValue> getProducedType() {
- return TypeInformation.of(PartitionAndValue.class);
- }
- }
-
- private static class WatermarkVerifyingOperator
- extends StreamMap<PartitionAndValue, PartitionAndValue> {
-
- public WatermarkVerifyingOperator(
- MapFunction<PartitionAndValue, PartitionAndValue> mapper) {
- super(mapper);
- }
-
- private static final long serialVersionUID = 2868223355944228209L;
-
- @Override
- public void open() throws Exception {
- getRuntimeContext().addAccumulator("timestamp", new ListAccumulator<Long>());
- }
-
- @Override
- public void processElement(StreamRecord<PartitionAndValue> element) {
- getRuntimeContext().getAccumulator("timestamp").add(element.getTimestamp());
- }
- }
-
- private void executeAndVerify(
- StreamExecutionEnvironment env, DataStream<PartitionAndValue> stream) throws Exception {
- stream.addSink(
- new RichSinkFunction<PartitionAndValue>() {
- @Override
- public void open(Configuration parameters) {
- getRuntimeContext()
- .addAccumulator("result", new ListAccumulator<PartitionAndValue>());
- }
-
- @Override
- public void invoke(PartitionAndValue value, Context context) {
- getRuntimeContext().getAccumulator("result").add(value);
- }
- });
- List<PartitionAndValue> result = env.execute().getAccumulatorResult("result");
- Map<String, List<Integer>> resultPerPartition = new HashMap<>();
- result.forEach(
- partitionAndValue ->
- resultPerPartition
- .computeIfAbsent(partitionAndValue.tp, ignored -> new ArrayList<>())
- .add(partitionAndValue.value));
-
- // Expected elements from partition P should be an integer sequence from P to
- // NUM_RECORDS_PER_PARTITION.
- resultPerPartition.forEach(
- (tp, values) -> {
- int firstExpectedValue =
- Integer.parseInt(tp.substring(tp.lastIndexOf('-') + 1));
- for (int i = 0; i < values.size(); i++) {
- assertThat((int) values.get(i))
- .as(
- String.format(
- "The %d-th value for partition %s should be %d",
- i, tp, firstExpectedValue + i))
- .isEqualTo(firstExpectedValue + i);
- }
- });
- }
-
- private static class OnEventWatermarkGenerator
- implements WatermarkGenerator<PartitionAndValue> {
- @Override
- public void onEvent(PartitionAndValue event, long eventTimestamp, WatermarkOutput output) {
- output.emitWatermark(new Watermark(eventTimestamp));
- }
-
- @Override
- public void onPeriodicEmit(WatermarkOutput output) {}
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java
deleted file mode 100644
index 5cc0ddf..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.connectors.kafka.KafkaConsumerTestBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaProducerTestBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl;
-
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * An IT case class that runs all the IT cases of the legacy {@link
- * org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer} with the new {@link KafkaSource}.
- */
-public class KafkaSourceLegacyITCase extends KafkaConsumerTestBase {
-
- public KafkaSourceLegacyITCase() throws Exception {
- super(true);
- }
-
- @BeforeClass
- public static void prepare() throws Exception {
- KafkaProducerTestBase.prepare();
- ((KafkaTestEnvironmentImpl) kafkaServer)
- .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
- }
-
- @Test
- public void testFailOnNoBroker() throws Exception {
- runFailOnNoBrokerTest();
- }
-
- @Test
- public void testConcurrentProducerConsumerTopology() throws Exception {
- runSimpleConcurrentProducerConsumerTopology();
- }
-
- @Test
- public void testKeyValueSupport() throws Exception {
- runKeyValueTest();
- }
-
- // --- canceling / failures ---
-
- @Test
- public void testCancelingEmptyTopic() throws Exception {
- runCancelingOnEmptyInputTest();
- }
-
- @Test
- public void testCancelingFullTopic() throws Exception {
- runCancelingOnFullInputTest();
- }
-
- // --- source to partition mappings and exactly once ---
-
- @Test
- public void testOneToOneSources() throws Exception {
- runOneToOneExactlyOnceTest();
- }
-
- @Test
- public void testOneSourceMultiplePartitions() throws Exception {
- runOneSourceMultiplePartitionsExactlyOnceTest();
- }
-
- @Test
- public void testMultipleSourcesOnePartition() throws Exception {
- runMultipleSourcesOnePartitionExactlyOnceTest();
- }
-
- // --- broker failure ---
-
- @Test
- @Ignore("FLINK-28267")
- public void testBrokerFailure() throws Exception {
- runBrokerFailureTest();
- }
-
- // --- special executions ---
-
- @Test
- public void testBigRecordJob() throws Exception {
- runBigRecordTestTopology();
- }
-
- @Test
- public void testMultipleTopicsWithLegacySerializer() throws Exception {
- runProduceConsumeMultipleTopics(true);
- }
-
- @Test
- public void testMultipleTopicsWithKafkaSerializer() throws Exception {
- runProduceConsumeMultipleTopics(false);
- }
-
- @Test
- public void testAllDeletes() throws Exception {
- runAllDeletesTest();
- }
-
- // --- startup mode ---
-
- @Test
- public void testStartFromEarliestOffsets() throws Exception {
- runStartFromEarliestOffsets();
- }
-
- @Test
- public void testStartFromLatestOffsets() throws Exception {
- runStartFromLatestOffsets();
- }
-
- @Test
- public void testStartFromGroupOffsets() throws Exception {
- runStartFromGroupOffsets();
- }
-
- @Test
- public void testStartFromSpecificOffsets() throws Exception {
- runStartFromSpecificOffsets();
- }
-
- @Test
- public void testStartFromTimestamp() throws Exception {
- runStartFromTimestamp();
- }
-
- // --- offset committing ---
-
- @Test
- public void testCommitOffsetsToKafka() throws Exception {
- runCommitOffsetsToKafka();
- }
-
- @Test
- public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
- runAutoOffsetRetrievalAndCommitToKafka();
- }
-
- @Test
- public void testCollectingSchema() throws Exception {
- runCollectingSchemaTest();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTestUtils.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTestUtils.java
deleted file mode 100644
index e95f05b..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTestUtils.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.api.connector.source.SourceReaderContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
-import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader;
-
-import java.util.Collection;
-import java.util.function.Consumer;
-
-/** Utility class for testing {@link KafkaSource}. */
-public class KafkaSourceTestUtils {
-
- /**
- * Create {@link KafkaSourceReader} with a custom hook handling IDs of finished {@link
- * org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit}.
- *
- * @param kafkaSource Kafka source
- * @param sourceReaderContext Context for SourceReader
- * @param splitFinishedHook Hook for handling finished splits
- * @param <T> Type of emitting records
- */
- public static <T> KafkaSourceReader<T> createReaderWithFinishedSplitHook(
- KafkaSource<T> kafkaSource,
- SourceReaderContext sourceReaderContext,
- Consumer<Collection<String>> splitFinishedHook)
- throws Exception {
- return ((KafkaSourceReader<T>)
- kafkaSource.createReader(sourceReaderContext, splitFinishedHook));
- }
-
- /** Get configuration of KafkaSource. */
- public static Configuration getKafkaSourceConfiguration(KafkaSource<?> kafkaSource) {
- return kafkaSource.getConfiguration();
- }
-
- /** Get stopping offsets initializer. */
- public static OffsetsInitializer getStoppingOffsetsInitializer(KafkaSource<?> kafkaSource) {
- return kafkaSource.getStoppingOffsetsInitializer();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
deleted file mode 100644
index 13363aa..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
+++ /dev/null
@@ -1,560 +0,0 @@
-/*
- * 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.enumerator;
-
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.ReaderInfo;
-import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext;
-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.enumerator.subscriber.KafkaSubscriber;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-import org.apache.flink.mock.Whitebox;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-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.StringJoiner;
-import java.util.regex.Pattern;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Unit tests for {@link KafkaSourceEnumerator}. */
-public class KafkaEnumeratorTest {
- private static final int NUM_SUBTASKS = 3;
- private static final String DYNAMIC_TOPIC_NAME = "dynamic_topic";
- private static final int NUM_PARTITIONS_DYNAMIC_TOPIC = 4;
-
- private static final String TOPIC1 = "topic";
- private static final String TOPIC2 = "pattern-topic";
-
- private static final int READER0 = 0;
- private static final int READER1 = 1;
- private static final Set<String> PRE_EXISTING_TOPICS =
- new HashSet<>(Arrays.asList(TOPIC1, TOPIC2));
- private static final int PARTITION_DISCOVERY_CALLABLE_INDEX = 0;
- private static final boolean ENABLE_PERIODIC_PARTITION_DISCOVERY = true;
- private static final boolean DISABLE_PERIODIC_PARTITION_DISCOVERY = false;
- private static final boolean INCLUDE_DYNAMIC_TOPIC = true;
- private static final boolean EXCLUDE_DYNAMIC_TOPIC = false;
-
- @BeforeClass
- public static void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- KafkaSourceTestEnv.setupTopic(TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.setupTopic(TOPIC2, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- KafkaSourceTestEnv.tearDown();
- }
-
- @Test
- public void testStartWithDiscoverPartitionsOnce() throws Exception {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) {
-
- // Start the enumerator and it should schedule a one time task to discover and assign
- // partitions.
- 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 testStartWithPeriodicPartitionDiscovery() throws Exception {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, ENABLE_PERIODIC_PARTITION_DISCOVERY)) {
-
- // Start the enumerator and it should schedule a one time task to discover and assign
- // partitions.
- enumerator.start();
- assertThat(context.getOneTimeCallables()).isEmpty();
- assertThat(context.getPeriodicCallables())
- .as("A periodic partition discovery callable should have been scheduled")
- .hasSize(1);
- }
- }
-
- @Test
- public void testDiscoverPartitionsTriggersAssignments() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) {
-
- // Start the enumerator and it should schedule a one time task to discover and assign
- // partitions.
- enumerator.start();
-
- // register reader 0.
- registerReader(context, enumerator, READER0);
- registerReader(context, enumerator, READER1);
- assertThat(context.getSplitsAssignmentSequence()).isEmpty();
-
- // Run the partition discover callable and check the partition assignment.
- runOneTimePartitionDiscovery(context);
-
- // Verify assignments for reader 0.
- verifyLastReadersAssignments(
- context, Arrays.asList(READER0, READER1), PRE_EXISTING_TOPICS, 1);
- }
- }
-
- @Test
- public void testReaderRegistrationTriggersAssignments() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) {
-
- // Start the enumerator and it should schedule a one time task to discover and assign
- // partitions.
- enumerator.start();
- runOneTimePartitionDiscovery(context);
- assertThat(context.getSplitsAssignmentSequence()).isEmpty();
-
- registerReader(context, enumerator, READER0);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER0), PRE_EXISTING_TOPICS, 1);
-
- registerReader(context, enumerator, READER1);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER1), PRE_EXISTING_TOPICS, 2);
- }
- }
-
- @Test(timeout = 30000L)
- public void testDiscoverPartitionsPeriodically() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(
- context,
- ENABLE_PERIODIC_PARTITION_DISCOVERY,
- INCLUDE_DYNAMIC_TOPIC);
- AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
-
- startEnumeratorAndRegisterReaders(context, enumerator);
-
- // invoke partition discovery callable again and there should be no new assignments.
- runPeriodicPartitionDiscovery(context);
- assertThat(context.getSplitsAssignmentSequence())
- .as("No assignments should be made because there is no partition change")
- .hasSize(2);
-
- // create the dynamic topic.
- adminClient
- .createTopics(
- Collections.singleton(
- new NewTopic(
- DYNAMIC_TOPIC_NAME,
- NUM_PARTITIONS_DYNAMIC_TOPIC,
- (short) 1)))
- .all()
- .get();
-
- // invoke partition discovery callable again.
- while (true) {
- runPeriodicPartitionDiscovery(context);
- if (context.getSplitsAssignmentSequence().size() < 3) {
- Thread.sleep(10);
- } else {
- break;
- }
- }
- verifyLastReadersAssignments(
- context,
- Arrays.asList(READER0, READER1),
- Collections.singleton(DYNAMIC_TOPIC_NAME),
- 3);
- } finally {
- try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
- adminClient.deleteTopics(Collections.singleton(DYNAMIC_TOPIC_NAME)).all().get();
- } catch (Exception e) {
- // Let it go.
- }
- }
- }
-
- @Test
- public void testAddSplitsBack() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, ENABLE_PERIODIC_PARTITION_DISCOVERY)) {
-
- startEnumeratorAndRegisterReaders(context, enumerator);
-
- // Simulate a reader failure.
- context.unregisterReader(READER0);
- enumerator.addSplitsBack(
- context.getSplitsAssignmentSequence().get(0).assignment().get(READER0),
- READER0);
- assertThat(context.getSplitsAssignmentSequence())
- .as("The added back splits should have not been assigned")
- .hasSize(2);
-
- // Simulate a reader recovery.
- registerReader(context, enumerator, READER0);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER0), PRE_EXISTING_TOPICS, 3);
- }
- }
-
- @Test
- public void testWorkWithPreexistingAssignments() throws Throwable {
- Set<TopicPartition> preexistingAssignments;
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context1 =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context1, ENABLE_PERIODIC_PARTITION_DISCOVERY)) {
- startEnumeratorAndRegisterReaders(context1, enumerator);
- preexistingAssignments =
- asEnumState(context1.getSplitsAssignmentSequence().get(0).assignment());
- }
-
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context2 =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(
- context2,
- ENABLE_PERIODIC_PARTITION_DISCOVERY,
- PRE_EXISTING_TOPICS,
- preexistingAssignments,
- new Properties())) {
- enumerator.start();
- runPeriodicPartitionDiscovery(context2);
-
- registerReader(context2, enumerator, READER0);
- assertThat(context2.getSplitsAssignmentSequence()).isEmpty();
-
- registerReader(context2, enumerator, READER1);
- verifyLastReadersAssignments(
- context2, Collections.singleton(READER1), PRE_EXISTING_TOPICS, 1);
- }
- }
-
- @Test
- public void testKafkaClientProperties() throws Exception {
- Properties properties = new Properties();
- String clientIdPrefix = "test-prefix";
- Integer defaultTimeoutMs = 99999;
- properties.setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), clientIdPrefix);
- properties.setProperty(
- ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(defaultTimeoutMs));
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(
- context,
- ENABLE_PERIODIC_PARTITION_DISCOVERY,
- PRE_EXISTING_TOPICS,
- Collections.emptySet(),
- properties)) {
- enumerator.start();
-
- AdminClient adminClient = Whitebox.getInternalState(enumerator, "adminClient");
- assertThat(adminClient).isNotNull();
- String clientId = Whitebox.getInternalState(adminClient, "clientId");
- assertThat(clientId).isNotNull().startsWith(clientIdPrefix);
- assertThat((Integer) Whitebox.getInternalState(adminClient, "defaultApiTimeoutMs"))
- .isEqualTo(defaultTimeoutMs);
-
- assertThat(clientId).isNotNull().startsWith(clientIdPrefix);
- }
- }
-
- @Test
- public void testSnapshotState() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator = createEnumerator(context, false)) {
- enumerator.start();
-
- // No reader is registered, so the state should be empty
- final KafkaSourceEnumState state1 = enumerator.snapshotState(1L);
- assertThat(state1.assignedPartitions()).isEmpty();
-
- registerReader(context, enumerator, READER0);
- registerReader(context, enumerator, READER1);
- runOneTimePartitionDiscovery(context);
-
- // The state should contain splits assigned to READER0 and READER1
- final KafkaSourceEnumState state2 = enumerator.snapshotState(1L);
- verifySplitAssignmentWithPartitions(
- getExpectedAssignments(
- new HashSet<>(Arrays.asList(READER0, READER1)), PRE_EXISTING_TOPICS),
- state2.assignedPartitions());
- }
- }
-
- @Test
- public void testPartitionChangeChecking() throws Throwable {
- try (MockSplitEnumeratorContext<KafkaPartitionSplit> context =
- new MockSplitEnumeratorContext<>(NUM_SUBTASKS);
- KafkaSourceEnumerator enumerator =
- createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) {
- enumerator.start();
- runOneTimePartitionDiscovery(context);
- registerReader(context, enumerator, READER0);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER0), PRE_EXISTING_TOPICS, 1);
-
- // All partitions of TOPIC1 and TOPIC2 should have been discovered now
-
- // Check partition change using only DYNAMIC_TOPIC_NAME-0
- TopicPartition newPartition = new TopicPartition(DYNAMIC_TOPIC_NAME, 0);
- Set<TopicPartition> fetchedPartitions = new HashSet<>();
- fetchedPartitions.add(newPartition);
- final KafkaSourceEnumerator.PartitionChange partitionChange =
- enumerator.getPartitionChange(fetchedPartitions);
-
- // Since enumerator never met DYNAMIC_TOPIC_NAME-0, it should be mark as a new partition
- Set<TopicPartition> expectedNewPartitions = Collections.singleton(newPartition);
-
- // All existing topics are not in the fetchedPartitions, so they should be marked as
- // removed
- Set<TopicPartition> expectedRemovedPartitions = new HashSet<>();
- for (int i = 0; i < KafkaSourceTestEnv.NUM_PARTITIONS; i++) {
- expectedRemovedPartitions.add(new TopicPartition(TOPIC1, i));
- expectedRemovedPartitions.add(new TopicPartition(TOPIC2, i));
- }
-
- assertThat(partitionChange.getNewPartitions()).isEqualTo(expectedNewPartitions);
- assertThat(partitionChange.getRemovedPartitions()).isEqualTo(expectedRemovedPartitions);
- }
- }
-
- // -------------- some common startup sequence ---------------
-
- private void startEnumeratorAndRegisterReaders(
- MockSplitEnumeratorContext<KafkaPartitionSplit> context,
- KafkaSourceEnumerator enumerator)
- throws Throwable {
- // Start the enumerator and it should schedule a one time task to discover and assign
- // partitions.
- enumerator.start();
-
- // register reader 0 before the partition discovery.
- registerReader(context, enumerator, READER0);
- assertThat(context.getSplitsAssignmentSequence()).isEmpty();
-
- // Run the partition discover callable and check the partition assignment.
- runPeriodicPartitionDiscovery(context);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER0), PRE_EXISTING_TOPICS, 1);
-
- // Register reader 1 after first partition discovery.
- registerReader(context, enumerator, READER1);
- verifyLastReadersAssignments(
- context, Collections.singleton(READER1), PRE_EXISTING_TOPICS, 2);
- }
-
- // ----------------------------------------
-
- private KafkaSourceEnumerator createEnumerator(
- MockSplitEnumeratorContext<KafkaPartitionSplit> enumContext,
- boolean enablePeriodicPartitionDiscovery) {
- return createEnumerator(
- enumContext, enablePeriodicPartitionDiscovery, EXCLUDE_DYNAMIC_TOPIC);
- }
-
- private KafkaSourceEnumerator createEnumerator(
- MockSplitEnumeratorContext<KafkaPartitionSplit> enumContext,
- boolean enablePeriodicPartitionDiscovery,
- boolean includeDynamicTopic) {
- List<String> topics = new ArrayList<>(PRE_EXISTING_TOPICS);
- if (includeDynamicTopic) {
- topics.add(DYNAMIC_TOPIC_NAME);
- }
- return createEnumerator(
- enumContext,
- enablePeriodicPartitionDiscovery,
- topics,
- Collections.emptySet(),
- new Properties());
- }
-
- /**
- * Create the enumerator. For the purpose of the tests in this class we don't care about the
- * subscriber and offsets initializer, so just use arbitrary settings.
- */
- private KafkaSourceEnumerator createEnumerator(
- MockSplitEnumeratorContext<KafkaPartitionSplit> enumContext,
- boolean enablePeriodicPartitionDiscovery,
- Collection<String> topicsToSubscribe,
- Set<TopicPartition> assignedPartitions,
- Properties overrideProperties) {
- // Use a TopicPatternSubscriber so that no exception if a subscribed topic hasn't been
- // created yet.
- StringJoiner topicNameJoiner = new StringJoiner("|");
- topicsToSubscribe.forEach(topicNameJoiner::add);
- Pattern topicPattern = Pattern.compile(topicNameJoiner.toString());
- KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern);
-
- OffsetsInitializer startingOffsetsInitializer = OffsetsInitializer.earliest();
- OffsetsInitializer stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer();
-
- Properties props =
- new Properties(KafkaSourceTestEnv.getConsumerProperties(StringDeserializer.class));
- KafkaSourceEnumerator.deepCopyProperties(overrideProperties, props);
- String partitionDiscoverInterval = enablePeriodicPartitionDiscovery ? "1" : "-1";
- props.setProperty(
- KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(),
- partitionDiscoverInterval);
-
- return new KafkaSourceEnumerator(
- subscriber,
- startingOffsetsInitializer,
- stoppingOffsetsInitializer,
- props,
- enumContext,
- Boundedness.CONTINUOUS_UNBOUNDED,
- assignedPartitions);
- }
-
- // ---------------------
-
- private void registerReader(
- MockSplitEnumeratorContext<KafkaPartitionSplit> context,
- KafkaSourceEnumerator enumerator,
- int reader) {
- context.registerReader(new ReaderInfo(reader, "location 0"));
- enumerator.addReader(reader);
- }
-
- private void verifyLastReadersAssignments(
- MockSplitEnumeratorContext<KafkaPartitionSplit> context,
- Collection<Integer> readers,
- Set<String> topics,
- int expectedAssignmentSeqSize) {
- verifyAssignments(
- getExpectedAssignments(new HashSet<>(readers), topics),
- context.getSplitsAssignmentSequence()
- .get(expectedAssignmentSeqSize - 1)
- .assignment());
- }
-
- private void verifyAssignments(
- Map<Integer, Set<TopicPartition>> expectedAssignments,
- Map<Integer, List<KafkaPartitionSplit>> actualAssignments) {
- actualAssignments.forEach(
- (reader, splits) -> {
- Set<TopicPartition> expectedAssignmentsForReader =
- expectedAssignments.get(reader);
- assertThat(expectedAssignmentsForReader).isNotNull();
- assertThat(splits.size()).isEqualTo(expectedAssignmentsForReader.size());
- for (KafkaPartitionSplit split : splits) {
- assertThat(expectedAssignmentsForReader)
- .contains(split.getTopicPartition());
- }
- });
- }
-
- private Map<Integer, Set<TopicPartition>> getExpectedAssignments(
- Set<Integer> readers, Set<String> topics) {
- Map<Integer, Set<TopicPartition>> expectedAssignments = new HashMap<>();
- Set<TopicPartition> allPartitions = new HashSet<>();
-
- if (topics.contains(DYNAMIC_TOPIC_NAME)) {
- for (int i = 0; i < NUM_PARTITIONS_DYNAMIC_TOPIC; i++) {
- allPartitions.add(new TopicPartition(DYNAMIC_TOPIC_NAME, i));
- }
- }
-
- for (TopicPartition tp : KafkaSourceTestEnv.getPartitionsForTopics(PRE_EXISTING_TOPICS)) {
- if (topics.contains(tp.topic())) {
- allPartitions.add(tp);
- }
- }
-
- for (TopicPartition tp : allPartitions) {
- int ownerReader = KafkaSourceEnumerator.getSplitOwner(tp, NUM_SUBTASKS);
- if (readers.contains(ownerReader)) {
- expectedAssignments.computeIfAbsent(ownerReader, r -> new HashSet<>()).add(tp);
- }
- }
- return expectedAssignments;
- }
-
- private void verifySplitAssignmentWithPartitions(
- Map<Integer, Set<TopicPartition>> expectedAssignment,
- Set<TopicPartition> actualTopicPartitions) {
- final Set<TopicPartition> allTopicPartitionsFromAssignment = new HashSet<>();
- expectedAssignment.forEach(
- (reader, topicPartitions) ->
- allTopicPartitionsFromAssignment.addAll(topicPartitions));
- assertThat(actualTopicPartitions).isEqualTo(allTopicPartitionsFromAssignment);
- }
-
- private Set<TopicPartition> asEnumState(Map<Integer, List<KafkaPartitionSplit>> assignments) {
- Set<TopicPartition> enumState = new HashSet<>();
- assignments.forEach(
- (reader, assignment) ->
- assignment.forEach(split -> enumState.add(split.getTopicPartition())));
- return enumState;
- }
-
- private void runOneTimePartitionDiscovery(
- MockSplitEnumeratorContext<KafkaPartitionSplit> context) throws Throwable {
- // Fetch potential topic descriptions
- context.runNextOneTimeCallable();
- // Initialize offsets for discovered partitions
- if (!context.getOneTimeCallables().isEmpty()) {
- context.runNextOneTimeCallable();
- }
- }
-
- private void runPeriodicPartitionDiscovery(
- MockSplitEnumeratorContext<KafkaPartitionSplit> context) throws Throwable {
- // Fetch potential topic descriptions
- context.runPeriodicCallable(PARTITION_DISCOVERY_CALLABLE_INDEX);
- // Initialize offsets for discovered partitions
- if (!context.getOneTimeCallables().isEmpty()) {
- context.runNextOneTimeCallable();
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java
deleted file mode 100644
index 058d2a0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.enumerator;
-
-import org.apache.flink.connector.base.source.utils.SerdeUtils;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer;
-
-import org.apache.kafka.common.TopicPartition;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link KafkaSourceEnumStateSerializer}. */
-public class KafkaSourceEnumStateSerializerTest {
-
- private static final int NUM_READERS = 10;
- private static final String TOPIC_PREFIX = "topic-";
- private static final int NUM_PARTITIONS_PER_TOPIC = 10;
- private static final long STARTING_OFFSET = KafkaPartitionSplit.EARLIEST_OFFSET;
-
- @Test
- public void testEnumStateSerde() throws IOException {
- final KafkaSourceEnumState state = new KafkaSourceEnumState(constructTopicPartitions());
- final KafkaSourceEnumStateSerializer serializer = new KafkaSourceEnumStateSerializer();
-
- final byte[] bytes = serializer.serialize(state);
-
- final KafkaSourceEnumState restoredState =
- serializer.deserialize(serializer.getVersion(), bytes);
-
- assertThat(restoredState.assignedPartitions()).isEqualTo(state.assignedPartitions());
- }
-
- @Test
- public void testBackwardCompatibility() throws IOException {
-
- final Set<TopicPartition> topicPartitions = constructTopicPartitions();
- final Map<Integer, Set<KafkaPartitionSplit>> splitAssignments =
- toSplitAssignments(topicPartitions);
-
- // Create bytes in the way of KafkaEnumStateSerializer version 0 doing serialization
- final byte[] bytes =
- SerdeUtils.serializeSplitAssignments(
- splitAssignments, new KafkaPartitionSplitSerializer());
-
- // Deserialize above bytes with KafkaEnumStateSerializer version 1 to check backward
- // compatibility
- final KafkaSourceEnumState kafkaSourceEnumState =
- new KafkaSourceEnumStateSerializer().deserialize(0, bytes);
-
- assertThat(kafkaSourceEnumState.assignedPartitions()).isEqualTo(topicPartitions);
- }
-
- private Set<TopicPartition> constructTopicPartitions() {
- // Create topic partitions for readers.
- // Reader i will be assigned with NUM_PARTITIONS_PER_TOPIC splits, with topic name
- // "topic-{i}" and
- // NUM_PARTITIONS_PER_TOPIC partitions.
- // Totally NUM_READERS * NUM_PARTITIONS_PER_TOPIC partitions will be created.
- Set<TopicPartition> topicPartitions = new HashSet<>();
- for (int readerId = 0; readerId < NUM_READERS; readerId++) {
- for (int partition = 0; partition < NUM_PARTITIONS_PER_TOPIC; partition++) {
- topicPartitions.add(new TopicPartition(TOPIC_PREFIX + readerId, partition));
- }
- }
- return topicPartitions;
- }
-
- private Map<Integer, Set<KafkaPartitionSplit>> toSplitAssignments(
- Collection<TopicPartition> topicPartitions) {
- // Assign splits to readers according to topic name. For example, topic "topic-5" will be
- // assigned to reader with ID=5
- Map<Integer, Set<KafkaPartitionSplit>> splitAssignments = new HashMap<>();
- topicPartitions.forEach(
- (tp) ->
- splitAssignments
- .computeIfAbsent(
- Integer.valueOf(
- tp.topic().substring(TOPIC_PREFIX.length())),
- HashSet::new)
- .add(new KafkaPartitionSplit(tp, STARTING_OFFSET)));
- return splitAssignments;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java
deleted file mode 100644
index e0cd850..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.enumerator.initializer;
-
-import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.TopicPartition;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Unit tests for {@link OffsetsInitializer}. */
-public class OffsetsInitializerTest {
- private static final String TOPIC = "topic";
- private static final String TOPIC2 = "topic2";
- private static final String EMPTY_TOPIC3 = "topic3";
- private static KafkaSourceEnumerator.PartitionOffsetsRetrieverImpl retriever;
-
- @BeforeClass
- public static void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- KafkaSourceTestEnv.setupTopic(TOPIC, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.setupTopic(TOPIC2, false, false, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.createTestTopic(EMPTY_TOPIC3);
-
- retriever =
- new KafkaSourceEnumerator.PartitionOffsetsRetrieverImpl(
- KafkaSourceTestEnv.getAdminClient(), KafkaSourceTestEnv.GROUP_ID);
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- retriever.close();
- KafkaSourceTestEnv.tearDown();
- }
-
- @Test
- public void testEarliestOffsetsInitializer() {
- OffsetsInitializer initializer = OffsetsInitializer.earliest();
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC);
- Map<TopicPartition, Long> offsets = initializer.getPartitionOffsets(partitions, retriever);
- assertThat(offsets).hasSameSizeAs(partitions);
- assertThat(offsets.keySet()).containsAll(partitions);
- for (long offset : offsets.values()) {
- assertThat(offset).isEqualTo(KafkaPartitionSplit.EARLIEST_OFFSET);
- }
- assertThat(initializer.getAutoOffsetResetStrategy())
- .isEqualTo(OffsetResetStrategy.EARLIEST);
- }
-
- @Test
- public void testLatestOffsetsInitializer() {
- OffsetsInitializer initializer = OffsetsInitializer.latest();
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC);
- Map<TopicPartition, Long> offsets = initializer.getPartitionOffsets(partitions, retriever);
- assertThat(offsets).hasSameSizeAs(partitions);
- assertThat(offsets.keySet()).containsAll(partitions);
- for (long offset : offsets.values()) {
- assertThat(offset).isEqualTo(KafkaPartitionSplit.LATEST_OFFSET);
- }
- assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.LATEST);
- }
-
- @Test
- public void testCommittedGroupOffsetsInitializer() {
- OffsetsInitializer initializer = OffsetsInitializer.committedOffsets();
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC);
- Map<TopicPartition, Long> offsets = initializer.getPartitionOffsets(partitions, retriever);
- assertThat(offsets).hasSameSizeAs(partitions);
- offsets.forEach(
- (tp, offset) ->
- assertThat((long) offset).isEqualTo(KafkaPartitionSplit.COMMITTED_OFFSET));
- assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.NONE);
- }
-
- @Test
- public void testTimestampOffsetsInitializer() {
- OffsetsInitializer initializer = OffsetsInitializer.timestamp(2001);
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC);
- Map<TopicPartition, Long> offsets = initializer.getPartitionOffsets(partitions, retriever);
- offsets.forEach(
- (tp, offset) -> {
- long expectedOffset = tp.partition() > 2 ? tp.partition() : 3L;
- assertThat((long) offset).isEqualTo(expectedOffset);
- });
- assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.LATEST);
- }
-
- @Test
- public void testTimestampOffsetsInitializerForEmptyPartitions() {
- OffsetsInitializer initializer = OffsetsInitializer.timestamp(2001);
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(EMPTY_TOPIC3);
- Map<TopicPartition, Long> expectedOffsets =
- partitions.stream().collect(Collectors.toMap(tp -> tp, tp -> 0L));
- assertThat(initializer.getPartitionOffsets(partitions, retriever))
- .as("offsets are equal to 0 since the timestamp is out of range.")
- .isEqualTo(expectedOffsets);
- assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.LATEST);
- }
-
- @Test
- public void testSpecificOffsetsInitializer() {
- Map<TopicPartition, Long> specifiedOffsets = new HashMap<>();
- List<TopicPartition> partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC);
- Map<TopicPartition, OffsetAndMetadata> committedOffsets =
- KafkaSourceTestEnv.getCommittedOffsets(partitions);
- partitions.forEach(tp -> specifiedOffsets.put(tp, (long) tp.partition()));
- // Remove the specified offsets for partition 0.
- TopicPartition partitionSetToCommitted = new TopicPartition(TOPIC, 0);
- specifiedOffsets.remove(partitionSetToCommitted);
- OffsetsInitializer initializer = OffsetsInitializer.offsets(specifiedOffsets);
-
- assertThat(initializer.getAutoOffsetResetStrategy())
- .isEqualTo(OffsetResetStrategy.EARLIEST);
- // The partition without committed offset should fallback to offset reset strategy.
- TopicPartition partitionSetToEarliest = new TopicPartition(TOPIC2, 0);
- partitions.add(partitionSetToEarliest);
-
- Map<TopicPartition, Long> offsets = initializer.getPartitionOffsets(partitions, retriever);
- for (TopicPartition tp : partitions) {
- Long offset = offsets.get(tp);
- long expectedOffset;
- if (tp.equals(partitionSetToCommitted)) {
- expectedOffset = committedOffsets.get(tp).offset();
- } else if (tp.equals(partitionSetToEarliest)) {
- expectedOffset = 0L;
- } else {
- expectedOffset = specifiedOffsets.get(tp);
- }
- assertThat((long) offset)
- .as(String.format("%s has incorrect offset.", tp))
- .isEqualTo(expectedOffset);
- }
- }
-
- @Test(expected = IllegalStateException.class)
- public void testSpecifiedOffsetsInitializerWithoutOffsetResetStrategy() {
- OffsetsInitializer initializer =
- OffsetsInitializer.offsets(Collections.emptyMap(), OffsetResetStrategy.NONE);
- initializer.getPartitionOffsets(KafkaSourceTestEnv.getPartitionsForTopic(TOPIC), retriever);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java
deleted file mode 100644
index 258c1c0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.enumerator.subscriber;
-
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Unit tests for {@link KafkaSubscriber}. */
-public class KafkaSubscriberTest {
- private static final String TOPIC1 = "topic1";
- private static final String TOPIC2 = "pattern-topic";
- private static final TopicPartition NON_EXISTING_TOPIC = new TopicPartition("removed", 0);
- private static AdminClient adminClient;
-
- @BeforeClass
- public static void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- KafkaSourceTestEnv.createTestTopic(TOPIC1);
- KafkaSourceTestEnv.createTestTopic(TOPIC2);
- adminClient = KafkaSourceTestEnv.getAdminClient();
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- adminClient.close();
- KafkaSourceTestEnv.tearDown();
- }
-
- @Test
- public void testTopicListSubscriber() {
- List<String> topics = Arrays.asList(TOPIC1, TOPIC2);
- KafkaSubscriber subscriber =
- KafkaSubscriber.getTopicListSubscriber(Arrays.asList(TOPIC1, TOPIC2));
- final Set<TopicPartition> subscribedPartitions =
- subscriber.getSubscribedTopicPartitions(adminClient);
-
- final Set<TopicPartition> expectedSubscribedPartitions =
- new HashSet<>(KafkaSourceTestEnv.getPartitionsForTopics(topics));
-
- assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions);
- }
-
- @Test
- public void testNonExistingTopic() {
- final KafkaSubscriber subscriber =
- KafkaSubscriber.getTopicListSubscriber(
- Collections.singletonList(NON_EXISTING_TOPIC.topic()));
-
- assertThatThrownBy(() -> subscriber.getSubscribedTopicPartitions(adminClient))
- .isInstanceOf(RuntimeException.class)
- .satisfies(anyCauseMatches(UnknownTopicOrPartitionException.class));
- }
-
- @Test
- public void testTopicPatternSubscriber() {
- KafkaSubscriber subscriber =
- KafkaSubscriber.getTopicPatternSubscriber(Pattern.compile("pattern.*"));
- final Set<TopicPartition> subscribedPartitions =
- subscriber.getSubscribedTopicPartitions(adminClient);
-
- final Set<TopicPartition> expectedSubscribedPartitions =
- new HashSet<>(
- KafkaSourceTestEnv.getPartitionsForTopics(Collections.singleton(TOPIC2)));
-
- assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions);
- }
-
- @Test
- public void testPartitionSetSubscriber() {
- List<String> topics = Arrays.asList(TOPIC1, TOPIC2);
- Set<TopicPartition> partitions =
- new HashSet<>(KafkaSourceTestEnv.getPartitionsForTopics(topics));
- partitions.remove(new TopicPartition(TOPIC1, 1));
-
- KafkaSubscriber subscriber = KafkaSubscriber.getPartitionSetSubscriber(partitions);
-
- final Set<TopicPartition> subscribedPartitions =
- subscriber.getSubscribedTopicPartitions(adminClient);
-
- assertThat(subscribedPartitions).isEqualTo(partitions);
- }
-
- @Test
- public void testNonExistingPartition() {
- TopicPartition nonExistingPartition = new TopicPartition(TOPIC1, Integer.MAX_VALUE);
- final KafkaSubscriber subscriber =
- KafkaSubscriber.getPartitionSetSubscriber(
- Collections.singleton(nonExistingPartition));
-
- assertThatThrownBy(() -> subscriber.getSubscribedTopicPartitions(adminClient))
- .isInstanceOf(RuntimeException.class)
- .hasMessage(
- String.format(
- "Partition '%s' does not exist on Kafka brokers",
- nonExistingPartition));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java
deleted file mode 100644
index d9dd1d3..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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.metrics;
-
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.testutils.MetricListener;
-import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup;
-
-import org.apache.kafka.common.TopicPartition;
-import org.junit.Test;
-
-import java.util.Optional;
-
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.PARTITION_GROUP;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.TOPIC_GROUP;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Unit test for {@link KafkaSourceReaderMetrics}. */
-public class KafkaSourceReaderMetricsTest {
-
- private static final TopicPartition FOO_0 = new TopicPartition("foo", 0);
- private static final TopicPartition FOO_1 = new TopicPartition("foo", 1);
- private static final TopicPartition BAR_0 = new TopicPartition("bar", 0);
- private static final TopicPartition BAR_1 = new TopicPartition("bar", 1);
-
- @Test
- public void testCurrentOffsetTracking() {
- MetricListener metricListener = new MetricListener();
-
- final KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(
- InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()));
-
- kafkaSourceReaderMetrics.registerTopicPartition(FOO_0);
- kafkaSourceReaderMetrics.registerTopicPartition(FOO_1);
- kafkaSourceReaderMetrics.registerTopicPartition(BAR_0);
- kafkaSourceReaderMetrics.registerTopicPartition(BAR_1);
-
- kafkaSourceReaderMetrics.recordCurrentOffset(FOO_0, 15213L);
- kafkaSourceReaderMetrics.recordCurrentOffset(FOO_1, 18213L);
- kafkaSourceReaderMetrics.recordCurrentOffset(BAR_0, 18613L);
- kafkaSourceReaderMetrics.recordCurrentOffset(BAR_1, 15513L);
-
- assertCurrentOffset(FOO_0, 15213L, metricListener);
- assertCurrentOffset(FOO_1, 18213L, metricListener);
- assertCurrentOffset(BAR_0, 18613L, metricListener);
- assertCurrentOffset(BAR_1, 15513L, metricListener);
- }
-
- @Test
- public void testCommitOffsetTracking() {
- MetricListener metricListener = new MetricListener();
-
- final KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(
- InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()));
-
- kafkaSourceReaderMetrics.registerTopicPartition(FOO_0);
- kafkaSourceReaderMetrics.registerTopicPartition(FOO_1);
- kafkaSourceReaderMetrics.registerTopicPartition(BAR_0);
- kafkaSourceReaderMetrics.registerTopicPartition(BAR_1);
-
- kafkaSourceReaderMetrics.recordCommittedOffset(FOO_0, 15213L);
- kafkaSourceReaderMetrics.recordCommittedOffset(FOO_1, 18213L);
- kafkaSourceReaderMetrics.recordCommittedOffset(BAR_0, 18613L);
- kafkaSourceReaderMetrics.recordCommittedOffset(BAR_1, 15513L);
-
- assertCommittedOffset(FOO_0, 15213L, metricListener);
- assertCommittedOffset(FOO_1, 18213L, metricListener);
- assertCommittedOffset(BAR_0, 18613L, metricListener);
- assertCommittedOffset(BAR_1, 15513L, metricListener);
-
- final Optional<Counter> commitsSucceededCounter =
- metricListener.getCounter(
- KafkaSourceReaderMetrics.KAFKA_SOURCE_READER_METRIC_GROUP,
- KafkaSourceReaderMetrics.COMMITS_SUCCEEDED_METRIC_COUNTER);
- assertThat(commitsSucceededCounter).isPresent();
- assertThat(commitsSucceededCounter.get().getCount()).isEqualTo(0L);
-
- kafkaSourceReaderMetrics.recordSucceededCommit();
-
- assertThat(commitsSucceededCounter.get().getCount()).isEqualTo(1L);
- }
-
- @Test
- public void testNonTrackingTopicPartition() {
- MetricListener metricListener = new MetricListener();
- final KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(
- InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()));
- assertThatThrownBy(() -> kafkaSourceReaderMetrics.recordCurrentOffset(FOO_0, 15213L))
- .isInstanceOf(IllegalArgumentException.class);
- assertThatThrownBy(() -> kafkaSourceReaderMetrics.recordCommittedOffset(FOO_0, 15213L))
- .isInstanceOf(IllegalArgumentException.class);
- }
-
- @Test
- public void testFailedCommit() {
- MetricListener metricListener = new MetricListener();
- final KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(
- InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()));
- kafkaSourceReaderMetrics.recordFailedCommit();
- final Optional<Counter> commitsFailedCounter =
- metricListener.getCounter(
- KafkaSourceReaderMetrics.KAFKA_SOURCE_READER_METRIC_GROUP,
- KafkaSourceReaderMetrics.COMMITS_FAILED_METRIC_COUNTER);
- assertThat(commitsFailedCounter).isPresent();
- assertThat(commitsFailedCounter.get().getCount()).isEqualTo(1L);
- }
-
- // ----------- Assertions --------------
-
- private void assertCurrentOffset(
- TopicPartition tp, long expectedOffset, MetricListener metricListener) {
- final Optional<Gauge<Long>> currentOffsetGauge =
- metricListener.getGauge(
- KafkaSourceReaderMetrics.KAFKA_SOURCE_READER_METRIC_GROUP,
- TOPIC_GROUP,
- tp.topic(),
- PARTITION_GROUP,
- String.valueOf(tp.partition()),
- KafkaSourceReaderMetrics.CURRENT_OFFSET_METRIC_GAUGE);
- assertThat(currentOffsetGauge).isPresent();
- assertThat((long) currentOffsetGauge.get().getValue()).isEqualTo(expectedOffset);
- }
-
- private void assertCommittedOffset(
- TopicPartition tp, long expectedOffset, MetricListener metricListener) {
- final Optional<Gauge<Long>> committedOffsetGauge =
- metricListener.getGauge(
- KafkaSourceReaderMetrics.KAFKA_SOURCE_READER_METRIC_GROUP,
- TOPIC_GROUP,
- tp.topic(),
- PARTITION_GROUP,
- String.valueOf(tp.partition()),
- KafkaSourceReaderMetrics.COMMITTED_OFFSET_METRIC_GAUGE);
- assertThat(committedOffsetGauge).isPresent();
- assertThat((long) committedOffsetGauge.get().getValue()).isEqualTo(expectedOffset);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java
deleted file mode 100644
index 7263bd0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/*
- * 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.reader;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
-import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-import org.apache.flink.connector.testutils.source.reader.TestingReaderContext;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.groups.OperatorMetricGroup;
-import org.apache.flink.metrics.groups.SourceReaderMetricGroup;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.metrics.testutils.MetricListener;
-import org.apache.flink.runtime.metrics.MetricNames;
-import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup;
-import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.KafkaException;
-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.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.CsvSource;
-import org.junit.jupiter.params.provider.EmptySource;
-import org.junit.jupiter.params.provider.ValueSource;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Unit tests for {@link KafkaPartitionSplitReader}. */
-public class KafkaPartitionSplitReaderTest {
- private static final int NUM_SUBTASKS = 3;
- private static final String TOPIC1 = "topic1";
- private static final String TOPIC2 = "topic2";
- private static final String TOPIC3 = "topic3";
-
- private static Map<Integer, Map<String, KafkaPartitionSplit>> splitsByOwners;
- private static Map<TopicPartition, Long> earliestOffsets;
-
- private final IntegerDeserializer deserializer = new IntegerDeserializer();
-
- @BeforeAll
- public static void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- KafkaSourceTestEnv.setupTopic(TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.setupTopic(TOPIC2, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.createTestTopic(TOPIC3);
- splitsByOwners =
- KafkaSourceTestEnv.getSplitsByOwners(Arrays.asList(TOPIC1, TOPIC2), NUM_SUBTASKS);
- earliestOffsets =
- KafkaSourceTestEnv.getEarliestOffsets(
- KafkaSourceTestEnv.getPartitionsForTopics(Arrays.asList(TOPIC1, TOPIC2)));
- }
-
- @AfterAll
- public static void tearDown() throws Exception {
- KafkaSourceTestEnv.tearDown();
- }
-
- @Test
- public void testHandleSplitChangesAndFetch() throws Exception {
- KafkaPartitionSplitReader reader = createReader();
- assignSplitsAndFetchUntilFinish(reader, 0);
- assignSplitsAndFetchUntilFinish(reader, 1);
- }
-
- @Test
- public void testWakeUp() throws Exception {
- KafkaPartitionSplitReader reader = createReader();
- TopicPartition nonExistingTopicPartition = new TopicPartition("NotExist", 0);
- assignSplits(
- reader,
- Collections.singletonMap(
- KafkaPartitionSplit.toSplitId(nonExistingTopicPartition),
- new KafkaPartitionSplit(nonExistingTopicPartition, 0)));
- AtomicReference<Throwable> error = new AtomicReference<>();
- Thread t =
- new Thread(
- () -> {
- try {
- reader.fetch();
- } catch (Throwable e) {
- error.set(e);
- }
- },
- "testWakeUp-thread");
- t.start();
- long deadline = System.currentTimeMillis() + 5000L;
- while (t.isAlive() && System.currentTimeMillis() < deadline) {
- reader.wakeUp();
- Thread.sleep(10);
- }
- assertThat(error.get()).isNull();
- }
-
- @Test
- public void testWakeupThenAssign() throws IOException {
- KafkaPartitionSplitReader reader = createReader();
- // Assign splits with records
- assignSplits(reader, splitsByOwners.get(0));
- // Run a fetch operation, and it should not block
- reader.fetch();
- // Wake the reader up then assign a new split. This assignment should not throw
- // WakeupException.
- reader.wakeUp();
- TopicPartition tp = new TopicPartition(TOPIC1, 0);
- assignSplits(
- reader,
- Collections.singletonMap(
- KafkaPartitionSplit.toSplitId(tp),
- new KafkaPartitionSplit(tp, KafkaPartitionSplit.EARLIEST_OFFSET)));
- }
-
- @Test
- public void testNumBytesInCounter() throws Exception {
- final OperatorMetricGroup operatorMetricGroup =
- UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup();
- final Counter numBytesInCounter =
- operatorMetricGroup.getIOMetricGroup().getNumBytesInCounter();
- KafkaPartitionSplitReader reader =
- createReader(
- new Properties(),
- InternalSourceReaderMetricGroup.wrap(operatorMetricGroup));
- // Add a split
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(new TopicPartition(TOPIC1, 0), 0L))));
- reader.fetch();
- final long latestNumBytesIn = numBytesInCounter.getCount();
- // Since it's hard to know the exact number of bytes consumed, we just check if it is
- // greater than 0
- assertThat(latestNumBytesIn).isGreaterThan(0L);
- // Add another split
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(new TopicPartition(TOPIC2, 0), 0L))));
- reader.fetch();
- // We just check if numBytesIn is increasing
- assertThat(numBytesInCounter.getCount()).isGreaterThan(latestNumBytesIn);
- }
-
- @ParameterizedTest
- @EmptySource
- @ValueSource(strings = {"_underscore.period-minus"})
- public void testPendingRecordsGauge(String topicSuffix) throws Throwable {
- final String topic1Name = TOPIC1 + topicSuffix;
- final String topic2Name = TOPIC2 + topicSuffix;
- if (!topicSuffix.isEmpty()) {
- KafkaSourceTestEnv.setupTopic(
- topic1Name, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- KafkaSourceTestEnv.setupTopic(
- topic2Name, true, true, KafkaSourceTestEnv::getRecordsForTopic);
- }
- MetricListener metricListener = new MetricListener();
- final Properties props = new Properties();
- props.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1");
- KafkaPartitionSplitReader reader =
- createReader(
- props,
- InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()));
- // Add a split
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(new TopicPartition(topic1Name, 0), 0L))));
- // pendingRecords should have not been registered because of lazily registration
- assertThat(metricListener.getGauge(MetricNames.PENDING_RECORDS)).isNotPresent();
- // Trigger first fetch
- reader.fetch();
- final Optional<Gauge<Long>> pendingRecords =
- metricListener.getGauge(MetricNames.PENDING_RECORDS);
- assertThat(pendingRecords).isPresent();
- // Validate pendingRecords
- assertThat(pendingRecords).isNotNull();
- assertThat((long) pendingRecords.get().getValue()).isEqualTo(NUM_RECORDS_PER_PARTITION - 1);
- for (int i = 1; i < NUM_RECORDS_PER_PARTITION; i++) {
- reader.fetch();
- assertThat((long) pendingRecords.get().getValue())
- .isEqualTo(NUM_RECORDS_PER_PARTITION - i - 1);
- }
- // Add another split
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(new TopicPartition(topic2Name, 0), 0L))));
- // Validate pendingRecords
- for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) {
- reader.fetch();
- assertThat((long) pendingRecords.get().getValue())
- .isEqualTo(NUM_RECORDS_PER_PARTITION - i - 1);
- }
- }
-
- @Test
- public void testAssignEmptySplit() throws Exception {
- KafkaPartitionSplitReader reader = createReader();
- final KafkaPartitionSplit normalSplit =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC1, 0),
- KafkaPartitionSplit.EARLIEST_OFFSET,
- KafkaPartitionSplit.NO_STOPPING_OFFSET);
- final KafkaPartitionSplit emptySplit =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC2, 0),
- KafkaPartitionSplit.LATEST_OFFSET,
- KafkaPartitionSplit.LATEST_OFFSET);
- final KafkaPartitionSplit emptySplitWithZeroStoppingOffset =
- new KafkaPartitionSplit(new TopicPartition(TOPIC3, 0), 0, 0);
-
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Arrays.asList(normalSplit, emptySplit, emptySplitWithZeroStoppingOffset)));
-
- // Fetch and check empty splits is added to finished splits
- RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> recordsWithSplitIds = reader.fetch();
- assertThat(recordsWithSplitIds.finishedSplits()).contains(emptySplit.splitId());
- assertThat(recordsWithSplitIds.finishedSplits())
- .contains(emptySplitWithZeroStoppingOffset.splitId());
-
- // Assign another valid split to avoid consumer.poll() blocking
- final KafkaPartitionSplit anotherNormalSplit =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC1, 1),
- KafkaPartitionSplit.EARLIEST_OFFSET,
- KafkaPartitionSplit.NO_STOPPING_OFFSET);
- reader.handleSplitsChanges(
- new SplitsAddition<>(Collections.singletonList(anotherNormalSplit)));
-
- // Fetch again and check empty split set is cleared
- recordsWithSplitIds = reader.fetch();
- assertThat(recordsWithSplitIds.finishedSplits()).isEmpty();
- }
-
- @Test
- public void testUsingCommittedOffsetsWithNoneOffsetResetStrategy() {
- final Properties props = new Properties();
- props.setProperty(
- ConsumerConfig.GROUP_ID_CONFIG, "using-committed-offset-with-none-offset-reset");
- KafkaPartitionSplitReader reader =
- createReader(props, UnregisteredMetricsGroup.createSourceReaderMetricGroup());
- // We expect that there is a committed offset, but the group does not actually have a
- // committed offset, and the offset reset strategy is none (Throw exception to the consumer
- // if no previous offset is found for the consumer's group);
- // So it is expected to throw an exception that missing the committed offset.
- assertThatThrownBy(
- () ->
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC1, 0),
- KafkaPartitionSplit
- .COMMITTED_OFFSET)))))
- .isInstanceOf(KafkaException.class)
- .hasMessageContaining("Undefined offset with no reset policy for partition");
- }
-
- @ParameterizedTest
- @CsvSource({"earliest, 0", "latest, 10"})
- public void testUsingCommittedOffsetsWithEarliestOrLatestOffsetResetStrategy(
- String offsetResetStrategy, Long expectedOffset) {
- final Properties props = new Properties();
- props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, offsetResetStrategy);
- props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "using-committed-offset");
- KafkaPartitionSplitReader reader =
- createReader(props, UnregisteredMetricsGroup.createSourceReaderMetricGroup());
- // Add committed offset split
- final TopicPartition partition = new TopicPartition(TOPIC1, 0);
- reader.handleSplitsChanges(
- new SplitsAddition<>(
- Collections.singletonList(
- new KafkaPartitionSplit(
- partition, KafkaPartitionSplit.COMMITTED_OFFSET))));
-
- // Verify that the current offset of the consumer is the expected offset
- assertThat(reader.consumer().position(partition)).isEqualTo(expectedOffset);
- }
-
- // ------------------
-
- private void assignSplitsAndFetchUntilFinish(KafkaPartitionSplitReader reader, int readerId)
- throws IOException {
- Map<String, KafkaPartitionSplit> splits =
- assignSplits(reader, splitsByOwners.get(readerId));
-
- Map<String, Integer> numConsumedRecords = new HashMap<>();
- Set<String> finishedSplits = new HashSet<>();
- while (finishedSplits.size() < splits.size()) {
- RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> recordsBySplitIds = reader.fetch();
- String splitId = recordsBySplitIds.nextSplit();
- while (splitId != null) {
- // Collect the records in this split.
- List<ConsumerRecord<byte[], byte[]>> splitFetch = new ArrayList<>();
- ConsumerRecord<byte[], byte[]> record;
- while ((record = recordsBySplitIds.nextRecordFromSplit()) != null) {
- splitFetch.add(record);
- }
-
- // Compute the expected next offset for the split.
- TopicPartition tp = splits.get(splitId).getTopicPartition();
- long earliestOffset = earliestOffsets.get(tp);
- int numConsumedRecordsForSplit = numConsumedRecords.getOrDefault(splitId, 0);
- long expectedStartingOffset = earliestOffset + numConsumedRecordsForSplit;
-
- // verify the consumed records.
- if (verifyConsumed(splits.get(splitId), expectedStartingOffset, splitFetch)) {
- finishedSplits.add(splitId);
- }
- numConsumedRecords.compute(
- splitId,
- (ignored, recordCount) ->
- recordCount == null
- ? splitFetch.size()
- : recordCount + splitFetch.size());
- splitId = recordsBySplitIds.nextSplit();
- }
- }
-
- // Verify the number of records consumed from each split.
- numConsumedRecords.forEach(
- (splitId, recordCount) -> {
- TopicPartition tp = splits.get(splitId).getTopicPartition();
- long earliestOffset = earliestOffsets.get(tp);
- long expectedRecordCount = NUM_RECORDS_PER_PARTITION - earliestOffset;
- assertThat((long) recordCount)
- .as(
- String.format(
- "%s should have %d records.",
- splits.get(splitId), expectedRecordCount))
- .isEqualTo(expectedRecordCount);
- });
- }
-
- // ------------------
-
- private KafkaPartitionSplitReader createReader() {
- return createReader(
- new Properties(), UnregisteredMetricsGroup.createSourceReaderMetricGroup());
- }
-
- private KafkaPartitionSplitReader createReader(
- Properties additionalProperties, SourceReaderMetricGroup sourceReaderMetricGroup) {
- Properties props = new Properties();
- props.putAll(KafkaSourceTestEnv.getConsumerProperties(ByteArrayDeserializer.class));
- props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
- if (!additionalProperties.isEmpty()) {
- props.putAll(additionalProperties);
- }
- KafkaSourceReaderMetrics kafkaSourceReaderMetrics =
- new KafkaSourceReaderMetrics(sourceReaderMetricGroup);
- return new KafkaPartitionSplitReader(
- props,
- new TestingReaderContext(new Configuration(), sourceReaderMetricGroup),
- kafkaSourceReaderMetrics);
- }
-
- private Map<String, KafkaPartitionSplit> assignSplits(
- KafkaPartitionSplitReader reader, Map<String, KafkaPartitionSplit> splits) {
- SplitsChange<KafkaPartitionSplit> splitsChange =
- new SplitsAddition<>(new ArrayList<>(splits.values()));
- reader.handleSplitsChanges(splitsChange);
- return splits;
- }
-
- private boolean verifyConsumed(
- final KafkaPartitionSplit split,
- final long expectedStartingOffset,
- final Collection<ConsumerRecord<byte[], byte[]>> consumed) {
- long expectedOffset = expectedStartingOffset;
-
- for (ConsumerRecord<byte[], byte[]> record : consumed) {
- int expectedValue = (int) expectedOffset;
- long expectedTimestamp = expectedOffset * 1000L;
-
- assertThat(deserializer.deserialize(record.topic(), record.value()))
- .isEqualTo(expectedValue);
- assertThat(record.offset()).isEqualTo(expectedOffset);
- assertThat(record.timestamp()).isEqualTo(expectedTimestamp);
-
- expectedOffset++;
- }
- if (split.getStoppingOffset().isPresent()) {
- return expectedOffset == split.getStoppingOffset().get();
- } else {
- return false;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java
deleted file mode 100644
index e4ee39d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java
+++ /dev/null
@@ -1,635 +0,0 @@
-/*
- * 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.reader;
-
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.ReaderOutput;
-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.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
-import org.apache.flink.connector.kafka.source.KafkaSourceTestUtils;
-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.source.split.KafkaPartitionSplit;
-import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
-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.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.testutils.MetricListener;
-import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.IntegerDeserializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.COMMITS_SUCCEEDED_METRIC_COUNTER;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.COMMITTED_OFFSET_METRIC_GAUGE;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.CURRENT_OFFSET_METRIC_GAUGE;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.INITIAL_OFFSET;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.KAFKA_CONSUMER_METRIC_GROUP;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.KAFKA_SOURCE_READER_METRIC_GROUP;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.PARTITION_GROUP;
-import static org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics.TOPIC_GROUP;
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv.NUM_PARTITIONS;
-import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Unit tests for {@link KafkaSourceReader}. */
-public class KafkaSourceReaderTest extends SourceReaderTestBase<KafkaPartitionSplit> {
- private static final String TOPIC = "KafkaSourceReaderTest";
-
- @BeforeAll
- public static void setup() throws Throwable {
- KafkaSourceTestEnv.setup();
- try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
- adminClient
- .createTopics(
- Collections.singleton(new NewTopic(TOPIC, NUM_PARTITIONS, (short) 1)))
- .all()
- .get();
- // Use the admin client to trigger the creation of internal __consumer_offsets topic.
- // This makes sure that we won't see unavailable coordinator in the tests.
- waitUtil(
- () -> {
- try {
- adminClient
- .listConsumerGroupOffsets("AnyGroup")
- .partitionsToOffsetAndMetadata()
- .get();
- } catch (Exception e) {
- return false;
- }
- return true;
- },
- Duration.ofSeconds(60),
- "Waiting for offsets topic creation failed.");
- }
- KafkaSourceTestEnv.produceToKafka(
- getRecords(), StringSerializer.class, IntegerSerializer.class);
- }
-
- @AfterAll
- public static void tearDown() throws Exception {
- KafkaSourceTestEnv.tearDown();
- }
-
- protected int getNumSplits() {
- return NUM_PARTITIONS;
- }
-
- // -----------------------------------------
-
- @Test
- void testCommitOffsetsWithoutAliveFetchers() throws Exception {
- final String groupId = "testCommitOffsetsWithoutAliveFetchers";
- try (KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(Boundedness.CONTINUOUS_UNBOUNDED, groupId)) {
- KafkaPartitionSplit split =
- new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 0, NUM_RECORDS_PER_SPLIT);
- reader.addSplits(Collections.singletonList(split));
- reader.notifyNoMoreSplits();
- ReaderOutput<Integer> output = new TestingReaderOutput<>();
- InputStatus status;
- do {
- status = reader.pollNext(output);
- } while (status != InputStatus.NOTHING_AVAILABLE);
- pollUntil(
- reader,
- output,
- () -> reader.getNumAliveFetchers() == 0,
- "The split fetcher did not exit before timeout.");
- reader.snapshotState(100L);
- reader.notifyCheckpointComplete(100L);
- // Due to a bug in KafkaConsumer, when the consumer closes, the offset commit callback
- // won't be fired, so the offsetsToCommit map won't be cleaned. To make the test
- // stable, we add a split whose starting offset is the log end offset, so the
- // split fetcher won't become idle and exit after commitOffsetAsync is invoked from
- // notifyCheckpointComplete().
- reader.addSplits(
- Collections.singletonList(
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 0), NUM_RECORDS_PER_SPLIT)));
- pollUntil(
- reader,
- output,
- () -> reader.getOffsetsToCommit().isEmpty(),
- "The offset commit did not finish before timeout.");
- }
- // Verify the committed offsets.
- try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
- Map<TopicPartition, OffsetAndMetadata> committedOffsets =
- adminClient
- .listConsumerGroupOffsets(groupId)
- .partitionsToOffsetAndMetadata()
- .get();
- assertThat(committedOffsets).hasSize(1);
- assertThat(committedOffsets.values())
- .extracting(OffsetAndMetadata::offset)
- .allMatch(offset -> offset == NUM_RECORDS_PER_SPLIT);
- }
- }
-
- @Test
- void testCommitEmptyOffsets() throws Exception {
- final String groupId = "testCommitEmptyOffsets";
- try (KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(Boundedness.CONTINUOUS_UNBOUNDED, groupId)) {
- reader.snapshotState(100L);
- reader.notifyCheckpointComplete(100L);
- }
- // Verify the committed offsets.
- try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
- Map<TopicPartition, OffsetAndMetadata> committedOffsets =
- adminClient
- .listConsumerGroupOffsets(groupId)
- .partitionsToOffsetAndMetadata()
- .get();
- assertThat(committedOffsets).isEmpty();
- }
- }
-
- @Test
- void testOffsetCommitOnCheckpointComplete() throws Exception {
- final String groupId = "testOffsetCommitOnCheckpointComplete";
- try (KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(Boundedness.CONTINUOUS_UNBOUNDED, groupId)) {
- reader.addSplits(
- getSplits(numSplits, NUM_RECORDS_PER_SPLIT, Boundedness.CONTINUOUS_UNBOUNDED));
- ValidatingSourceOutput output = new ValidatingSourceOutput();
- long checkpointId = 0;
- do {
- checkpointId++;
- reader.pollNext(output);
- // Create a checkpoint for each message consumption, but not complete them.
- reader.snapshotState(checkpointId);
- } while (output.count() < totalNumRecords);
-
- // The completion of the last checkpoint should subsume all the previous checkpoitns.
- assertThat(reader.getOffsetsToCommit()).hasSize((int) checkpointId);
-
- long lastCheckpointId = checkpointId;
- waitUtil(
- () -> {
- try {
- reader.notifyCheckpointComplete(lastCheckpointId);
- } catch (Exception exception) {
- throw new RuntimeException(
- "Caught unexpected exception when polling from the reader",
- exception);
- }
- return reader.getOffsetsToCommit().isEmpty();
- },
- Duration.ofSeconds(60),
- Duration.ofSeconds(1),
- "The offset commit did not finish before timeout.");
- }
-
- // Verify the committed offsets.
- try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) {
- Map<TopicPartition, OffsetAndMetadata> committedOffsets =
- adminClient
- .listConsumerGroupOffsets(groupId)
- .partitionsToOffsetAndMetadata()
- .get();
- assertThat(committedOffsets).hasSize(numSplits);
- assertThat(committedOffsets.values())
- .extracting(OffsetAndMetadata::offset)
- .allMatch(offset -> offset == NUM_RECORDS_PER_SPLIT);
- }
- }
-
- @Test
- void testNotCommitOffsetsForUninitializedSplits() throws Exception {
- final long checkpointId = 1234L;
- try (KafkaSourceReader<Integer> reader = (KafkaSourceReader<Integer>) createReader()) {
- KafkaPartitionSplit split =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 0), KafkaPartitionSplit.EARLIEST_OFFSET);
- reader.addSplits(Collections.singletonList(split));
- reader.snapshotState(checkpointId);
- assertThat(reader.getOffsetsToCommit()).hasSize(1);
- assertThat(reader.getOffsetsToCommit().get(checkpointId)).isEmpty();
- }
- }
-
- @Test
- void testDisableOffsetCommit() throws Exception {
- final Properties properties = new Properties();
- properties.setProperty(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false");
- try (KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(
- Boundedness.CONTINUOUS_UNBOUNDED,
- new TestingReaderContext(),
- (ignore) -> {},
- properties)) {
- reader.addSplits(
- getSplits(numSplits, NUM_RECORDS_PER_SPLIT, Boundedness.CONTINUOUS_UNBOUNDED));
- ValidatingSourceOutput output = new ValidatingSourceOutput();
- long checkpointId = 0;
- do {
- checkpointId++;
- reader.pollNext(output);
- // Create a checkpoint for each message consumption, but not complete them.
- reader.snapshotState(checkpointId);
- // Offsets to commit should be always empty because offset commit is disabled
- assertThat(reader.getOffsetsToCommit()).isEmpty();
- } while (output.count() < totalNumRecords);
- }
- }
-
- @Test
- void testKafkaSourceMetrics() throws Exception {
- final MetricListener metricListener = new MetricListener();
- final String groupId = "testKafkaSourceMetrics";
- final TopicPartition tp0 = new TopicPartition(TOPIC, 0);
- final TopicPartition tp1 = new TopicPartition(TOPIC, 1);
-
- try (KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(
- Boundedness.CONTINUOUS_UNBOUNDED,
- groupId,
- metricListener.getMetricGroup())) {
-
- KafkaPartitionSplit split0 =
- new KafkaPartitionSplit(tp0, KafkaPartitionSplit.EARLIEST_OFFSET);
- KafkaPartitionSplit split1 =
- new KafkaPartitionSplit(tp1, KafkaPartitionSplit.EARLIEST_OFFSET);
- reader.addSplits(Arrays.asList(split0, split1));
-
- TestingReaderOutput<Integer> output = new TestingReaderOutput<>();
- pollUntil(
- reader,
- output,
- () -> output.getEmittedRecords().size() == NUM_RECORDS_PER_SPLIT * 2,
- String.format(
- "Failed to poll %d records until timeout", NUM_RECORDS_PER_SPLIT * 2));
-
- // Metric "records-consumed-total" of KafkaConsumer should be NUM_RECORDS_PER_SPLIT
- assertThat(getKafkaConsumerMetric("records-consumed-total", metricListener))
- .isEqualTo(NUM_RECORDS_PER_SPLIT * 2);
-
- // Current consuming offset should be NUM_RECORD_PER_SPLIT - 1
- assertThat(getCurrentOffsetMetric(tp0, metricListener))
- .isEqualTo(NUM_RECORDS_PER_SPLIT - 1);
- assertThat(getCurrentOffsetMetric(tp1, metricListener))
- .isEqualTo(NUM_RECORDS_PER_SPLIT - 1);
-
- // No offset is committed till now
- assertThat(getCommittedOffsetMetric(tp0, metricListener)).isEqualTo(INITIAL_OFFSET);
- assertThat(getCommittedOffsetMetric(tp1, metricListener)).isEqualTo(INITIAL_OFFSET);
-
- // Trigger offset commit
- final long checkpointId = 15213L;
- reader.snapshotState(checkpointId);
- waitUtil(
- () -> {
- try {
- reader.notifyCheckpointComplete(checkpointId);
- } catch (Exception e) {
- throw new RuntimeException(
- "Failed to notify checkpoint complete to reader", e);
- }
- return reader.getOffsetsToCommit().isEmpty();
- },
- Duration.ofSeconds(60),
- Duration.ofSeconds(1),
- String.format(
- "Offsets are not committed successfully. Dangling offsets: %s",
- reader.getOffsetsToCommit()));
-
- // Metric "commit-total" of KafkaConsumer should be greater than 0
- // It's hard to know the exactly number of commit because of the retry
- assertThat(getKafkaConsumerMetric("commit-total", metricListener)).isGreaterThan(0L);
-
- // Committed offset should be NUM_RECORD_PER_SPLIT
- assertThat(getCommittedOffsetMetric(tp0, metricListener))
- .isEqualTo(NUM_RECORDS_PER_SPLIT);
- assertThat(getCommittedOffsetMetric(tp1, metricListener))
- .isEqualTo(NUM_RECORDS_PER_SPLIT);
-
- // Number of successful commits should be greater than 0
- final Optional<Counter> commitsSucceeded =
- metricListener.getCounter(
- KAFKA_SOURCE_READER_METRIC_GROUP, COMMITS_SUCCEEDED_METRIC_COUNTER);
- assertThat(commitsSucceeded).isPresent();
- assertThat(commitsSucceeded.get().getCount()).isGreaterThan(0L);
- }
- }
-
- @Test
- void testAssigningEmptySplits() throws Exception {
- // Normal split with NUM_RECORDS_PER_SPLIT records
- final KafkaPartitionSplit normalSplit =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 0), 0, KafkaPartitionSplit.LATEST_OFFSET);
- // Empty split with no record
- final KafkaPartitionSplit emptySplit =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 1), NUM_RECORDS_PER_SPLIT, NUM_RECORDS_PER_SPLIT);
- // Split finished hook for listening finished splits
- final Set<String> finishedSplits = new HashSet<>();
- final Consumer<Collection<String>> splitFinishedHook = finishedSplits::addAll;
-
- try (final KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(
- Boundedness.BOUNDED,
- "KafkaSourceReaderTestGroup",
- new TestingReaderContext(),
- splitFinishedHook)) {
- reader.addSplits(Arrays.asList(normalSplit, emptySplit));
- pollUntil(
- reader,
- new TestingReaderOutput<>(),
- () -> reader.getNumAliveFetchers() == 0,
- "The split fetcher did not exit before timeout.");
- assertThat(finishedSplits)
- .containsExactlyInAnyOrder(
- KafkaPartitionSplit.toSplitId(normalSplit.getTopicPartition()),
- KafkaPartitionSplit.toSplitId(emptySplit.getTopicPartition()));
- }
- }
-
- @Test
- void testAssigningEmptySplitOnly() throws Exception {
- // Empty split with no record
- KafkaPartitionSplit emptySplit0 =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 0), NUM_RECORDS_PER_SPLIT, NUM_RECORDS_PER_SPLIT);
- KafkaPartitionSplit emptySplit1 =
- new KafkaPartitionSplit(
- new TopicPartition(TOPIC, 1), NUM_RECORDS_PER_SPLIT, NUM_RECORDS_PER_SPLIT);
- // Split finished hook for listening finished splits
- final Set<String> finishedSplits = new HashSet<>();
- final Consumer<Collection<String>> splitFinishedHook = finishedSplits::addAll;
-
- try (final KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(
- Boundedness.BOUNDED,
- "KafkaSourceReaderTestGroup",
- new TestingReaderContext(),
- splitFinishedHook)) {
- reader.addSplits(Arrays.asList(emptySplit0, emptySplit1));
- pollUntil(
- reader,
- new TestingReaderOutput<>(),
- () -> reader.getNumAliveFetchers() == 0,
- "The split fetcher did not exit before timeout.");
- assertThat(reader.getNumAliveFetchers()).isEqualTo(0);
- assertThat(finishedSplits)
- .containsExactly(emptySplit0.splitId(), emptySplit1.splitId());
- }
- }
-
- @Test
- public void testSupportsPausingOrResumingSplits() throws Exception {
- final Set<String> finishedSplits = new HashSet<>();
-
- try (final KafkaSourceReader<Integer> reader =
- (KafkaSourceReader<Integer>)
- createReader(
- Boundedness.BOUNDED,
- "groupId",
- new TestingReaderContext(),
- finishedSplits::addAll)) {
- KafkaPartitionSplit split1 =
- new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 0, NUM_RECORDS_PER_SPLIT);
- KafkaPartitionSplit split2 =
- new KafkaPartitionSplit(new TopicPartition(TOPIC, 1), 0, NUM_RECORDS_PER_SPLIT);
- reader.addSplits(Arrays.asList(split1, split2));
-
- TestingReaderOutput<Integer> output = new TestingReaderOutput<>();
-
- reader.pauseOrResumeSplits(
- Collections.singleton(split1.splitId()), Collections.emptyList());
-
- pollUntil(
- reader,
- output,
- () ->
- finishedSplits.contains(split2.splitId())
- && output.getEmittedRecords().size() == NUM_RECORDS_PER_SPLIT,
- "The split fetcher did not exit before timeout.");
-
- reader.pauseOrResumeSplits(
- Collections.emptyList(), Collections.singleton(split1.splitId()));
-
- pollUntil(
- reader,
- output,
- () ->
- finishedSplits.contains(split1.splitId())
- && output.getEmittedRecords().size()
- == NUM_RECORDS_PER_SPLIT * 2,
- "The split fetcher did not exit before timeout.");
-
- assertThat(finishedSplits).containsExactly(split1.splitId(), split2.splitId());
- }
- }
-
- // ------------------------------------------
-
- @Override
- protected SourceReader<Integer, KafkaPartitionSplit> createReader() throws Exception {
- return createReader(Boundedness.BOUNDED, "KafkaSourceReaderTestGroup");
- }
-
- @Override
- protected List<KafkaPartitionSplit> getSplits(
- int numSplits, int numRecordsPerSplit, Boundedness boundedness) {
- List<KafkaPartitionSplit> splits = new ArrayList<>();
- for (int i = 0; i < numSplits; i++) {
- splits.add(getSplit(i, numRecordsPerSplit, boundedness));
- }
- return splits;
- }
-
- @Override
- protected KafkaPartitionSplit getSplit(int splitId, int numRecords, Boundedness boundedness) {
- long stoppingOffset =
- boundedness == Boundedness.BOUNDED
- ? NUM_RECORDS_PER_SPLIT
- : KafkaPartitionSplit.NO_STOPPING_OFFSET;
- return new KafkaPartitionSplit(new TopicPartition(TOPIC, splitId), 0L, stoppingOffset);
- }
-
- @Override
- protected long getNextRecordIndex(KafkaPartitionSplit split) {
- return split.getStartingOffset();
- }
-
- // ---------------------
-
- private SourceReader<Integer, KafkaPartitionSplit> createReader(
- Boundedness boundedness, String groupId) throws Exception {
- return createReader(boundedness, groupId, new TestingReaderContext(), (ignore) -> {});
- }
-
- private SourceReader<Integer, KafkaPartitionSplit> createReader(
- Boundedness boundedness, String groupId, MetricGroup metricGroup) throws Exception {
- return createReader(
- boundedness,
- groupId,
- new TestingReaderContext(
- new Configuration(), InternalSourceReaderMetricGroup.mock(metricGroup)),
- (ignore) -> {});
- }
-
- private SourceReader<Integer, KafkaPartitionSplit> createReader(
- Boundedness boundedness,
- String groupId,
- SourceReaderContext context,
- Consumer<Collection<String>> splitFinishedHook)
- throws Exception {
- Properties properties = new Properties();
- properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId);
- return createReader(boundedness, context, splitFinishedHook, properties);
- }
-
- private SourceReader<Integer, KafkaPartitionSplit> createReader(
- Boundedness boundedness,
- SourceReaderContext context,
- Consumer<Collection<String>> splitFinishedHook,
- Properties props)
- throws Exception {
- KafkaSourceBuilder<Integer> builder =
- KafkaSource.<Integer>builder()
- .setClientIdPrefix("KafkaSourceReaderTest")
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(
- IntegerDeserializer.class))
- .setPartitions(Collections.singleton(new TopicPartition("AnyTopic", 0)))
- .setProperty(
- ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
- KafkaSourceTestEnv.brokerConnectionStrings)
- .setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
- .setProperties(props);
- if (boundedness == Boundedness.BOUNDED) {
- builder.setBounded(OffsetsInitializer.latest());
- }
-
- return KafkaSourceTestUtils.createReaderWithFinishedSplitHook(
- builder.build(), context, splitFinishedHook);
- }
-
- private void pollUntil(
- KafkaSourceReader<Integer> reader,
- ReaderOutput<Integer> output,
- Supplier<Boolean> condition,
- String errorMessage)
- throws Exception {
- waitUtil(
- () -> {
- try {
- reader.pollNext(output);
- } catch (Exception exception) {
- throw new RuntimeException(
- "Caught unexpected exception when polling from the reader",
- exception);
- }
- return condition.get();
- },
- Duration.ofSeconds(60),
- errorMessage);
- }
-
- private long getKafkaConsumerMetric(String name, MetricListener listener) {
- final Optional<Gauge<Object>> kafkaConsumerGauge =
- listener.getGauge(
- KAFKA_SOURCE_READER_METRIC_GROUP, KAFKA_CONSUMER_METRIC_GROUP, name);
- assertThat(kafkaConsumerGauge).isPresent();
- return ((Double) kafkaConsumerGauge.get().getValue()).longValue();
- }
-
- private long getCurrentOffsetMetric(TopicPartition tp, MetricListener listener) {
- final Optional<Gauge<Object>> currentOffsetGauge =
- listener.getGauge(
- KAFKA_SOURCE_READER_METRIC_GROUP,
- TOPIC_GROUP,
- tp.topic(),
- PARTITION_GROUP,
- String.valueOf(tp.partition()),
- CURRENT_OFFSET_METRIC_GAUGE);
- assertThat(currentOffsetGauge).isPresent();
- return (long) currentOffsetGauge.get().getValue();
- }
-
- private long getCommittedOffsetMetric(TopicPartition tp, MetricListener listener) {
- final Optional<Gauge<Object>> committedOffsetGauge =
- listener.getGauge(
- KAFKA_SOURCE_READER_METRIC_GROUP,
- TOPIC_GROUP,
- tp.topic(),
- PARTITION_GROUP,
- String.valueOf(tp.partition()),
- COMMITTED_OFFSET_METRIC_GAUGE);
- assertThat(committedOffsetGauge).isPresent();
- return (long) committedOffsetGauge.get().getValue();
- }
-
- // ---------------------
-
- private static List<ProducerRecord<String, Integer>> getRecords() {
- List<ProducerRecord<String, Integer>> records = new ArrayList<>();
- for (int part = 0; part < NUM_PARTITIONS; part++) {
- for (int i = 0; i < NUM_RECORDS_PER_SPLIT; i++) {
- records.add(
- new ProducerRecord<>(
- TOPIC, part, TOPIC + "-" + part, part * NUM_RECORDS_PER_SPLIT + i));
- }
- }
- return records;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java
deleted file mode 100644
index 8766719..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.reader.deserializer;
-
-import org.apache.flink.connector.testutils.formats.DummyInitializationContext;
-import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext;
-import org.apache.flink.formats.json.JsonDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.common.Configurable;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Unit tests for KafkaRecordDeserializationSchema. */
-public class KafkaRecordDeserializationSchemaTest {
-
- private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper();
-
- private static Map<String, ?> configurableConfiguration;
- private static Map<String, ?> configuration;
- private static boolean isKeyDeserializer;
-
- @Before
- public void setUp() {
- configurableConfiguration = new HashMap<>(1);
- configuration = new HashMap<>(1);
- isKeyDeserializer = false;
- }
-
- @Test
- public void testKafkaDeserializationSchemaWrapper() throws Exception {
- final ConsumerRecord<byte[], byte[]> consumerRecord = getConsumerRecord();
- KafkaRecordDeserializationSchema<ObjectNode> schema =
- KafkaRecordDeserializationSchema.of(new JSONKeyValueDeserializationSchema(true));
- schema.open(new DummyInitializationContext());
- SimpleCollector<ObjectNode> collector = new SimpleCollector<>();
- schema.deserialize(consumerRecord, collector);
-
- assertThat(collector.list).hasSize(1);
- ObjectNode deserializedValue = collector.list.get(0);
-
- assertThat(deserializedValue.get("key").get("index").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("value").get("word").asText()).isEqualTo("world");
- assertThat(deserializedValue.get("metadata").get("topic").asText()).isEqualTo("topic#1");
- assertThat(deserializedValue.get("metadata").get("offset").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("metadata").get("partition").asInt()).isEqualTo(3);
- }
-
- @Test
- public void testKafkaValueDeserializationSchemaWrapper() throws Exception {
- final ConsumerRecord<byte[], byte[]> consumerRecord = getConsumerRecord();
- KafkaRecordDeserializationSchema<ObjectNode> schema =
- KafkaRecordDeserializationSchema.valueOnly(
- new JsonDeserializationSchema<>(ObjectNode.class));
- schema.open(new DummyInitializationContext());
- SimpleCollector<ObjectNode> collector = new SimpleCollector<>();
- schema.deserialize(consumerRecord, collector);
-
- assertThat(collector.list).hasSize(1);
- ObjectNode deserializedValue = collector.list.get(0);
-
- assertThat(deserializedValue.get("word").asText()).isEqualTo("world");
- assertThat(deserializedValue.get("key")).isNull();
- assertThat(deserializedValue.get("metadata")).isNull();
- }
-
- @Test
- public void testKafkaValueDeserializerWrapper() throws Exception {
- final String topic = "Topic";
- byte[] value = new StringSerializer().serialize(topic, "world");
- final ConsumerRecord<byte[], byte[]> consumerRecord =
- new ConsumerRecord<>(topic, 0, 0L, null, value);
- KafkaRecordDeserializationSchema<String> schema =
- KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class);
- schema.open(new TestingDeserializationContext());
-
- SimpleCollector<String> collector = new SimpleCollector<>();
- schema.deserialize(consumerRecord, collector);
-
- assertThat(collector.list).hasSize(1);
- assertThat(collector.list.get(0)).isEqualTo("world");
- }
-
- @Test
- public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Exception {
- final Map<String, String> config = ImmutableMap.of("simpleKey", "simpleValue");
- KafkaRecordDeserializationSchema<String> schema =
- KafkaRecordDeserializationSchema.valueOnly(SimpleStringSerializer.class, config);
- schema.open(new TestingDeserializationContext());
- assertThat(config).isEqualTo(configuration);
- assertThat(isKeyDeserializer).isFalse();
- assertThat(configurableConfiguration).isEmpty();
- }
-
- @Test
- public void testKafkaValueDeserializerWrapperWithConfigurable() throws Exception {
- final Map<String, String> config = ImmutableMap.of("configKey", "configValue");
- KafkaRecordDeserializationSchema<String> schema =
- KafkaRecordDeserializationSchema.valueOnly(
- ConfigurableStringSerializer.class, config);
- schema.open(new TestingDeserializationContext());
- assertThat(config).isEqualTo(configurableConfiguration);
- assertThat(isKeyDeserializer).isFalse();
- assertThat(configuration).isEmpty();
- }
-
- private ConsumerRecord<byte[], byte[]> getConsumerRecord() throws JsonProcessingException {
- ObjectNode initialKey = OBJECT_MAPPER.createObjectNode();
- initialKey.put("index", 4);
- byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey);
-
- ObjectNode initialValue = OBJECT_MAPPER.createObjectNode();
- initialValue.put("word", "world");
- byte[] serializedValue = OBJECT_MAPPER.writeValueAsBytes(initialValue);
-
- return new ConsumerRecord<>("topic#1", 3, 4L, serializedKey, serializedValue);
- }
-
- private static class SimpleCollector<T> implements Collector<T> {
-
- private final List<T> list = new ArrayList<>();
-
- @Override
- public void collect(T record) {
- list.add(record);
- }
-
- @Override
- public void close() {
- // do nothing
- }
- }
-
- /**
- * Serializer based on Kafka's serialization stack. This is the special case that implements
- * {@link Configurable}
- *
- * <p>This class must be public to make it instantiable by the tests.
- */
- public static class ConfigurableStringSerializer extends StringDeserializer
- implements Configurable {
- @Override
- public void configure(Map<String, ?> configs) {
- configurableConfiguration = configs;
- }
- }
-
- /**
- * Serializer based on Kafka's serialization stack.
- *
- * <p>This class must be public to make it instantiable by the tests.
- */
- public static class SimpleStringSerializer extends StringDeserializer {
- @Override
- public void configure(Map<String, ?> configs, boolean isKey) {
- configuration = configs;
- isKeyDeserializer = isKey;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java
deleted file mode 100644
index 4ca5c9c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.split;
-
-import org.apache.kafka.common.TopicPartition;
-import org.assertj.core.util.Lists;
-import org.junit.jupiter.api.Test;
-
-import java.io.IOException;
-import java.util.List;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link KafkaPartitionSplitSerializer}. */
-public class KafkaPartitionSplitSerializerTest {
-
- @Test
- public void testSerializer() throws IOException {
- String topic = "topic";
- Long offsetZero = 0L;
- Long normalOffset = 1L;
- TopicPartition topicPartition = new TopicPartition(topic, 1);
- List<Long> stoppingOffsets =
- Lists.newArrayList(
- KafkaPartitionSplit.COMMITTED_OFFSET,
- KafkaPartitionSplit.LATEST_OFFSET,
- offsetZero,
- normalOffset);
- KafkaPartitionSplitSerializer splitSerializer = new KafkaPartitionSplitSerializer();
- for (Long stoppingOffset : stoppingOffsets) {
- KafkaPartitionSplit kafkaPartitionSplit =
- new KafkaPartitionSplit(topicPartition, 0, stoppingOffset);
- byte[] serialize = splitSerializer.serialize(kafkaPartitionSplit);
- KafkaPartitionSplit deserializeSplit =
- splitSerializer.deserialize(splitSerializer.getVersion(), serialize);
- assertThat(deserializeSplit).isEqualTo(kafkaPartitionSplit);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaPartitionDataWriter.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaPartitionDataWriter.java
deleted file mode 100644
index e223619..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaPartitionDataWriter.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.testframe.external.ExternalSystemSplitDataWriter;
-
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.Properties;
-
-/** Source split data writer for writing test data into Kafka topic partitions. */
-public class KafkaPartitionDataWriter implements ExternalSystemSplitDataWriter<String> {
-
- private final KafkaProducer<byte[], byte[]> kafkaProducer;
- private final TopicPartition topicPartition;
-
- public KafkaPartitionDataWriter(Properties producerProperties, TopicPartition topicPartition) {
- this.kafkaProducer = new KafkaProducer<>(producerProperties);
- this.topicPartition = topicPartition;
- }
-
- @Override
- public void writeRecords(List<String> records) {
- for (String record : records) {
- ProducerRecord<byte[], byte[]> producerRecord =
- new ProducerRecord<>(
- topicPartition.topic(),
- topicPartition.partition(),
- null,
- record.getBytes(StandardCharsets.UTF_8));
- kafkaProducer.send(producerRecord);
- }
- kafkaProducer.flush();
- }
-
- @Override
- public void close() {
- kafkaProducer.close();
- }
-
- public TopicPartition getTopicPartition() {
- return topicPartition;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContext.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContext.java
deleted file mode 100644
index 658a1c9..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContext.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * 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.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
-import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
-import org.apache.flink.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.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.AdminClientConfig;
-import org.apache.kafka.clients.admin.NewPartitions;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.regex.Pattern;
-
-import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
-
-/** External context for testing {@link KafkaSource}. */
-public class KafkaSourceExternalContext implements DataStreamSourceExternalContext<String> {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceExternalContext.class);
- private static final String TOPIC_NAME_PREFIX = "kafka-test-topic-";
- private static final Pattern TOPIC_NAME_PATTERN = Pattern.compile(TOPIC_NAME_PREFIX + ".*");
- private static final String GROUP_ID_PREFIX = "kafka-source-external-context-";
- private static final int NUM_RECORDS_UPPER_BOUND = 500;
- private static final int NUM_RECORDS_LOWER_BOUND = 100;
-
- private final List<URL> connectorJarPaths;
- private final String bootstrapServers;
- private final String topicName;
- private final SplitMappingMode splitMappingMode;
- private final AdminClient adminClient;
- private final List<KafkaPartitionDataWriter> writers = new ArrayList<>();
-
- protected KafkaSourceExternalContext(
- String bootstrapServers,
- SplitMappingMode splitMappingMode,
- List<URL> connectorJarPaths) {
- this.connectorJarPaths = connectorJarPaths;
- this.bootstrapServers = bootstrapServers;
- this.topicName = randomize(TOPIC_NAME_PREFIX);
- this.splitMappingMode = splitMappingMode;
- this.adminClient = createAdminClient();
- }
-
- @Override
- public List<URL> getConnectorJarPaths() {
- return this.connectorJarPaths;
- }
-
- @Override
- public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
- final KafkaSourceBuilder<String> builder = KafkaSource.builder();
-
- builder.setBootstrapServers(bootstrapServers)
- .setTopicPattern(TOPIC_NAME_PATTERN)
- .setGroupId(randomize(GROUP_ID_PREFIX))
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class));
-
- if (sourceSettings.getBoundedness().equals(Boundedness.BOUNDED)) {
- builder.setBounded(OffsetsInitializer.latest());
- }
-
- return builder.build();
- }
-
- @Override
- public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
- TestingSourceSettings sourceSettings) {
- KafkaPartitionDataWriter writer;
- try {
- switch (splitMappingMode) {
- case TOPIC:
- writer = createSinglePartitionTopic(writers.size());
- break;
- case PARTITION:
- writer = scaleOutTopic(this.topicName);
- break;
- default:
- throw new IllegalArgumentException(
- "Split mode should be either TOPIC or PARTITION");
- }
- } catch (Exception e) {
- throw new RuntimeException("Failed to create new splits", e);
- }
- writers.add(writer);
- return writer;
- }
-
- @Override
- public List<String> generateTestData(
- TestingSourceSettings sourceSettings, int splitIndex, long seed) {
- Random random = new Random(seed);
- int recordNum =
- random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND)
- + NUM_RECORDS_LOWER_BOUND;
- List<String> records = new ArrayList<>(recordNum);
-
- for (int i = 0; i < recordNum; i++) {
- int stringLength = random.nextInt(50) + 1;
- records.add(splitIndex + "-" + randomAlphanumeric(stringLength));
- }
-
- return records;
- }
-
- @Override
- public TypeInformation<String> getProducedType() {
- return TypeInformation.of(String.class);
- }
-
- @Override
- public void close() throws Exception {
- final List<String> topics = new ArrayList<>();
- writers.forEach(
- writer -> {
- topics.add(writer.getTopicPartition().topic());
- writer.close();
- });
- adminClient.deleteTopics(topics).all().get();
- }
-
- @Override
- public String toString() {
- return "KafkaSource-" + splitMappingMode.toString();
- }
-
- private String randomize(String prefix) {
- return prefix + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE);
- }
-
- private AdminClient createAdminClient() {
- Properties config = new Properties();
- config.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- return AdminClient.create(config);
- }
-
- private KafkaPartitionDataWriter createSinglePartitionTopic(int topicIndex) throws Exception {
- String newTopicName = topicName + "-" + topicIndex;
- LOG.info("Creating topic '{}'", newTopicName);
- adminClient
- .createTopics(Collections.singletonList(new NewTopic(newTopicName, 1, (short) 1)))
- .all()
- .get();
- return new KafkaPartitionDataWriter(
- getKafkaProducerProperties(topicIndex), new TopicPartition(newTopicName, 0));
- }
-
- private KafkaPartitionDataWriter scaleOutTopic(String topicName) throws Exception {
- final Set<String> topics = adminClient.listTopics().names().get();
- if (topics.contains(topicName)) {
- final Map<String, TopicDescription> topicDescriptions =
- adminClient
- .describeTopics(Collections.singletonList(topicName))
- .allTopicNames()
- .get();
- final int numPartitions = topicDescriptions.get(topicName).partitions().size();
- LOG.info("Creating partition {} for topic '{}'", numPartitions + 1, topicName);
- adminClient
- .createPartitions(
- Collections.singletonMap(
- topicName, NewPartitions.increaseTo(numPartitions + 1)))
- .all()
- .get();
- return new KafkaPartitionDataWriter(
- getKafkaProducerProperties(numPartitions),
- new TopicPartition(topicName, numPartitions));
- } else {
- LOG.info("Creating topic '{}'", topicName);
- adminClient
- .createTopics(Collections.singletonList(new NewTopic(topicName, 1, (short) 1)))
- .all()
- .get();
- return new KafkaPartitionDataWriter(
- getKafkaProducerProperties(0), new TopicPartition(topicName, 0));
- }
- }
-
- private Properties getKafkaProducerProperties(int producerId) {
- Properties kafkaProducerProperties = new Properties();
- kafkaProducerProperties.setProperty(
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
- kafkaProducerProperties.setProperty(
- ProducerConfig.CLIENT_ID_CONFIG,
- String.join(
- "-",
- "flink-kafka-split-writer",
- Integer.toString(producerId),
- Long.toString(ThreadLocalRandom.current().nextLong(Long.MAX_VALUE))));
- kafkaProducerProperties.setProperty(
- ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
- kafkaProducerProperties.setProperty(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
- return kafkaProducerProperties;
- }
-
- /** Mode of mapping split to Kafka components. */
- public enum SplitMappingMode {
- /** Use a single-partitioned topic as a split. */
- TOPIC,
-
- /** Use a partition in topic as a split. */
- PARTITION
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContextFactory.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContextFactory.java
deleted file mode 100644
index ef9113a..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceExternalContextFactory.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.testframe.external.ExternalContextFactory;
-
-import org.testcontainers.containers.KafkaContainer;
-
-import java.net.URL;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/** Factory of {@link KafkaSourceExternalContext}. */
-public class KafkaSourceExternalContextFactory
- implements ExternalContextFactory<KafkaSourceExternalContext> {
-
- private final KafkaContainer kafkaContainer;
- private final List<URL> connectorJars;
- private final KafkaSourceExternalContext.SplitMappingMode splitMappingMode;
-
- public KafkaSourceExternalContextFactory(
- KafkaContainer kafkaContainer,
- List<URL> connectorJars,
- KafkaSourceExternalContext.SplitMappingMode splitMappingMode) {
- this.kafkaContainer = kafkaContainer;
- this.connectorJars = connectorJars;
- this.splitMappingMode = splitMappingMode;
- }
-
- protected String getBootstrapServer() {
- final String internalEndpoints =
- kafkaContainer.getNetworkAliases().stream()
- .map(host -> String.join(":", host, Integer.toString(9092)))
- .collect(Collectors.joining(","));
- return String.join(",", kafkaContainer.getBootstrapServers(), internalEndpoints);
- }
-
- @Override
- public KafkaSourceExternalContext createExternalContext(String testName) {
- return new KafkaSourceExternalContext(
- getBootstrapServer(), this.splitMappingMode, this.connectorJars);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java
deleted file mode 100644
index 5173f9d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * 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.source.split.KafkaPartitionSplit;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestBase;
-
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
-import org.apache.kafka.clients.admin.RecordsToDelete;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.IntegerDeserializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ExecutionException;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Base class for KafkaSource unit tests. */
-public class KafkaSourceTestEnv extends KafkaTestBase {
- public static final String GROUP_ID = "KafkaSourceTestEnv";
- public static final int NUM_PARTITIONS = 10;
- public static final int NUM_RECORDS_PER_PARTITION = 10;
-
- private static AdminClient adminClient;
- private static KafkaConsumer<String, Integer> consumer;
-
- public static void setup() throws Throwable {
- prepare();
- adminClient = getAdminClient();
- consumer = getConsumer();
- }
-
- public static void tearDown() throws Exception {
- consumer.close();
- adminClient.close();
- shutDownServices();
- }
-
- // --------------------- public client related helpers ------------------
-
- public static AdminClient getAdminClient() {
- Properties props = new Properties();
- props.putAll(standardProps);
- return AdminClient.create(props);
- }
-
- public static KafkaConsumer<String, Integer> getConsumer() {
- Properties props = new Properties();
- props.putAll(standardProps);
- props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, GROUP_ID);
- props.setProperty(
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
- props.setProperty(
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
- IntegerDeserializer.class.getName());
- return new KafkaConsumer<>(props);
- }
-
- public static Properties getConsumerProperties(Class<?> deserializerClass) {
- Properties props = new Properties();
- props.putAll(standardProps);
- props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, GROUP_ID);
- props.setProperty(
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserializerClass.getName());
- props.setProperty(
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserializerClass.getName());
- return props;
- }
-
- // ------------------- topic information helpers -------------------
-
- public static Map<Integer, Map<String, KafkaPartitionSplit>> getSplitsByOwners(
- final Collection<String> topics, final int numSubtasks) {
- final Map<Integer, Map<String, KafkaPartitionSplit>> splitsByOwners = new HashMap<>();
- for (String topic : topics) {
- getPartitionsForTopic(topic)
- .forEach(
- tp -> {
- int ownerReader = Math.abs(tp.hashCode()) % numSubtasks;
- KafkaPartitionSplit split =
- new KafkaPartitionSplit(
- tp,
- getEarliestOffset(tp),
- (long) NUM_RECORDS_PER_PARTITION);
- splitsByOwners
- .computeIfAbsent(ownerReader, r -> new HashMap<>())
- .put(KafkaPartitionSplit.toSplitId(tp), split);
- });
- }
- return splitsByOwners;
- }
-
- /**
- * For a given partition {@code TOPIC-PARTITION} the {@code i}-th records looks like following.
- *
- * <pre>{@code
- * topic: TOPIC
- * partition: PARTITION
- * timestamp: 1000 * PARTITION
- * key: TOPIC-PARTITION
- * value: i
- * }</pre>
- */
- public static List<ProducerRecord<String, Integer>> getRecordsForPartition(TopicPartition tp) {
- List<ProducerRecord<String, Integer>> records = new ArrayList<>();
- for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) {
- records.add(
- new ProducerRecord<>(tp.topic(), tp.partition(), i * 1000L, tp.toString(), i));
- }
- return records;
- }
-
- /**
- * For a given partition {@code TOPIC-PARTITION} the {@code i}-th records looks like following.
- *
- * <pre>{@code
- * topic: TOPIC
- * partition: PARTITION
- * timestamp: null
- * key: TOPIC-PARTITION
- * value: i
- * }</pre>
- */
- public static List<ProducerRecord<String, Integer>> getRecordsForPartitionWithoutTimestamp(
- TopicPartition tp) {
- List<ProducerRecord<String, Integer>> records = new ArrayList<>();
- for (int i = 0; i < NUM_RECORDS_PER_PARTITION; i++) {
- records.add(new ProducerRecord<>(tp.topic(), tp.partition(), null, tp.toString(), i));
- }
- return records;
- }
-
- public static List<ProducerRecord<String, Integer>> getRecordsForTopic(String topic) {
- List<ProducerRecord<String, Integer>> records = new ArrayList<>();
- for (TopicPartition tp : getPartitionsForTopic(topic)) {
- records.addAll(getRecordsForPartition(tp));
- }
- return records;
- }
-
- public static List<ProducerRecord<String, Integer>> getRecordsForTopicWithoutTimestamp(
- String topic) {
- List<ProducerRecord<String, Integer>> records = new ArrayList<>();
- for (TopicPartition tp : getPartitionsForTopic(topic)) {
- records.addAll(getRecordsForPartitionWithoutTimestamp(tp));
- }
- return records;
- }
-
- public static List<TopicPartition> getPartitionsForTopics(Collection<String> topics) {
- List<TopicPartition> partitions = new ArrayList<>();
- topics.forEach(t -> partitions.addAll(getPartitionsForTopic(t)));
- return partitions;
- }
-
- public static List<TopicPartition> getPartitionsForTopic(String topic) {
- return consumer.partitionsFor(topic).stream()
- .map(pi -> new TopicPartition(pi.topic(), pi.partition()))
- .collect(Collectors.toList());
- }
-
- public static Map<TopicPartition, Long> getEarliestOffsets(List<TopicPartition> partitions) {
- Map<TopicPartition, Long> earliestOffsets = new HashMap<>();
- for (TopicPartition tp : partitions) {
- earliestOffsets.put(tp, getEarliestOffset(tp));
- }
- return earliestOffsets;
- }
-
- public static Map<TopicPartition, OffsetAndMetadata> getCommittedOffsets(
- List<TopicPartition> partitions) {
- Map<TopicPartition, OffsetAndMetadata> committedOffsets = new HashMap<>();
- for (TopicPartition tp : partitions) {
- committedOffsets.put(tp, new OffsetAndMetadata(tp.partition() + 2));
- }
- return committedOffsets;
- }
-
- public static long getEarliestOffset(TopicPartition tp) {
- return tp.partition();
- }
-
- // --------------- topic manipulation helpers ---------------
-
- public static void createTestTopic(String topic) {
- createTestTopic(topic, NUM_PARTITIONS, 1);
- }
-
- public static void setupEarliestOffsets(String topic) throws Throwable {
- // Delete some records to move the starting partition.
- List<TopicPartition> partitions = getPartitionsForTopic(topic);
- setupEarliestOffsets(partitions);
- }
-
- public static void setupEarliestOffsets(List<TopicPartition> partitions) throws Throwable {
- Map<TopicPartition, RecordsToDelete> toDelete = new HashMap<>();
- getEarliestOffsets(partitions)
- .forEach((tp, offset) -> toDelete.put(tp, RecordsToDelete.beforeOffset(offset)));
- adminClient.deleteRecords(toDelete).all().get();
- }
-
- public static void setupCommittedOffsets(String topic)
- throws ExecutionException, InterruptedException {
- List<TopicPartition> partitions = getPartitionsForTopic(topic);
- Map<TopicPartition, OffsetAndMetadata> committedOffsets = getCommittedOffsets(partitions);
- consumer.commitSync(committedOffsets);
- Map<TopicPartition, OffsetAndMetadata> toVerify =
- adminClient
- .listConsumerGroupOffsets(
- GROUP_ID,
- new ListConsumerGroupOffsetsOptions()
- .topicPartitions(
- new ArrayList<>(committedOffsets.keySet())))
- .partitionsToOffsetAndMetadata()
- .get();
- assertThat(toVerify).as("The offsets are not committed").isEqualTo(committedOffsets);
- }
-
- public static void produceToKafka(Collection<ProducerRecord<String, Integer>> records)
- throws Throwable {
- produceToKafka(records, StringSerializer.class, IntegerSerializer.class);
- }
-
- public static void setupTopic(
- String topic,
- boolean setupEarliestOffsets,
- boolean setupCommittedOffsets,
- Function<String, List<ProducerRecord<String, Integer>>> testDataGenerator)
- throws Throwable {
- createTestTopic(topic);
- produceToKafka(testDataGenerator.apply(topic));
- if (setupEarliestOffsets) {
- setupEarliestOffsets(topic);
- }
- if (setupCommittedOffsets) {
- setupCommittedOffsets(topic);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java
deleted file mode 100644
index 267f7c7..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.StringUtils;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.output.Slf4jLogConsumer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-/** Collection of methods to interact with a Kafka cluster. */
-public class KafkaUtil {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaUtil.class);
- private static final Duration CONSUMER_POLL_DURATION = Duration.ofSeconds(1);
-
- private KafkaUtil() {}
-
- /**
- * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log
- * levels with the ones used by the capturing logger.
- *
- * @param dockerImageVersion describing the Kafka image
- * @param logger to derive the log level from
- * @return configured Kafka container
- */
- public static KafkaContainer createKafkaContainer(String dockerImageVersion, Logger logger) {
- return createKafkaContainer(dockerImageVersion, logger, null);
- }
-
- /**
- * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log
- * levels with the ones used by the capturing logger, and set the prefix of logger.
- */
- public static KafkaContainer createKafkaContainer(
- String dockerImageVersion, Logger logger, String loggerPrefix) {
- String logLevel;
- if (logger.isTraceEnabled()) {
- logLevel = "TRACE";
- } else if (logger.isDebugEnabled()) {
- logLevel = "DEBUG";
- } else if (logger.isInfoEnabled()) {
- logLevel = "INFO";
- } else if (logger.isWarnEnabled()) {
- logLevel = "WARN";
- } else if (logger.isErrorEnabled()) {
- logLevel = "ERROR";
- } else {
- logLevel = "OFF";
- }
-
- Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(logger);
- if (!StringUtils.isNullOrWhitespaceOnly(loggerPrefix)) {
- logConsumer.withPrefix(loggerPrefix);
- }
- return new KafkaContainer(DockerImageName.parse(dockerImageVersion))
- .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1")
- .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1")
- .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1")
- .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false")
- .withEnv("KAFKA_LOG4J_ROOT_LOGLEVEL", logLevel)
- .withEnv("KAFKA_LOG4J_LOGGERS", "state.change.logger=" + logLevel)
- .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false")
- .withEnv(
- "KAFKA_TRANSACTION_MAX_TIMEOUT_MS",
- String.valueOf(Duration.ofHours(2).toMillis()))
- .withEnv("KAFKA_LOG4J_TOOLS_ROOT_LOGLEVEL", logLevel)
- .withLogConsumer(logConsumer);
- }
-
- /**
- * Drain all records available from the given topic from the beginning until the current highest
- * offset.
- *
- * <p>This method will fetch the latest offsets for the partitions once and only return records
- * until that point.
- *
- * @param topic to fetch from
- * @param properties used to configure the created {@link KafkaConsumer}
- * @param committed determines the mode {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG} with which
- * the consumer reads the records.
- * @return all {@link ConsumerRecord} in the topic
- * @throws KafkaException
- */
- public static List<ConsumerRecord<byte[], byte[]>> drainAllRecordsFromTopic(
- String topic, Properties properties, boolean committed) throws KafkaException {
- final Properties consumerConfig = new Properties();
- consumerConfig.putAll(properties);
- consumerConfig.put(
- ConsumerConfig.ISOLATION_LEVEL_CONFIG,
- committed ? "read_committed" : "read_uncommitted");
- return drainAllRecordsFromTopic(topic, consumerConfig);
- }
-
- /**
- * Drain all records available from the given topic from the beginning until the current highest
- * offset.
- *
- * <p>This method will fetch the latest offsets for the partitions once and only return records
- * until that point.
- *
- * @param topic to fetch from
- * @param properties used to configure the created {@link KafkaConsumer}
- * @return all {@link ConsumerRecord} in the topic
- * @throws KafkaException
- */
- public static List<ConsumerRecord<byte[], byte[]>> drainAllRecordsFromTopic(
- String topic, Properties properties) throws KafkaException {
- final Properties consumerConfig = new Properties();
- consumerConfig.putAll(properties);
- consumerConfig.put("key.deserializer", ByteArrayDeserializer.class.getName());
- consumerConfig.put("value.deserializer", ByteArrayDeserializer.class.getName());
- try (KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerConfig)) {
- Set<TopicPartition> topicPartitions = getAllPartitions(consumer, topic);
- Map<TopicPartition, Long> endOffsets = consumer.endOffsets(topicPartitions);
- consumer.assign(topicPartitions);
- consumer.seekToBeginning(topicPartitions);
-
- final List<ConsumerRecord<byte[], byte[]>> consumerRecords = new ArrayList<>();
- while (!topicPartitions.isEmpty()) {
- ConsumerRecords<byte[], byte[]> records = consumer.poll(CONSUMER_POLL_DURATION);
- LOG.debug("Fetched {} records from topic {}.", records.count(), topic);
-
- // Remove partitions from polling which have reached its end.
- final List<TopicPartition> finishedPartitions = new ArrayList<>();
- for (final TopicPartition topicPartition : topicPartitions) {
- final long position = consumer.position(topicPartition);
- final long endOffset = endOffsets.get(topicPartition);
- LOG.debug(
- "Endoffset {} and current position {} for partition {}",
- endOffset,
- position,
- topicPartition.partition());
- if (endOffset - position > 0) {
- continue;
- }
- finishedPartitions.add(topicPartition);
- }
- if (topicPartitions.removeAll(finishedPartitions)) {
- consumer.assign(topicPartitions);
- }
- for (ConsumerRecord<byte[], byte[]> r : records) {
- consumerRecords.add(r);
- }
- }
- return consumerRecords;
- }
- }
-
- private static Set<TopicPartition> getAllPartitions(
- KafkaConsumer<byte[], byte[]> consumer, String topic) {
- return consumer.partitionsFor(topic).stream()
- .map(info -> new TopicPartition(info.topic(), info.partition()))
- .collect(Collectors.toSet());
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java
deleted file mode 100644
index 8db2e59..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the {@link FlinkFixedPartitioner}. */
-public class FlinkFixedPartitionerTest {
-
- /**
- * Test for when there are more sinks than partitions.
- *
- * <pre>
- * Flink Sinks: Kafka Partitions
- * 1 ----------------> 1
- * 2 --------------/
- * 3 -------------/
- * 4 ------------/
- * </pre>
- */
- @Test
- public void testMoreFlinkThanBrokers() {
- FlinkFixedPartitioner<String> part = new FlinkFixedPartitioner<>();
-
- int[] partitions = new int[] {0};
-
- part.open(0, 4);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(0);
-
- part.open(1, 4);
- assertThat(part.partition("abc2", null, null, null, partitions)).isEqualTo(0);
-
- part.open(2, 4);
- assertThat(part.partition("abc3", null, null, null, partitions)).isEqualTo(0);
- assertThat(part.partition("abc3", null, null, null, partitions))
- .isEqualTo(0); // check if it is changing ;)
-
- part.open(3, 4);
- assertThat(part.partition("abc4", null, null, null, partitions)).isEqualTo(0);
- }
-
- /**
- * Tests for when there are more partitions than sinks.
- *
- * <pre>
- * Flink Sinks: Kafka Partitions
- * 1 ----------------> 1
- * 2 ----------------> 2
- * 3
- * 4
- * 5
- *
- * </pre>
- */
- @Test
- public void testFewerPartitions() {
- FlinkFixedPartitioner<String> part = new FlinkFixedPartitioner<>();
-
- int[] partitions = new int[] {0, 1, 2, 3, 4};
- part.open(0, 2);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(0);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(0);
-
- part.open(1, 2);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(1);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(1);
- }
-
- /*
- * Flink Sinks: Kafka Partitions
- * 1 ------------>---> 1
- * 2 -----------/----> 2
- * 3 ----------/
- */
- @Test
- public void testMixedCase() {
- FlinkFixedPartitioner<String> part = new FlinkFixedPartitioner<>();
- int[] partitions = new int[] {0, 1};
-
- part.open(0, 3);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(0);
-
- part.open(1, 3);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(1);
-
- part.open(2, 3);
- assertThat(part.partition("abc1", null, null, null, partitions)).isEqualTo(0);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
deleted file mode 100644
index 8c72e0b..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
+++ /dev/null
@@ -1,425 +0,0 @@
-/*
- * 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.FlinkVersion;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.flink.test.util.MigrationTest;
-import org.apache.flink.util.SerializedValue;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
- * done using previous Flink versions' {@link FlinkKafkaConsumerBase}.
- */
-@RunWith(Parameterized.class)
-public class FlinkKafkaConsumerBaseMigrationTest implements MigrationTest {
-
- private static final HashMap<KafkaTopicPartition, Long> PARTITION_STATE = new HashMap<>();
-
- static {
- PARTITION_STATE.put(new KafkaTopicPartition("abc", 13), 16768L);
- PARTITION_STATE.put(new KafkaTopicPartition("def", 7), 987654321L);
- }
-
- private static final List<String> TOPICS =
- new ArrayList<>(PARTITION_STATE.keySet())
- .stream().map(p -> p.getTopic()).distinct().collect(Collectors.toList());
-
- private final FlinkVersion testMigrateVersion;
-
- @Parameterized.Parameters(name = "Migration Savepoint: {0}")
- public static Collection<FlinkVersion> parameters() {
- return FlinkVersion.rangeOf(
- FlinkVersion.v1_8, MigrationTest.getMostRecentlyPublishedVersion());
- }
-
- public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) {
- this.testMigrateVersion = testMigrateVersion;
- }
-
- /** Manually run this to write binary snapshot data. */
- @SnapshotsGenerator
- public void writeSnapshot(FlinkVersion flinkGenerateSavepointVersion) throws Exception {
- writeSnapshot(
- "src/test/resources/kafka-consumer-migration-test-flink"
- + flinkGenerateSavepointVersion
- + "-snapshot",
- PARTITION_STATE);
-
- final HashMap<KafkaTopicPartition, Long> emptyState = new HashMap<>();
- writeSnapshot(
- "src/test/resources/kafka-consumer-migration-test-flink"
- + flinkGenerateSavepointVersion
- + "-empty-state-snapshot",
- emptyState);
- }
-
- private void writeSnapshot(String path, HashMap<KafkaTopicPartition, Long> state)
- throws Exception {
-
- final OneShotLatch latch = new OneShotLatch();
- final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
- doAnswer(
- new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocation) throws Throwable {
- latch.trigger();
- return null;
- }
- })
- .when(fetcher)
- .runFetchLoop();
-
- when(fetcher.snapshotCurrentState()).thenReturn(state);
-
- final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
- final DummyFlinkKafkaConsumer<String> consumerFunction =
- new DummyFlinkKafkaConsumer<>(
- fetcher,
- TOPICS,
- partitions,
- FlinkKafkaConsumerBase.PARTITION_DISCOVERY_DISABLED);
-
- StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
- new StreamSource<>(consumerFunction);
-
- final AbstractStreamOperatorTestHarness<String> testHarness =
- new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
- testHarness.setup();
- testHarness.open();
-
- final Throwable[] error = new Throwable[1];
-
- // run the source asynchronously
- Thread runner =
- new Thread() {
- @Override
- public void run() {
- try {
- consumerFunction.run(
- new DummySourceContext() {
- @Override
- public void collect(String element) {}
- });
- } catch (Throwable t) {
- t.printStackTrace();
- error[0] = t;
- }
- }
- };
- runner.start();
-
- if (!latch.isTriggered()) {
- latch.await();
- }
-
- final OperatorSubtaskState snapshot;
- synchronized (testHarness.getCheckpointLock()) {
- snapshot = testHarness.snapshot(0L, 0L);
- }
-
- OperatorSnapshotUtil.writeStateHandle(snapshot, path);
-
- consumerOperator.close();
- runner.join();
- }
-
- /** Test restoring from an legacy empty state, when no partitions could be found for topics. */
- @Test
- public void testRestoreFromEmptyStateNoPartitions() throws Exception {
- final DummyFlinkKafkaConsumer<String> consumerFunction =
- new DummyFlinkKafkaConsumer<>(
- Collections.singletonList("dummy-topic"),
- Collections.<KafkaTopicPartition>emptyList(),
- FlinkKafkaConsumerBase.PARTITION_DISCOVERY_DISABLED);
-
- StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
- new StreamSource<>(consumerFunction);
-
- final AbstractStreamOperatorTestHarness<String> testHarness =
- new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
- testHarness.setup();
-
- // restore state from binary snapshot file
- testHarness.initializeState(
- OperatorSnapshotUtil.getResourceFilename(
- "kafka-consumer-migration-test-flink"
- + testMigrateVersion
- + "-empty-state-snapshot"));
-
- testHarness.open();
-
- // assert that no partitions were found and is empty
- assertThat(consumerFunction.getSubscribedPartitionsToStartOffsets()).isEmpty();
-
- // assert that no state was restored
- assertThat(consumerFunction.getRestoredState()).isEmpty();
-
- consumerOperator.close();
- consumerOperator.cancel();
- }
-
- /**
- * Test restoring from an empty state taken using a previous Flink version, when some partitions
- * could be found for topics.
- */
- @Test
- public void testRestoreFromEmptyStateWithPartitions() throws Exception {
- final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
- final DummyFlinkKafkaConsumer<String> consumerFunction =
- new DummyFlinkKafkaConsumer<>(
- TOPICS, partitions, FlinkKafkaConsumerBase.PARTITION_DISCOVERY_DISABLED);
-
- StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
- new StreamSource<>(consumerFunction);
-
- final AbstractStreamOperatorTestHarness<String> testHarness =
- new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
- testHarness.setup();
-
- // restore state from binary snapshot file
- testHarness.initializeState(
- OperatorSnapshotUtil.getResourceFilename(
- "kafka-consumer-migration-test-flink"
- + testMigrateVersion
- + "-empty-state-snapshot"));
-
- testHarness.open();
-
- // the expected state in "kafka-consumer-migration-test-flink1.x-snapshot-empty-state";
- // all new partitions after the snapshot are considered as partitions that were created
- // while the
- // consumer wasn't running, and should start from the earliest offset.
- final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets =
- new HashMap<>();
- for (KafkaTopicPartition partition : PARTITION_STATE.keySet()) {
- expectedSubscribedPartitionsWithStartOffsets.put(
- partition, KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET);
- }
-
- // assert that there are partitions and is identical to expected list
- assertThat(consumerFunction.getSubscribedPartitionsToStartOffsets())
- .isNotEmpty()
- .isEqualTo(expectedSubscribedPartitionsWithStartOffsets);
-
- // the new partitions should have been considered as restored state
- assertThat(consumerFunction.getRestoredState()).isNotNull();
- assertThat(consumerFunction.getSubscribedPartitionsToStartOffsets()).isNotEmpty();
- for (Map.Entry<KafkaTopicPartition, Long> expectedEntry :
- expectedSubscribedPartitionsWithStartOffsets.entrySet()) {
- assertThat(consumerFunction.getRestoredState())
- .containsEntry(expectedEntry.getKey(), expectedEntry.getValue());
- }
-
- consumerOperator.close();
- consumerOperator.cancel();
- }
-
- /**
- * Test restoring from a non-empty state taken using a previous Flink version, when some
- * partitions could be found for topics.
- */
- @Test
- public void testRestore() throws Exception {
- final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
- final DummyFlinkKafkaConsumer<String> consumerFunction =
- new DummyFlinkKafkaConsumer<>(
- TOPICS, partitions, FlinkKafkaConsumerBase.PARTITION_DISCOVERY_DISABLED);
-
- StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
- new StreamSource<>(consumerFunction);
-
- final AbstractStreamOperatorTestHarness<String> testHarness =
- new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
- testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
- testHarness.setup();
-
- // restore state from binary snapshot file
- testHarness.initializeState(
- OperatorSnapshotUtil.getResourceFilename(
- "kafka-consumer-migration-test-flink" + testMigrateVersion + "-snapshot"));
-
- testHarness.open();
-
- // assert that there are partitions and is identical to expected list
- assertThat(consumerFunction.getSubscribedPartitionsToStartOffsets())
- .isNotEmpty()
- // on restore, subscribedPartitionsToStartOffsets should be identical to the
- // restored state
- .isEqualTo(PARTITION_STATE);
-
- // assert that state is correctly restored from legacy checkpoint
- assertThat(consumerFunction.getRestoredState()).isNotNull().isEqualTo(PARTITION_STATE);
-
- consumerOperator.close();
- consumerOperator.cancel();
- }
-
- // ------------------------------------------------------------------------
-
- private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
- private static final long serialVersionUID = 1L;
-
- private final List<KafkaTopicPartition> partitions;
-
- private final AbstractFetcher<T, ?> fetcher;
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- AbstractFetcher<T, ?> fetcher,
- List<String> topics,
- List<KafkaTopicPartition> partitions,
- long discoveryInterval) {
-
- super(
- topics,
- null,
- (KafkaDeserializationSchema<T>) mock(KafkaDeserializationSchema.class),
- discoveryInterval,
- false);
-
- this.fetcher = fetcher;
- this.partitions = partitions;
- }
-
- DummyFlinkKafkaConsumer(
- List<String> topics, List<KafkaTopicPartition> partitions, long discoveryInterval) {
- this(mock(AbstractFetcher.class), topics, partitions, discoveryInterval);
- }
-
- @Override
- protected AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- return fetcher;
- }
-
- @Override
- protected AbstractPartitionDiscoverer createPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks) {
-
- AbstractPartitionDiscoverer mockPartitionDiscoverer =
- mock(AbstractPartitionDiscoverer.class);
-
- try {
- when(mockPartitionDiscoverer.discoverPartitions()).thenReturn(partitions);
- } catch (Exception e) {
- // ignore
- }
- when(mockPartitionDiscoverer.setAndCheckDiscoveredPartition(
- any(KafkaTopicPartition.class)))
- .thenReturn(true);
-
- return mockPartitionDiscoverer;
- }
-
- @Override
- protected boolean getIsAutoCommitEnabled() {
- return false;
- }
-
- @Override
- protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
- Collection<KafkaTopicPartition> partitions, long timestamp) {
- throw new UnsupportedOperationException();
- }
- }
-
- private abstract static class DummySourceContext
- implements SourceFunction.SourceContext<String> {
-
- private final Object lock = new Object();
-
- @Override
- public void collectWithTimestamp(String element, long timestamp) {}
-
- @Override
- public void emitWatermark(Watermark mark) {}
-
- @Override
- public Object getCheckpointLock() {
- return lock;
- }
-
- @Override
- public void close() {}
-
- @Override
- public void markAsTemporarilyIdle() {}
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
deleted file mode 100644
index a243816..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
+++ /dev/null
@@ -1,1523 +0,0 @@
-/*
- * 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.api.common.ExecutionConfig;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.state.BroadcastState;
-import org.apache.flink.api.common.state.KeyedStateStore;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.testutils.CheckedThread;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.MockDeserializationSchema;
-import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.SerializedValue;
-import org.apache.flink.util.TestLogger;
-import org.apache.flink.util.function.SupplierWithException;
-import org.apache.flink.util.function.ThrowingRunnable;
-
-import org.junit.Test;
-
-import javax.annotation.Nonnull;
-
-import java.io.Serializable;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.OptionalLong;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkState;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.assertj.core.api.Assertions.fail;
-import static org.assertj.core.api.HamcrestCondition.matching;
-import static org.hamcrest.Matchers.everyItem;
-import static org.hamcrest.Matchers.hasSize;
-import static org.hamcrest.collection.IsIn.isIn;
-import static org.hamcrest.collection.IsMapContaining.hasKey;
-import static org.hamcrest.core.IsNot.not;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-
-/** Tests for the {@link FlinkKafkaConsumerBase}. */
-public class FlinkKafkaConsumerBaseTest extends TestLogger {
-
- private static final int maxParallelism = Short.MAX_VALUE / 2;
-
- /** Tests that not both types of timestamp extractors / watermark generators can be used. */
- @Test
- @SuppressWarnings("unchecked")
- public void testEitherWatermarkExtractor() {
- assertThatThrownBy(
- () ->
- new DummyFlinkKafkaConsumer<String>()
- .assignTimestampsAndWatermarks(
- (AssignerWithPeriodicWatermarks<String>) null))
- .isInstanceOf(NullPointerException.class);
-
- assertThatThrownBy(
- () ->
- new DummyFlinkKafkaConsumer<String>()
- .assignTimestampsAndWatermarks(
- (AssignerWithPunctuatedWatermarks<String>) null))
- .isInstanceOf(NullPointerException.class);
-
- final AssignerWithPeriodicWatermarks<String> periodicAssigner =
- mock(AssignerWithPeriodicWatermarks.class);
- final AssignerWithPunctuatedWatermarks<String> punctuatedAssigner =
- mock(AssignerWithPunctuatedWatermarks.class);
-
- DummyFlinkKafkaConsumer<String> c1 = new DummyFlinkKafkaConsumer<>();
- c1.assignTimestampsAndWatermarks(periodicAssigner);
- assertThatThrownBy(() -> c1.assignTimestampsAndWatermarks(punctuatedAssigner))
- .isInstanceOf(IllegalStateException.class);
-
- DummyFlinkKafkaConsumer<String> c2 = new DummyFlinkKafkaConsumer<>();
- c2.assignTimestampsAndWatermarks(punctuatedAssigner);
- assertThatThrownBy(() -> c2.assignTimestampsAndWatermarks(periodicAssigner))
- .isInstanceOf(IllegalStateException.class);
- }
-
- /** Tests that no checkpoints happen when the fetcher is not running. */
- @Test
- public void ignoreCheckpointWhenNotRunning() throws Exception {
- @SuppressWarnings("unchecked")
- final MockFetcher<String> fetcher = new MockFetcher<>();
- final FlinkKafkaConsumerBase<String> consumer =
- new DummyFlinkKafkaConsumer<>(
- fetcher, mock(AbstractPartitionDiscoverer.class), false);
-
- final TestingListState<Tuple2<KafkaTopicPartition, Long>> listState =
- new TestingListState<>();
- setupConsumer(consumer, false, listState, true, 0, 1);
-
- // snapshot before the fetcher starts running
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
-
- // no state should have been checkpointed
- assertThat(listState.get().iterator().hasNext()).isFalse();
-
- // acknowledgement of the checkpoint should also not result in any offset commits
- consumer.notifyCheckpointComplete(1L);
- assertThat(fetcher.getAndClearLastCommittedOffsets()).isNull();
- assertThat(fetcher.getCommitCount()).isEqualTo(0);
- }
-
- /**
- * Tests that when taking a checkpoint when the fetcher is not running yet, the checkpoint
- * correctly contains the restored state instead.
- */
- @Test
- public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception {
- @SuppressWarnings("unchecked")
- final FlinkKafkaConsumerBase<String> consumer = new DummyFlinkKafkaConsumer<>();
-
- final TestingListState<Tuple2<KafkaTopicPartition, Long>> restoredListState =
- new TestingListState<>();
- setupConsumer(consumer, true, restoredListState, true, 0, 1);
-
- // snapshot before the fetcher starts running
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(17, 17));
-
- // ensure that the list was cleared and refilled. while this is an implementation detail, we
- // use it here
- // to figure out that snapshotState() actually did something.
- assertThat(restoredListState.isClearCalled()).isTrue();
-
- Set<Serializable> expected = new HashSet<>();
-
- for (Serializable serializable : restoredListState.get()) {
- expected.add(serializable);
- }
-
- int counter = 0;
-
- for (Serializable serializable : restoredListState.get()) {
- assertThat(expected).contains(serializable);
- counter++;
- }
-
- assertThat(counter).isEqualTo(expected.size());
- }
-
- @Test
- public void testConfigureOnCheckpointsCommitMode() throws Exception {
- @SuppressWarnings("unchecked")
- // auto-commit enabled; this should be ignored in this case
- final DummyFlinkKafkaConsumer<String> consumer = new DummyFlinkKafkaConsumer<>(true);
-
- setupConsumer(
- consumer, false, null, true, // enable checkpointing; auto commit should be ignored
- 0, 1);
-
- assertThat(consumer.getOffsetCommitMode()).isEqualTo(OffsetCommitMode.ON_CHECKPOINTS);
- }
-
- @Test
- public void testConfigureAutoCommitMode() throws Exception {
- @SuppressWarnings("unchecked")
- final DummyFlinkKafkaConsumer<String> consumer = new DummyFlinkKafkaConsumer<>(true);
-
- setupConsumer(consumer);
-
- assertThat(consumer.getOffsetCommitMode()).isEqualTo(OffsetCommitMode.KAFKA_PERIODIC);
- }
-
- @Test
- public void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception {
- @SuppressWarnings("unchecked")
- // auto-commit enabled; this should be ignored in this case
- final DummyFlinkKafkaConsumer<String> consumer = new DummyFlinkKafkaConsumer<>(true);
- consumer.setCommitOffsetsOnCheckpoints(
- false); // disabling offset committing should override everything
-
- setupConsumer(
- consumer, false, null, true, // enable checkpointing; auto commit should be ignored
- 0, 1);
-
- assertThat(consumer.getOffsetCommitMode()).isEqualTo(OffsetCommitMode.DISABLED);
- }
-
- @Test
- public void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception {
- @SuppressWarnings("unchecked")
- final DummyFlinkKafkaConsumer<String> consumer = new DummyFlinkKafkaConsumer<>(false);
-
- setupConsumer(consumer);
-
- assertThat(consumer.getOffsetCommitMode()).isEqualTo(OffsetCommitMode.DISABLED);
- }
-
- /**
- * Tests that subscribed partitions didn't change when there's no change on the initial topics.
- * (filterRestoredPartitionsWithDiscovered is active)
- */
- @Test
- public void testSetFilterRestoredParitionsNoChange() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- false);
- }
-
- /**
- * Tests that removed partitions will be removed from subscribed partitions Even if it's still
- * in restored partitions. (filterRestoredPartitionsWithDiscovered is active)
- */
- @Test
- public void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1"}),
- Arrays.asList(new String[] {"kafka_topic_1"}),
- false);
- }
-
- /**
- * Tests that newly added partitions will be added to subscribed partitions.
- * (filterRestoredPartitionsWithDiscovered is active)
- */
- @Test
- public void testSetFilterRestoredParitionsWithAddedTopic() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- false);
- }
-
- /**
- * Tests that subscribed partitions are the same when there's no change on the initial topics.
- * (filterRestoredPartitionsWithDiscovered is disabled)
- */
- @Test
- public void testDisableFilterRestoredParitionsNoChange() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- true);
- }
-
- /**
- * Tests that removed partitions will not be removed from subscribed partitions Even if it's
- * still in restored partitions. (filterRestoredPartitionsWithDiscovered is disabled)
- */
- @Test
- public void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- true);
- }
-
- /**
- * Tests that newly added partitions will be added to subscribed partitions.
- * (filterRestoredPartitionsWithDiscovered is disabled)
- */
- @Test
- public void testDisableFilterRestoredParitionsWithAddedTopic() throws Exception {
- checkFilterRestoredPartitionsWithDisovered(
- Arrays.asList(new String[] {"kafka_topic_1"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}),
- true);
- }
-
- private void checkFilterRestoredPartitionsWithDisovered(
- List<String> restoredKafkaTopics,
- List<String> initKafkaTopics,
- List<String> expectedSubscribedPartitions,
- Boolean disableFiltering)
- throws Exception {
- final AbstractPartitionDiscoverer discoverer =
- new TestPartitionDiscoverer(
- new KafkaTopicsDescriptor(initKafkaTopics, null),
- 0,
- 1,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- initKafkaTopics),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- initKafkaTopics.stream()
- .map(topic -> new KafkaTopicPartition(topic, 0))
- .collect(Collectors.toList())));
- final FlinkKafkaConsumerBase<String> consumer =
- new DummyFlinkKafkaConsumer<>(initKafkaTopics, discoverer);
- if (disableFiltering) {
- consumer.disableFilterRestoredPartitionsWithSubscribedTopics();
- }
-
- final TestingListState<Tuple2<KafkaTopicPartition, Long>> listState =
- new TestingListState<>();
-
- for (int i = 0; i < restoredKafkaTopics.size(); i++) {
- listState.add(
- new Tuple2<>(new KafkaTopicPartition(restoredKafkaTopics.get(i), 0), 12345L));
- }
-
- setupConsumer(consumer, true, listState, true, 0, 1);
-
- Map<KafkaTopicPartition, Long> subscribedPartitionsToStartOffsets =
- consumer.getSubscribedPartitionsToStartOffsets();
-
- assertThat(
- subscribedPartitionsToStartOffsets.keySet().stream()
- .map(partition -> partition.getTopic())
- .collect(Collectors.toSet()))
- .isEqualTo(new HashSet<>(expectedSubscribedPartitions));
- }
-
- @Test
- @SuppressWarnings("unchecked")
- public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception {
-
- // --------------------------------------------------------------------
- // prepare fake states
- // --------------------------------------------------------------------
-
- final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
- state1.put(new KafkaTopicPartition("abc", 13), 16768L);
- state1.put(new KafkaTopicPartition("def", 7), 987654321L);
-
- final HashMap<KafkaTopicPartition, Long> state2 = new HashMap<>();
- state2.put(new KafkaTopicPartition("abc", 13), 16770L);
- state2.put(new KafkaTopicPartition("def", 7), 987654329L);
-
- final HashMap<KafkaTopicPartition, Long> state3 = new HashMap<>();
- state3.put(new KafkaTopicPartition("abc", 13), 16780L);
- state3.put(new KafkaTopicPartition("def", 7), 987654377L);
-
- // --------------------------------------------------------------------
-
- final MockFetcher<String> fetcher = new MockFetcher<>(state1, state2, state3);
-
- final FlinkKafkaConsumerBase<String> consumer =
- new DummyFlinkKafkaConsumer<>(
- fetcher, mock(AbstractPartitionDiscoverer.class), false);
-
- final TestingListState<Serializable> listState = new TestingListState<>();
-
- // setup and run the consumer; wait until the consumer reaches the main fetch loop before
- // continuing test
- setupConsumer(consumer, false, listState, true, 0, 1);
-
- final CheckedThread runThread =
- new CheckedThread() {
- @Override
- public void go() throws Exception {
- consumer.run(new TestSourceContext<>());
- }
- };
- runThread.start();
- fetcher.waitUntilRun();
-
- assertThat(consumer.getPendingOffsetsToCommit()).isEmpty();
-
- // checkpoint 1
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138));
-
- HashMap<KafkaTopicPartition, Long> snapshot1 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot1.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot1).isEqualTo(state1);
- assertThat(consumer.getPendingOffsetsToCommit()).hasSize(1);
- assertThat(consumer.getPendingOffsetsToCommit().get(138L)).isEqualTo(state1);
-
- // checkpoint 2
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140));
-
- HashMap<KafkaTopicPartition, Long> snapshot2 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot2.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot2).isEqualTo(state2);
- assertThat(consumer.getPendingOffsetsToCommit()).hasSize(2);
- assertThat(consumer.getPendingOffsetsToCommit().get(140L)).isEqualTo(state2);
-
- // ack checkpoint 1
- consumer.notifyCheckpointComplete(138L);
- assertThat(consumer.getPendingOffsetsToCommit()).hasSize(1);
- assertThat(consumer.getPendingOffsetsToCommit()).containsKey(140L);
- assertThat(fetcher.getAndClearLastCommittedOffsets()).isEqualTo(state1);
- assertThat(fetcher.getCommitCount()).isEqualTo(1);
-
- // checkpoint 3
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141));
-
- HashMap<KafkaTopicPartition, Long> snapshot3 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot3.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot3).isEqualTo(state3);
- assertThat(consumer.getPendingOffsetsToCommit()).hasSize(2);
- assertThat(consumer.getPendingOffsetsToCommit().get(141L)).isEqualTo(state3);
-
- // ack checkpoint 3, subsumes number 2
- consumer.notifyCheckpointComplete(141L);
- assertThat(consumer.getPendingOffsetsToCommit()).isEmpty();
- assertThat(fetcher.getAndClearLastCommittedOffsets()).isEqualTo(state3);
- assertThat(fetcher.getCommitCount()).isEqualTo(2);
-
- consumer.notifyCheckpointComplete(666); // invalid checkpoint
- assertThat(consumer.getPendingOffsetsToCommit()).isEmpty();
- assertThat(fetcher.getAndClearLastCommittedOffsets()).isNull();
- assertThat(fetcher.getCommitCount()).isEqualTo(2);
-
- consumer.cancel();
- runThread.sync();
- }
-
- @Test
- @SuppressWarnings("unchecked")
- public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception {
- // --------------------------------------------------------------------
- // prepare fake states
- // --------------------------------------------------------------------
-
- final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
- state1.put(new KafkaTopicPartition("abc", 13), 16768L);
- state1.put(new KafkaTopicPartition("def", 7), 987654321L);
-
- final HashMap<KafkaTopicPartition, Long> state2 = new HashMap<>();
- state2.put(new KafkaTopicPartition("abc", 13), 16770L);
- state2.put(new KafkaTopicPartition("def", 7), 987654329L);
-
- final HashMap<KafkaTopicPartition, Long> state3 = new HashMap<>();
- state3.put(new KafkaTopicPartition("abc", 13), 16780L);
- state3.put(new KafkaTopicPartition("def", 7), 987654377L);
-
- // --------------------------------------------------------------------
-
- final MockFetcher<String> fetcher = new MockFetcher<>(state1, state2, state3);
-
- final FlinkKafkaConsumerBase<String> consumer =
- new DummyFlinkKafkaConsumer<>(
- fetcher, mock(AbstractPartitionDiscoverer.class), false);
- consumer.setCommitOffsetsOnCheckpoints(false); // disable offset committing
-
- final TestingListState<Serializable> listState = new TestingListState<>();
-
- // setup and run the consumer; wait until the consumer reaches the main fetch loop before
- // continuing test
- setupConsumer(consumer, false, listState, true, 0, 1);
-
- final CheckedThread runThread =
- new CheckedThread() {
- @Override
- public void go() throws Exception {
- consumer.run(new TestSourceContext<>());
- }
- };
- runThread.start();
- fetcher.waitUntilRun();
-
- assertThat(consumer.getPendingOffsetsToCommit()).isEmpty();
-
- // checkpoint 1
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(138, 138));
-
- HashMap<KafkaTopicPartition, Long> snapshot1 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot1.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot1).isEqualTo(state1);
- assertThat(consumer.getPendingOffsetsToCommit().size())
- .isEqualTo(0); // pending offsets to commit should not be updated
-
- // checkpoint 2
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140));
-
- HashMap<KafkaTopicPartition, Long> snapshot2 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot2.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot2).isEqualTo(state2);
- assertThat(consumer.getPendingOffsetsToCommit().size())
- .isEqualTo(0); // pending offsets to commit should not be updated
-
- // ack checkpoint 1
- consumer.notifyCheckpointComplete(138L);
- assertThat(fetcher.getCommitCount()).isEqualTo(0);
- assertThat(fetcher.getAndClearLastCommittedOffsets())
- .isNull(); // no offsets should be committed
-
- // checkpoint 3
- consumer.snapshotState(new StateSnapshotContextSynchronousImpl(141, 141));
-
- HashMap<KafkaTopicPartition, Long> snapshot3 = new HashMap<>();
-
- for (Serializable serializable : listState.get()) {
- Tuple2<KafkaTopicPartition, Long> kafkaTopicPartitionLongTuple2 =
- (Tuple2<KafkaTopicPartition, Long>) serializable;
- snapshot3.put(kafkaTopicPartitionLongTuple2.f0, kafkaTopicPartitionLongTuple2.f1);
- }
-
- assertThat(snapshot3).isEqualTo(state3);
- assertThat(consumer.getPendingOffsetsToCommit().size())
- .isEqualTo(0); // pending offsets to commit should not be updated
-
- // ack checkpoint 3, subsumes number 2
- consumer.notifyCheckpointComplete(141L);
- assertThat(fetcher.getCommitCount()).isEqualTo(0);
- assertThat(fetcher.getAndClearLastCommittedOffsets())
- .isNull(); // no offsets should be committed
-
- consumer.notifyCheckpointComplete(666); // invalid checkpoint
- assertThat(fetcher.getCommitCount()).isEqualTo(0);
- assertThat(fetcher.getAndClearLastCommittedOffsets())
- .isNull(); // no offsets should be committed
-
- consumer.cancel();
- runThread.sync();
- }
-
- @Test
- public void testClosePartitionDiscovererWhenOpenThrowException() throws Exception {
- final RuntimeException failureCause =
- new RuntimeException(new FlinkException("Test partition discoverer exception"));
- final FailingPartitionDiscoverer failingPartitionDiscoverer =
- new FailingPartitionDiscoverer(failureCause);
-
- final DummyFlinkKafkaConsumer<String> consumer =
- new DummyFlinkKafkaConsumer<>(failingPartitionDiscoverer);
-
- testFailingConsumerLifecycle(consumer, failureCause);
- assertThat(failingPartitionDiscoverer.isClosed())
- .as("partitionDiscoverer should be closed when consumer is closed")
- .isTrue();
- }
-
- @Test
- public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exception {
- final FlinkException failureCause = new FlinkException("Create Kafka fetcher failure.");
-
- final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer();
- final DummyFlinkKafkaConsumer<String> consumer =
- new DummyFlinkKafkaConsumer<>(
- () -> {
- throw failureCause;
- },
- testPartitionDiscoverer,
- 100L);
-
- testFailingConsumerLifecycle(consumer, failureCause);
- assertThat(testPartitionDiscoverer.isClosed())
- .as("partitionDiscoverer should be closed when consumer is closed")
- .isTrue();
- }
-
- @Test
- public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception {
- final FlinkException failureCause = new FlinkException("Run Kafka fetcher failure.");
-
- // in this scenario, the partition discoverer will be concurrently accessed;
- // use the WakeupBeforeCloseTestingPartitionDiscoverer to verify that we always call
- // wakeup() before closing the discoverer
- final WakeupBeforeCloseTestingPartitionDiscoverer testPartitionDiscoverer =
- new WakeupBeforeCloseTestingPartitionDiscoverer();
-
- final AbstractFetcher<String, ?> mock =
- (AbstractFetcher<String, ?>) mock(AbstractFetcher.class);
- doThrow(failureCause).when(mock).runFetchLoop();
-
- final DummyFlinkKafkaConsumer<String> consumer =
- new DummyFlinkKafkaConsumer<>(() -> mock, testPartitionDiscoverer, 100L);
-
- testFailingConsumerLifecycle(consumer, failureCause);
- assertThat(testPartitionDiscoverer.isClosed())
- .as("partitionDiscoverer should be closed when consumer is closed")
- .isTrue();
- }
-
- private void testFailingConsumerLifecycle(
- FlinkKafkaConsumerBase<String> testKafkaConsumer, @Nonnull Exception expectedException)
- throws Exception {
- try {
- setupConsumer(testKafkaConsumer);
- testKafkaConsumer.run(new TestSourceContext<>());
-
- fail(
- "Exception should have been thrown from open / run method of FlinkKafkaConsumerBase.");
- } catch (Exception e) {
- assertThat(
- ExceptionUtils.findThrowable(
- e, throwable -> throwable.equals(expectedException)))
- .isPresent();
- }
- testKafkaConsumer.close();
- }
-
- @Test
- public void testClosePartitionDiscovererWithCancellation() throws Exception {
- final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer();
-
- final TestingFlinkKafkaConsumer<String> consumer =
- new TestingFlinkKafkaConsumer<>(testPartitionDiscoverer, 100L);
-
- testNormalConsumerLifecycle(consumer);
- assertThat(testPartitionDiscoverer.isClosed())
- .as("partitionDiscoverer should be closed when consumer is closed")
- .isTrue();
- }
-
- private void testNormalConsumerLifecycle(FlinkKafkaConsumerBase<String> testKafkaConsumer)
- throws Exception {
- setupConsumer(testKafkaConsumer);
- final CompletableFuture<Void> runFuture =
- CompletableFuture.runAsync(
- ThrowingRunnable.unchecked(
- () -> testKafkaConsumer.run(new TestSourceContext<>())));
- testKafkaConsumer.close();
- runFuture.get();
- }
-
- private void setupConsumer(FlinkKafkaConsumerBase<String> consumer) throws Exception {
- setupConsumer(consumer, false, null, false, 0, 1);
- }
-
- /**
- * Before using an explicit TypeSerializer for the partition state the {@link
- * FlinkKafkaConsumerBase} was creating a serializer using a {@link TypeHint}. Here, we verify
- * that the two methods create compatible serializers.
- */
- @Test
- public void testExplicitStateSerializerCompatibility() throws Exception {
- ExecutionConfig executionConfig = new ExecutionConfig();
-
- Tuple2<KafkaTopicPartition, Long> tuple =
- new Tuple2<>(new KafkaTopicPartition("dummy", 0), 42L);
-
- // This is how the KafkaConsumerBase used to create the TypeSerializer
- TypeInformation<Tuple2<KafkaTopicPartition, Long>> originalTypeHintTypeInfo =
- new TypeHint<Tuple2<KafkaTopicPartition, Long>>() {}.getTypeInfo();
- TypeSerializer<Tuple2<KafkaTopicPartition, Long>> serializerFromTypeHint =
- originalTypeHintTypeInfo.createSerializer(executionConfig);
- byte[] bytes = InstantiationUtil.serializeToByteArray(serializerFromTypeHint, tuple);
-
- // Directly use the Consumer to create the TypeSerializer (using the new method)
- TupleSerializer<Tuple2<KafkaTopicPartition, Long>> kafkaConsumerSerializer =
- FlinkKafkaConsumerBase.createStateSerializer(executionConfig);
- Tuple2<KafkaTopicPartition, Long> actualTuple =
- InstantiationUtil.deserializeFromByteArray(kafkaConsumerSerializer, bytes);
-
- assertThat(actualTuple)
- .as(
- "Explicit Serializer is not compatible with previous method of creating Serializer using TypeHint.")
- .isEqualTo(tuple);
- }
-
- @Test
- public void testScaleUp() throws Exception {
- testRescaling(5, 2, 8, 30);
- }
-
- @Test
- public void testScaleDown() throws Exception {
- testRescaling(5, 10, 2, 100);
- }
-
- /**
- * Tests whether the Kafka consumer behaves correctly when scaling the parallelism up/down,
- * which means that operator state is being reshuffled.
- *
- * <p>This also verifies that a restoring source is always impervious to changes in the list of
- * topics fetched from Kafka.
- */
- @SuppressWarnings("unchecked")
- private void testRescaling(
- final int initialParallelism,
- final int numPartitions,
- final int restoredParallelism,
- final int restoredNumPartitions)
- throws Exception {
-
- Preconditions.checkArgument(
- restoredNumPartitions >= numPartitions,
- "invalid test case for Kafka repartitioning; Kafka only allows increasing partitions.");
-
- List<KafkaTopicPartition> mockFetchedPartitionsOnStartup = new ArrayList<>();
- for (int i = 0; i < numPartitions; i++) {
- mockFetchedPartitionsOnStartup.add(new KafkaTopicPartition("test-topic", i));
- }
-
- DummyFlinkKafkaConsumer<String>[] consumers =
- new DummyFlinkKafkaConsumer[initialParallelism];
-
- AbstractStreamOperatorTestHarness<String>[] testHarnesses =
- new AbstractStreamOperatorTestHarness[initialParallelism];
-
- List<String> testTopics = Collections.singletonList("test-topic");
-
- for (int i = 0; i < initialParallelism; i++) {
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- new KafkaTopicsDescriptor(testTopics, null),
- i,
- initialParallelism,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- testTopics),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockFetchedPartitionsOnStartup));
-
- consumers[i] = new DummyFlinkKafkaConsumer<>(testTopics, partitionDiscoverer);
- testHarnesses[i] = createTestHarness(consumers[i], initialParallelism, i);
-
- // initializeState() is always called, null signals that we didn't restore
- testHarnesses[i].initializeEmptyState();
- testHarnesses[i].open();
- }
-
- Map<KafkaTopicPartition, Long> globalSubscribedPartitions = new HashMap<>();
-
- for (int i = 0; i < initialParallelism; i++) {
- Map<KafkaTopicPartition, Long> subscribedPartitions =
- consumers[i].getSubscribedPartitionsToStartOffsets();
-
- // make sure that no one else is subscribed to these partitions
- for (KafkaTopicPartition partition : subscribedPartitions.keySet()) {
- assertThat(globalSubscribedPartitions).satisfies(matching(not(hasKey(partition))));
- }
- globalSubscribedPartitions.putAll(subscribedPartitions);
- }
-
- assertThat(globalSubscribedPartitions.values()).satisfies(matching(hasSize(numPartitions)));
- assertThat(mockFetchedPartitionsOnStartup)
- .satisfies(matching(everyItem(isIn(globalSubscribedPartitions.keySet()))));
-
- OperatorSubtaskState[] state = new OperatorSubtaskState[initialParallelism];
-
- for (int i = 0; i < initialParallelism; i++) {
- state[i] = testHarnesses[i].snapshot(0, 0);
- }
-
- OperatorSubtaskState mergedState = AbstractStreamOperatorTestHarness.repackageState(state);
-
- // -----------------------------------------------------------------------------------------
- // restore
-
- List<KafkaTopicPartition> mockFetchedPartitionsAfterRestore = new ArrayList<>();
- for (int i = 0; i < restoredNumPartitions; i++) {
- mockFetchedPartitionsAfterRestore.add(new KafkaTopicPartition("test-topic", i));
- }
-
- DummyFlinkKafkaConsumer<String>[] restoredConsumers =
- new DummyFlinkKafkaConsumer[restoredParallelism];
-
- AbstractStreamOperatorTestHarness<String>[] restoredTestHarnesses =
- new AbstractStreamOperatorTestHarness[restoredParallelism];
-
- for (int i = 0; i < restoredParallelism; i++) {
- OperatorSubtaskState initState =
- AbstractStreamOperatorTestHarness.repartitionOperatorState(
- mergedState,
- maxParallelism,
- initialParallelism,
- restoredParallelism,
- i);
-
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- new KafkaTopicsDescriptor(testTopics, null),
- i,
- restoredParallelism,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- testTopics),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockFetchedPartitionsAfterRestore));
-
- restoredConsumers[i] = new DummyFlinkKafkaConsumer<>(testTopics, partitionDiscoverer);
- restoredTestHarnesses[i] =
- createTestHarness(restoredConsumers[i], restoredParallelism, i);
-
- // initializeState() is always called, null signals that we didn't restore
- restoredTestHarnesses[i].initializeState(initState);
- restoredTestHarnesses[i].open();
- }
-
- Map<KafkaTopicPartition, Long> restoredGlobalSubscribedPartitions = new HashMap<>();
-
- for (int i = 0; i < restoredParallelism; i++) {
- Map<KafkaTopicPartition, Long> subscribedPartitions =
- restoredConsumers[i].getSubscribedPartitionsToStartOffsets();
-
- // make sure that no one else is subscribed to these partitions
- for (KafkaTopicPartition partition : subscribedPartitions.keySet()) {
- assertThat(restoredGlobalSubscribedPartitions)
- .satisfies(matching(not(hasKey(partition))));
- }
- restoredGlobalSubscribedPartitions.putAll(subscribedPartitions);
- }
-
- assertThat(restoredGlobalSubscribedPartitions.values())
- .satisfies(matching(hasSize(restoredNumPartitions)));
- assertThat(mockFetchedPartitionsOnStartup)
- .satisfies(matching(everyItem(isIn(restoredGlobalSubscribedPartitions.keySet()))));
- }
-
- @Test
- public void testOpen() throws Exception {
- MockDeserializationSchema<Object> deserializationSchema = new MockDeserializationSchema<>();
-
- AbstractStreamOperatorTestHarness<Object> testHarness =
- createTestHarness(
- new DummyFlinkKafkaConsumer<>(
- new KafkaDeserializationSchemaWrapper<>(deserializationSchema)),
- 1,
- 0);
-
- testHarness.open();
- assertThat(deserializationSchema.isOpenCalled()).as("Open method was not called").isTrue();
- }
-
- @Test
- public void testOpenWithRestoreState() throws Exception {
- MockDeserializationSchema<String> deserializationSchema = new MockDeserializationSchema<>();
- final FlinkKafkaConsumerBase<String> consumer =
- new DummyFlinkKafkaConsumer<>(
- new KafkaDeserializationSchemaWrapper<>(deserializationSchema));
-
- final TestingListState<Tuple2<KafkaTopicPartition, Long>> restoredListState =
- new TestingListState<>();
- setupConsumer(consumer, true, restoredListState, true, 0, 1);
-
- assertThat(deserializationSchema.isOpenCalled())
- .as("DeserializationSchema's open method was not invoked")
- .isTrue();
- }
-
- // ------------------------------------------------------------------------
-
- private static <T> AbstractStreamOperatorTestHarness<T> createTestHarness(
- SourceFunction<T> source, int numSubtasks, int subtaskIndex) throws Exception {
-
- AbstractStreamOperatorTestHarness<T> testHarness =
- new AbstractStreamOperatorTestHarness<>(
- new StreamSource<>(source), maxParallelism, numSubtasks, subtaskIndex);
-
- testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
-
- return testHarness;
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * A dummy partition discoverer that always throws an exception from discoverPartitions()
- * method.
- */
- private static class FailingPartitionDiscoverer extends AbstractPartitionDiscoverer {
-
- private volatile boolean closed = false;
-
- private final RuntimeException failureCause;
-
- public FailingPartitionDiscoverer(RuntimeException failureCause) {
- super(new KafkaTopicsDescriptor(Arrays.asList("foo"), null), 0, 1);
- this.failureCause = failureCause;
- }
-
- @Override
- protected void initializeConnections() throws Exception {
- closed = false;
- }
-
- @Override
- protected void wakeupConnections() {}
-
- @Override
- protected void closeConnections() throws Exception {
- closed = true;
- }
-
- @Override
- protected List<String> getAllTopics() throws WakeupException {
- return null;
- }
-
- @Override
- protected List<KafkaTopicPartition> getAllPartitionsForTopics(List<String> topics)
- throws WakeupException {
- return null;
- }
-
- @Override
- public List<KafkaTopicPartition> discoverPartitions()
- throws WakeupException, ClosedException {
- throw failureCause;
- }
-
- public boolean isClosed() {
- return closed;
- }
- }
-
- private static class WakeupBeforeCloseTestingPartitionDiscoverer
- extends DummyPartitionDiscoverer {
- @Override
- protected void closeConnections() {
- if (!isWakedUp()) {
- fail("Partition discoverer should have been waked up first before closing.");
- }
-
- super.closeConnections();
- }
- }
-
- private static class DummyPartitionDiscoverer extends AbstractPartitionDiscoverer {
-
- private final List<String> allTopics;
- private final List<KafkaTopicPartition> allPartitions;
- private volatile boolean closed = false;
- private volatile boolean wakedUp = false;
-
- private DummyPartitionDiscoverer() {
- super(new KafkaTopicsDescriptor(Collections.singletonList("foo"), null), 0, 1);
- this.allTopics = Collections.singletonList("foo");
- this.allPartitions = Collections.singletonList(new KafkaTopicPartition("foo", 0));
- }
-
- @Override
- protected void initializeConnections() {
- // noop
- }
-
- @Override
- protected void wakeupConnections() {
- wakedUp = true;
- }
-
- @Override
- protected void closeConnections() {
- closed = true;
- }
-
- @Override
- protected List<String> getAllTopics() throws WakeupException {
- checkState();
-
- return allTopics;
- }
-
- @Override
- protected List<KafkaTopicPartition> getAllPartitionsForTopics(List<String> topics)
- throws WakeupException {
- checkState();
- return allPartitions;
- }
-
- private void checkState() throws WakeupException {
- if (wakedUp || closed) {
- throw new WakeupException();
- }
- }
-
- boolean isClosed() {
- return closed;
- }
-
- public boolean isWakedUp() {
- return wakedUp;
- }
- }
-
- private static class TestingFetcher<T, KPH> extends AbstractFetcher<T, KPH> {
-
- private volatile boolean isRunning = true;
-
- protected TestingFetcher(
- SourceFunction.SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> seedPartitionsWithInitialOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval,
- ClassLoader userCodeClassLoader,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- super(
- sourceContext,
- seedPartitionsWithInitialOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- userCodeClassLoader,
- consumerMetricGroup,
- useMetrics);
- }
-
- @Override
- public void runFetchLoop() throws Exception {
- while (isRunning) {
- Thread.sleep(10L);
- }
- }
-
- @Override
- public void cancel() {
- isRunning = false;
- }
-
- @Override
- protected void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback commitCallback)
- throws Exception {}
-
- @Override
- protected KPH createKafkaPartitionHandle(KafkaTopicPartition partition) {
- return null;
- }
- }
-
- /**
- * An instantiable dummy {@link FlinkKafkaConsumerBase} that supports injecting mocks for {@link
- * FlinkKafkaConsumerBase#kafkaFetcher}, {@link FlinkKafkaConsumerBase#partitionDiscoverer}, and
- * {@link FlinkKafkaConsumerBase#getIsAutoCommitEnabled()}.
- */
- private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
- private static final long serialVersionUID = 1L;
-
- private SupplierWithException<AbstractFetcher<T, ?>, Exception> testFetcherSupplier;
- private AbstractPartitionDiscoverer testPartitionDiscoverer;
- private boolean isAutoCommitEnabled;
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer() {
- this(false);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(boolean isAutoCommitEnabled) {
- this(
- mock(AbstractFetcher.class),
- mock(AbstractPartitionDiscoverer.class),
- isAutoCommitEnabled);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(AbstractPartitionDiscoverer abstractPartitionDiscoverer) {
- this(mock(AbstractFetcher.class), abstractPartitionDiscoverer, false);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(KafkaDeserializationSchema<T> kafkaDeserializationSchema) {
- this(
- () -> mock(AbstractFetcher.class),
- mock(AbstractPartitionDiscoverer.class),
- false,
- PARTITION_DISCOVERY_DISABLED,
- Collections.singletonList("dummy-topic"),
- kafkaDeserializationSchema);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- List<String> topics, AbstractPartitionDiscoverer abstractPartitionDiscoverer) {
- this(
- () -> mock(AbstractFetcher.class),
- abstractPartitionDiscoverer,
- false,
- PARTITION_DISCOVERY_DISABLED,
- topics,
- (KeyedDeserializationSchema<T>) mock(KeyedDeserializationSchema.class));
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- SupplierWithException<AbstractFetcher<T, ?>, Exception> abstractFetcherSupplier,
- AbstractPartitionDiscoverer abstractPartitionDiscoverer,
- long discoveryIntervalMillis) {
- this(
- abstractFetcherSupplier,
- abstractPartitionDiscoverer,
- false,
- discoveryIntervalMillis);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- AbstractFetcher<T, ?> testFetcher,
- AbstractPartitionDiscoverer testPartitionDiscoverer,
- boolean isAutoCommitEnabled) {
- this(
- testFetcher,
- testPartitionDiscoverer,
- isAutoCommitEnabled,
- PARTITION_DISCOVERY_DISABLED);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- AbstractFetcher<T, ?> testFetcher,
- AbstractPartitionDiscoverer testPartitionDiscoverer,
- boolean isAutoCommitEnabled,
- long discoveryIntervalMillis) {
- this(
- () -> testFetcher,
- testPartitionDiscoverer,
- isAutoCommitEnabled,
- discoveryIntervalMillis);
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- SupplierWithException<AbstractFetcher<T, ?>, Exception> testFetcherSupplier,
- AbstractPartitionDiscoverer testPartitionDiscoverer,
- boolean isAutoCommitEnabled,
- long discoveryIntervalMillis) {
- this(
- testFetcherSupplier,
- testPartitionDiscoverer,
- isAutoCommitEnabled,
- discoveryIntervalMillis,
- Collections.singletonList("dummy-topic"),
- (KeyedDeserializationSchema<T>) mock(KeyedDeserializationSchema.class));
- }
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaConsumer(
- SupplierWithException<AbstractFetcher<T, ?>, Exception> testFetcherSupplier,
- AbstractPartitionDiscoverer testPartitionDiscoverer,
- boolean isAutoCommitEnabled,
- long discoveryIntervalMillis,
- List<String> topics,
- KafkaDeserializationSchema<T> mock) {
-
- super(topics, null, mock, discoveryIntervalMillis, false);
-
- this.testFetcherSupplier = testFetcherSupplier;
- this.testPartitionDiscoverer = testPartitionDiscoverer;
- this.isAutoCommitEnabled = isAutoCommitEnabled;
- }
-
- @Override
- protected AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- return testFetcherSupplier.get();
- }
-
- @Override
- protected AbstractPartitionDiscoverer createPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks) {
- return this.testPartitionDiscoverer;
- }
-
- @Override
- protected boolean getIsAutoCommitEnabled() {
- return isAutoCommitEnabled;
- }
-
- @Override
- protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
- Collection<KafkaTopicPartition> partitions, long timestamp) {
- throw new UnsupportedOperationException();
- }
- }
-
- private static class TestingFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-
- private static final long serialVersionUID = 935384661907656996L;
-
- private final AbstractPartitionDiscoverer partitionDiscoverer;
-
- TestingFlinkKafkaConsumer(
- final AbstractPartitionDiscoverer partitionDiscoverer,
- long discoveryIntervalMillis) {
- super(
- Collections.singletonList("dummy-topic"),
- null,
- (KafkaDeserializationSchema<T>) mock(KafkaDeserializationSchema.class),
- discoveryIntervalMillis,
- false);
- this.partitionDiscoverer = partitionDiscoverer;
- }
-
- @Override
- protected AbstractFetcher<T, ?> createFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- StreamingRuntimeContext runtimeContext,
- OffsetCommitMode offsetCommitMode,
- MetricGroup consumerMetricGroup,
- boolean useMetrics)
- throws Exception {
- return new TestingFetcher<T, String>(
- sourceContext,
- thisSubtaskPartitionsWithStartOffsets,
- watermarkStrategy,
- runtimeContext.getProcessingTimeService(),
- 0L,
- getClass().getClassLoader(),
- consumerMetricGroup,
- useMetrics);
- }
-
- @Override
- protected AbstractPartitionDiscoverer createPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks) {
- return partitionDiscoverer;
- }
-
- @Override
- protected boolean getIsAutoCommitEnabled() {
- return false;
- }
-
- @Override
- protected Map<KafkaTopicPartition, Long> fetchOffsetsWithTimestamp(
- Collection<KafkaTopicPartition> partitions, long timestamp) {
- throw new UnsupportedOperationException("fetchOffsetsWithTimestamp is not supported");
- }
- }
-
- private static final class TestingListState<T> implements ListState<T> {
-
- private final List<T> list = new ArrayList<>();
- private boolean clearCalled = false;
-
- @Override
- public void clear() {
- list.clear();
- clearCalled = true;
- }
-
- @Override
- public Iterable<T> get() throws Exception {
- return list;
- }
-
- @Override
- public void add(T value) throws Exception {
- Preconditions.checkNotNull(value, "You cannot add null to a ListState.");
- list.add(value);
- }
-
- public List<T> getList() {
- return list;
- }
-
- boolean isClearCalled() {
- return clearCalled;
- }
-
- @Override
- public void update(List<T> values) throws Exception {
- clear();
-
- addAll(values);
- }
-
- @Override
- public void addAll(List<T> values) throws Exception {
- if (values != null) {
- values.forEach(
- v -> Preconditions.checkNotNull(v, "You cannot add null to a ListState."));
-
- list.addAll(values);
- }
- }
- }
-
- @SuppressWarnings("unchecked")
- private static <T, S> void setupConsumer(
- FlinkKafkaConsumerBase<T> consumer,
- boolean isRestored,
- ListState<S> restoredListState,
- boolean isCheckpointingEnabled,
- int subtaskIndex,
- int totalNumSubtasks)
- throws Exception {
-
- // run setup procedure in operator life cycle
- consumer.setRuntimeContext(
- new MockStreamingRuntimeContext(
- isCheckpointingEnabled, totalNumSubtasks, subtaskIndex));
- consumer.initializeState(
- new MockFunctionInitializationContext(
- isRestored, new MockOperatorStateStore(restoredListState)));
- consumer.open(new Configuration());
- }
-
- private static class MockFetcher<T> extends AbstractFetcher<T, Object> {
-
- private final OneShotLatch runLatch = new OneShotLatch();
- private final OneShotLatch stopLatch = new OneShotLatch();
-
- private final ArrayDeque<HashMap<KafkaTopicPartition, Long>> stateSnapshotsToReturn =
- new ArrayDeque<>();
-
- private Map<KafkaTopicPartition, Long> lastCommittedOffsets;
- private int commitCount = 0;
-
- @SafeVarargs
- private MockFetcher(HashMap<KafkaTopicPartition, Long>... stateSnapshotsToReturn)
- throws Exception {
- super(
- new TestSourceContext<>(),
- new HashMap<>(),
- null /* watermark strategy */,
- new TestProcessingTimeService(),
- 0,
- MockFetcher.class.getClassLoader(),
- new UnregisteredMetricsGroup(),
- false);
-
- this.stateSnapshotsToReturn.addAll(Arrays.asList(stateSnapshotsToReturn));
- }
-
- @Override
- protected void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback commitCallback)
- throws Exception {
- this.lastCommittedOffsets = offsets;
- this.commitCount++;
- commitCallback.onSuccess();
- }
-
- @Override
- public void runFetchLoop() throws Exception {
- runLatch.trigger();
- stopLatch.await();
- }
-
- @Override
- public HashMap<KafkaTopicPartition, Long> snapshotCurrentState() {
- checkState(!stateSnapshotsToReturn.isEmpty());
- return stateSnapshotsToReturn.poll();
- }
-
- @Override
- protected Object createKafkaPartitionHandle(KafkaTopicPartition partition) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void cancel() {
- stopLatch.trigger();
- }
-
- private void waitUntilRun() throws InterruptedException {
- runLatch.await();
- }
-
- private Map<KafkaTopicPartition, Long> getAndClearLastCommittedOffsets() {
- Map<KafkaTopicPartition, Long> offsets = this.lastCommittedOffsets;
- this.lastCommittedOffsets = null;
- return offsets;
- }
-
- private int getCommitCount() {
- return commitCount;
- }
- }
-
- private static class MockOperatorStateStore implements OperatorStateStore {
-
- private final ListState<?> mockRestoredUnionListState;
-
- private MockOperatorStateStore(ListState<?> restoredUnionListState) {
- this.mockRestoredUnionListState = restoredUnionListState;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <S> ListState<S> getUnionListState(ListStateDescriptor<S> stateDescriptor)
- throws Exception {
- return (ListState<S>) mockRestoredUnionListState;
- }
-
- @Override
- public <K, V> BroadcastState<K, V> getBroadcastState(
- MapStateDescriptor<K, V> stateDescriptor) throws Exception {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public <S> ListState<S> getListState(ListStateDescriptor<S> stateDescriptor)
- throws Exception {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Set<String> getRegisteredStateNames() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Set<String> getRegisteredBroadcastStateNames() {
- throw new UnsupportedOperationException();
- }
- }
-
- private static class MockFunctionInitializationContext
- implements FunctionInitializationContext {
-
- private final boolean isRestored;
- private final OperatorStateStore operatorStateStore;
-
- private MockFunctionInitializationContext(
- boolean isRestored, OperatorStateStore operatorStateStore) {
- this.isRestored = isRestored;
- this.operatorStateStore = operatorStateStore;
- }
-
- @Override
- public boolean isRestored() {
- return isRestored;
- }
-
- @Override
- public OptionalLong getRestoredCheckpointId() {
- return isRestored ? OptionalLong.of(1L) : OptionalLong.empty();
- }
-
- @Override
- public OperatorStateStore getOperatorStateStore() {
- return operatorStateStore;
- }
-
- @Override
- public KeyedStateStore getKeyedStateStore() {
- throw new UnsupportedOperationException();
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java
deleted file mode 100644
index 2f39337..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * 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.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.time.Duration;
-import java.util.Collections;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for our own {@link FlinkKafkaInternalProducer}. */
-@SuppressWarnings("serial")
-public class FlinkKafkaInternalProducerITCase extends KafkaTestBase {
- protected String transactionalId;
- protected Properties extraProperties;
- private volatile Exception exceptionInCallback;
-
- @BeforeClass
- public static void prepare() throws Exception {
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Starting KafkaTestBase ");
- LOG.info("-------------------------------------------------------------------------");
-
- Properties serverProperties = new Properties();
- serverProperties.put("transaction.state.log.num.partitions", Integer.toString(1));
- serverProperties.put("auto.leader.rebalance.enable", Boolean.toString(false));
- startClusters(
- KafkaTestEnvironment.createConfig()
- .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS)
- .setSecureMode(false)
- .setHideKafkaBehindProxy(true)
- .setKafkaServerProperties(serverProperties));
- }
-
- @Before
- public void before() {
- transactionalId = UUID.randomUUID().toString();
- extraProperties = new Properties();
- extraProperties.putAll(standardProps);
- extraProperties.put("transactional.id", transactionalId);
- extraProperties.put(
- "key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- extraProperties.put(
- "value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- extraProperties.put(
- "key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
- extraProperties.put(
- "value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
- extraProperties.put("isolation.level", "read_committed");
- }
-
- @Test(timeout = 60000L)
- public void testHappyPath() throws Exception {
- String topicName = "flink-kafka-producer-happy-path";
-
- Producer<String, String> kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties);
- try {
- kafkaProducer.initTransactions();
- kafkaProducer.beginTransaction();
- kafkaProducer.send(
- new ProducerRecord<>(topicName, "42", "42"), new ErrorCheckingCallback());
- kafkaProducer.commitTransaction();
- } finally {
- kafkaProducer.close(Duration.ofSeconds(5));
- }
- assertThat(exceptionInCallback)
- .as("The message should have been successfully sent")
- .isNull();
- assertRecord(topicName, "42", "42");
- deleteTestTopic(topicName);
- }
-
- @Test(timeout = 30000L)
- public void testResumeTransaction() throws Exception {
- String topicName = "flink-kafka-producer-resume-transaction";
- FlinkKafkaInternalProducer<String, String> kafkaProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- try {
- kafkaProducer.initTransactions();
- kafkaProducer.beginTransaction();
- kafkaProducer.send(
- new ProducerRecord<>(topicName, "42", "42"), new ErrorCheckingCallback());
- kafkaProducer.flush();
- assertThat(exceptionInCallback)
- .as("The message should have been successfully sent")
- .isNull();
- long producerId = kafkaProducer.getProducerId();
- short epoch = kafkaProducer.getEpoch();
-
- FlinkKafkaInternalProducer<String, String> resumeProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- try {
- resumeProducer.resumeTransaction(producerId, epoch);
- resumeProducer.commitTransaction();
- } finally {
- resumeProducer.close(Duration.ofSeconds(5));
- }
-
- assertRecord(topicName, "42", "42");
-
- // this shouldn't throw - in case of network split, old producer might attempt to commit
- // it's transaction
- kafkaProducer.commitTransaction();
-
- // this shouldn't fail also, for same reason as above
- resumeProducer = new FlinkKafkaInternalProducer<>(extraProperties);
- try {
- resumeProducer.resumeTransaction(producerId, epoch);
- resumeProducer.commitTransaction();
- } finally {
- resumeProducer.close(Duration.ofSeconds(5));
- }
- } finally {
- kafkaProducer.close(Duration.ofSeconds(5));
- }
- deleteTestTopic(topicName);
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testPartitionsForAfterClosed() {
- FlinkKafkaInternalProducer<String, String> kafkaProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- kafkaProducer.close(Duration.ofSeconds(5));
- kafkaProducer.partitionsFor("Topic");
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testInitTransactionsAfterClosed() {
- FlinkKafkaInternalProducer<String, String> kafkaProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- kafkaProducer.close(Duration.ofSeconds(5));
- kafkaProducer.initTransactions();
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testBeginTransactionAfterClosed() {
- FlinkKafkaInternalProducer<String, String> kafkaProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- kafkaProducer.initTransactions();
- kafkaProducer.close(Duration.ofSeconds(5));
- kafkaProducer.beginTransaction();
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testCommitTransactionAfterClosed() {
- String topicName = "testCommitTransactionAfterClosed";
- FlinkKafkaInternalProducer<String, String> kafkaProducer = getClosedProducer(topicName);
- kafkaProducer.commitTransaction();
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testResumeTransactionAfterClosed() {
- String topicName = "testAbortTransactionAfterClosed";
- FlinkKafkaInternalProducer<String, String> kafkaProducer = getClosedProducer(topicName);
- kafkaProducer.resumeTransaction(0L, (short) 1);
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testAbortTransactionAfterClosed() {
- String topicName = "testAbortTransactionAfterClosed";
- FlinkKafkaInternalProducer<String, String> kafkaProducer = getClosedProducer(topicName);
- kafkaProducer.abortTransaction();
- kafkaProducer.resumeTransaction(0L, (short) 1);
- }
-
- @Test(timeout = 30000L, expected = IllegalStateException.class)
- public void testFlushAfterClosed() {
- String topicName = "testCommitTransactionAfterClosed";
- FlinkKafkaInternalProducer<String, String> kafkaProducer = getClosedProducer(topicName);
- kafkaProducer.flush();
- }
-
- @Test(timeout = 30000L)
- public void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception {
- String topic = "flink-kafka-producer-txn-coordinator-changed-" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
- Producer<String, String> kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties);
- try {
- kafkaProducer.initTransactions();
- kafkaProducer.beginTransaction();
- restartBroker(kafkaServer.getLeaderToShutDown("__transaction_state"));
- kafkaProducer.flush();
- kafkaProducer.commitTransaction();
- } finally {
- kafkaProducer.close(Duration.ofSeconds(5));
- }
- deleteTestTopic(topic);
- }
-
- private FlinkKafkaInternalProducer<String, String> getClosedProducer(String topicName) {
- FlinkKafkaInternalProducer<String, String> kafkaProducer =
- new FlinkKafkaInternalProducer<>(extraProperties);
- kafkaProducer.initTransactions();
- kafkaProducer.beginTransaction();
- kafkaProducer.send(
- new ProducerRecord<>(topicName, "42", "42"), new ErrorCheckingCallback());
- kafkaProducer.close(Duration.ofSeconds(5));
- assertThat(exceptionInCallback)
- .as("The message should have been successfully sent")
- .isNull();
- return kafkaProducer;
- }
-
- private void assertRecord(String topicName, String expectedKey, String expectedValue) {
- try (KafkaConsumer<String, String> kafkaConsumer = new KafkaConsumer<>(extraProperties)) {
- kafkaConsumer.subscribe(Collections.singletonList(topicName));
- ConsumerRecords<String, String> records = ConsumerRecords.empty();
- while (records.isEmpty()) {
- records = kafkaConsumer.poll(Duration.ofMillis(10000));
- }
-
- ConsumerRecord<String, String> record = Iterables.getOnlyElement(records);
- assertThat(record.key()).isEqualTo(expectedKey);
- assertThat(record.value()).isEqualTo(expectedValue);
- }
- }
-
- private void restartBroker(int brokerId) throws Exception {
- kafkaServer.stopBroker(brokerId);
- kafkaServer.restartBroker(brokerId);
- }
-
- private class ErrorCheckingCallback implements Callback {
- @Override
- public void onCompletion(RecordMetadata metadata, Exception exception) {
- exceptionInCallback = exception;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
deleted file mode 100644
index 4274fcf..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
+++ /dev/null
@@ -1,482 +0,0 @@
-/*
- * 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.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.testutils.CheckedThread;
-import org.apache.flink.core.testutils.MultiShotLatch;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-
-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.PartitionInfo;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/** Tests for the {@link FlinkKafkaProducerBase}. */
-public class FlinkKafkaProducerBaseTest {
-
- /** Tests that the constructor eagerly checks bootstrap servers are set in config. */
- @Test(expected = IllegalArgumentException.class)
- public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception {
- // no bootstrap servers set in props
- Properties props = new Properties();
- // should throw IllegalArgumentException
- new DummyFlinkKafkaProducer<>(
- props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null);
- }
-
- /**
- * Tests that constructor defaults to key value serializers in config to byte array
- * deserializers if not set.
- */
- @Test
- public void testKeyValueDeserializersSetIfMissing() throws Exception {
- Properties props = new Properties();
- props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345");
- // should set missing key value deserializers
- new DummyFlinkKafkaProducer<>(
- props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null);
-
- assertThat(props)
- .containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)
- .containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
- assertThat(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG))
- .isEqualTo(ByteArraySerializer.class.getName());
- assertThat(props.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG))
- .isEqualTo(ByteArraySerializer.class.getName());
- }
-
- /** Tests that partitions list is determinate and correctly provided to custom partitioner. */
- @SuppressWarnings("unchecked")
- @Test
- public void testPartitionerInvokedWithDeterminatePartitionList() throws Exception {
- FlinkKafkaPartitioner<String> mockPartitioner = mock(FlinkKafkaPartitioner.class);
-
- RuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class);
- when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0);
- when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1);
-
- // out-of-order list of 4 partitions
- List<PartitionInfo> mockPartitionsList = new ArrayList<>(4);
- mockPartitionsList.add(
- new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 3, null, null, null));
- mockPartitionsList.add(
- new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 1, null, null, null));
- mockPartitionsList.add(
- new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 0, null, null, null));
- mockPartitionsList.add(
- new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 2, null, null, null));
-
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- mockPartitioner);
- producer.setRuntimeContext(mockRuntimeContext);
-
- final KafkaProducer mockProducer = producer.getMockKafkaProducer();
- when(mockProducer.partitionsFor(anyString())).thenReturn(mockPartitionsList);
- when(mockProducer.metrics()).thenReturn(null);
-
- producer.open(new Configuration());
- verify(mockPartitioner, times(1)).open(0, 1);
-
- producer.invoke("foobar", SinkContextUtil.forTimestamp(0));
- verify(mockPartitioner, times(1))
- .partition(
- "foobar",
- null,
- "foobar".getBytes(),
- DummyFlinkKafkaProducer.DUMMY_TOPIC,
- new int[] {0, 1, 2, 3});
- }
-
- /**
- * Test ensuring that if an invoke call happens right after an async exception is caught, it
- * should be rethrown.
- */
- @Test
- public void testAsyncErrorRethrownOnInvoke() throws Throwable {
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- null);
-
- OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer));
-
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>("msg-1"));
-
- // let the message request return an async exception
- producer.getPendingCallbacks()
- .get(0)
- .onCompletion(null, new Exception("artificial async exception"));
-
- try {
- testHarness.processElement(new StreamRecord<>("msg-2"));
- } catch (Exception e) {
- // the next invoke should rethrow the async exception
- assertThat(e.getCause().getMessage()).contains("artificial async exception");
-
- // test succeeded
- return;
- }
-
- fail("unknown failure");
- }
-
- /**
- * Test ensuring that if a snapshot call happens right after an async exception is caught, it
- * should be rethrown.
- */
- @Test
- public void testAsyncErrorRethrownOnCheckpoint() throws Throwable {
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- null);
-
- OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer));
-
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>("msg-1"));
-
- // let the message request return an async exception
- producer.getPendingCallbacks()
- .get(0)
- .onCompletion(null, new Exception("artificial async exception"));
-
- try {
- testHarness.snapshot(123L, 123L);
- } catch (Exception e) {
- // the next invoke should rethrow the async exception
- assertThat(e.getCause().getMessage()).contains("artificial async exception");
-
- // test succeeded
- return;
- }
-
- fail("unknown failure");
- }
-
- /**
- * Test ensuring that if an async exception is caught for one of the flushed requests on
- * checkpoint, it should be rethrown; we set a timeout because the test will not finish if the
- * logic is broken.
- *
- * <p>Note that this test does not test the snapshot method is blocked correctly when there are
- * pending records. The test for that is covered in testAtLeastOnceProducer.
- */
- @SuppressWarnings("unchecked")
- @Test(timeout = 5000)
- public void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable {
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- null);
- producer.setFlushOnCheckpoint(true);
-
- final KafkaProducer<?, ?> mockProducer = producer.getMockKafkaProducer();
-
- final OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer));
-
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>("msg-1"));
- testHarness.processElement(new StreamRecord<>("msg-2"));
- testHarness.processElement(new StreamRecord<>("msg-3"));
-
- verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class));
-
- // only let the first callback succeed for now
- producer.getPendingCallbacks().get(0).onCompletion(null, null);
-
- CheckedThread snapshotThread =
- new CheckedThread() {
- @Override
- public void go() throws Exception {
- // this should block at first, since there are still two pending records
- // that needs to be flushed
- testHarness.snapshot(123L, 123L);
- }
- };
- snapshotThread.start();
-
- // let the 2nd message fail with an async exception
- producer.getPendingCallbacks()
- .get(1)
- .onCompletion(null, new Exception("artificial async failure for 2nd message"));
- producer.getPendingCallbacks().get(2).onCompletion(null, null);
-
- try {
- snapshotThread.sync();
- } catch (Exception e) {
- // the snapshot should have failed with the async exception
- assertThat(e.getCause().getMessage())
- .contains("artificial async failure for 2nd message");
-
- // test succeeded
- return;
- }
-
- fail("unknown failure");
- }
-
- /**
- * Test ensuring that the producer is not dropping buffered records; we set a timeout because
- * the test will not finish if the logic is broken.
- */
- @SuppressWarnings("unchecked")
- @Test(timeout = 10000)
- public void testAtLeastOnceProducer() throws Throwable {
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- null);
- producer.setFlushOnCheckpoint(true);
-
- final KafkaProducer<?, ?> mockProducer = producer.getMockKafkaProducer();
-
- final OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer));
-
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>("msg-1"));
- testHarness.processElement(new StreamRecord<>("msg-2"));
- testHarness.processElement(new StreamRecord<>("msg-3"));
-
- verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class));
- assertThat(producer.getPendingSize()).isEqualTo(3);
-
- // start a thread to perform checkpointing
- CheckedThread snapshotThread =
- new CheckedThread() {
- @Override
- public void go() throws Exception {
- // this should block until all records are flushed;
- // if the snapshot implementation returns before pending records are
- // flushed,
- testHarness.snapshot(123L, 123L);
- }
- };
- snapshotThread.start();
-
- // before proceeding, make sure that flushing has started and that the snapshot is still
- // blocked;
- // this would block forever if the snapshot didn't perform a flush
- producer.waitUntilFlushStarted();
- assertThat(snapshotThread.isAlive())
- .as("Snapshot returned before all records were flushed")
- .isTrue();
-
- // now, complete the callbacks
- producer.getPendingCallbacks().get(0).onCompletion(null, null);
- assertThat(snapshotThread.isAlive())
- .as("Snapshot returned before all records were flushed")
- .isTrue();
- assertThat(producer.getPendingSize()).isEqualTo(2);
-
- producer.getPendingCallbacks().get(1).onCompletion(null, null);
- assertThat(snapshotThread.isAlive())
- .as("Snapshot returned before all records were flushed")
- .isTrue();
- assertThat(producer.getPendingSize()).isEqualTo(1);
-
- producer.getPendingCallbacks().get(2).onCompletion(null, null);
- assertThat(producer.getPendingSize()).isEqualTo(0);
-
- // this would fail with an exception if flushing wasn't completed before the snapshot method
- // returned
- snapshotThread.sync();
-
- testHarness.close();
- }
-
- /**
- * This test is meant to assure that testAtLeastOnceProducer is valid by testing that if
- * flushing is disabled, the snapshot method does indeed finishes without waiting for pending
- * records; we set a timeout because the test will not finish if the logic is broken.
- */
- @SuppressWarnings("unchecked")
- @Test(timeout = 5000)
- public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable {
- final DummyFlinkKafkaProducer<String> producer =
- new DummyFlinkKafkaProducer<>(
- FakeStandardProducerConfig.get(),
- new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()),
- null);
- producer.setFlushOnCheckpoint(false);
-
- final KafkaProducer<?, ?> mockProducer = producer.getMockKafkaProducer();
-
- final OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer));
-
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>("msg"));
-
- // make sure that all callbacks have not been completed
- verify(mockProducer, times(1)).send(any(ProducerRecord.class), any(Callback.class));
-
- // should return even if there are pending records
- testHarness.snapshot(123L, 123L);
-
- testHarness.close();
- }
-
- // ------------------------------------------------------------------------
-
- private static class DummyFlinkKafkaProducer<T> extends FlinkKafkaProducerBase<T> {
- private static final long serialVersionUID = 1L;
-
- private static final String DUMMY_TOPIC = "dummy-topic";
-
- private transient KafkaProducer<?, ?> mockProducer;
- private transient List<Callback> pendingCallbacks;
- private transient MultiShotLatch flushLatch;
- private boolean isFlushed;
-
- @SuppressWarnings("unchecked")
- DummyFlinkKafkaProducer(
- Properties producerConfig,
- KeyedSerializationSchema<T> schema,
- FlinkKafkaPartitioner partitioner) {
-
- super(DUMMY_TOPIC, schema, producerConfig, partitioner);
-
- this.mockProducer = mock(KafkaProducer.class);
- when(mockProducer.send(any(ProducerRecord.class), any(Callback.class)))
- .thenAnswer(
- new Answer<Object>() {
- @Override
- public Object answer(InvocationOnMock invocationOnMock)
- throws Throwable {
- pendingCallbacks.add(invocationOnMock.getArgument(1));
- return null;
- }
- });
-
- this.pendingCallbacks = new ArrayList<>();
- this.flushLatch = new MultiShotLatch();
- }
-
- long getPendingSize() {
- if (flushOnCheckpoint) {
- return numPendingRecords();
- } else {
- // when flushing is disabled, the implementation does not
- // maintain the current number of pending records to reduce
- // the extra locking overhead required to do so
- throw new UnsupportedOperationException(
- "getPendingSize not supported when flushing is disabled");
- }
- }
-
- List<Callback> getPendingCallbacks() {
- return pendingCallbacks;
- }
-
- KafkaProducer<?, ?> getMockKafkaProducer() {
- return mockProducer;
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
- isFlushed = false;
-
- super.snapshotState(ctx);
-
- // if the snapshot implementation doesn't wait until all pending records are flushed, we
- // should fail the test
- if (flushOnCheckpoint && !isFlushed) {
- throw new RuntimeException(
- "Flushing is enabled; snapshots should be blocked until all pending records are flushed");
- }
- }
-
- public void waitUntilFlushStarted() throws Exception {
- flushLatch.await();
- }
-
- @SuppressWarnings("unchecked")
- @Override
- protected <K, V> KafkaProducer<K, V> getKafkaProducer(Properties props) {
- return (KafkaProducer<K, V>) mockProducer;
- }
-
- @Override
- protected void flush() {
- flushLatch.trigger();
-
- // simply wait until the producer's pending records become zero.
- // This relies on the fact that the producer's Callback implementation
- // and pending records tracking logic is implemented correctly, otherwise
- // we will loop forever.
- while (numPendingRecords() > 0) {
- try {
- Thread.sleep(10);
- } catch (InterruptedException e) {
- throw new RuntimeException("Unable to flush producer, task was interrupted");
- }
- }
-
- isFlushed = true;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java
deleted file mode 100644
index 7b345bf..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java
+++ /dev/null
@@ -1,833 +0,0 @@
-/*
- * 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.api.common.ExecutionConfig;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-
-import org.apache.kafka.common.errors.ProducerFencedException;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import static org.apache.flink.util.ExceptionUtils.findThrowable;
-import static org.apache.flink.util.Preconditions.checkState;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.assertj.core.api.Assertions.fail;
-
-/**
- * IT cases for the {@link FlinkKafkaProducer}.
- *
- * <p>Do not run this class in the same junit execution with other tests in your IDE. This may lead
- * leaking threads.
- */
-public class FlinkKafkaProducerITCase extends KafkaTestBase {
-
- protected String transactionalId;
- protected Properties extraProperties;
-
- protected TypeInformationSerializationSchema<Integer> integerSerializationSchema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
- protected KeyedSerializationSchema<Integer> integerKeyedSerializationSchema =
- new KeyedSerializationSchemaWrapper<>(integerSerializationSchema);
-
- @Before
- public void before() {
- transactionalId = UUID.randomUUID().toString();
- extraProperties = new Properties();
- extraProperties.putAll(standardProps);
- extraProperties.put("transactional.id", transactionalId);
- extraProperties.put(
- "key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- extraProperties.put(
- "value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- extraProperties.put(
- "key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
- extraProperties.put(
- "value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
- extraProperties.put("isolation.level", "read_committed");
- }
-
- @Test
- public void resourceCleanUpNone() throws Exception {
- resourceCleanUp(FlinkKafkaProducer.Semantic.NONE);
- }
-
- @Test
- public void resourceCleanUpAtLeastOnce() throws Exception {
- resourceCleanUp(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
- }
-
- /** This tests checks whether there is some resource leak in form of growing threads number. */
- public void resourceCleanUp(FlinkKafkaProducer.Semantic semantic) throws Exception {
- String topic = "flink-kafka-producer-resource-cleanup-" + semantic;
-
- final int allowedEpsilonThreadCountGrow = 50;
-
- Optional<Integer> initialActiveThreads = Optional.empty();
- for (int i = 0; i < allowedEpsilonThreadCountGrow * 2; i++) {
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness1 =
- createTestHarness(topic, 1, 1, 0, semantic)) {
- testHarness1.setup();
- testHarness1.open();
- }
-
- if (initialActiveThreads.isPresent()) {
- assertThat(Thread.activeCount())
- .as("active threads count")
- .isLessThan(initialActiveThreads.get() + allowedEpsilonThreadCountGrow);
- } else {
- initialActiveThreads = Optional.of(Thread.activeCount());
- }
- }
- checkProducerLeak();
- }
-
- /**
- * This test ensures that transactions reusing transactional.ids (after returning to the pool)
- * will not clash with previous transactions using same transactional.ids.
- */
- @Test
- public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exception {
- String topic = "flink-kafka-producer-fail-before-notify";
-
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness1 =
- createTestHarness(topic)) {
- testHarness1.setup();
- testHarness1.open();
- testHarness1.processElement(42, 0);
- OperatorSubtaskState snapshot = testHarness1.snapshot(0, 0);
- testHarness1.processElement(43, 0);
- testHarness1.notifyOfCompletedCheckpoint(0);
- try {
- for (int i = 0; i < FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE; i++) {
- testHarness1.snapshot(i + 1, 0);
- testHarness1.processElement(i, 0);
- }
- throw new IllegalStateException("This should not be reached.");
- } catch (Exception ex) {
- if (!isCausedBy(FlinkKafkaErrorCode.PRODUCERS_POOL_EMPTY, ex)) {
- throw ex;
- }
- }
-
- // Resume transactions before testHarness1 is being closed (in case of failures close()
- // might not be called)
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness2 =
- createTestHarness(topic)) {
- testHarness2.setup();
- // restore from snapshot1, transactions with records 43 and 44 should be aborted
- testHarness2.initializeState(snapshot);
- testHarness2.open();
- }
-
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42));
- deleteTestTopic(topic);
- } catch (Exception ex) {
- // testHarness1 will be fenced off after creating and closing testHarness2
- if (!findThrowable(ex, ProducerFencedException.class).isPresent()) {
- throw ex;
- }
- }
- checkProducerLeak();
- }
-
- /** This test hangs when running it in your IDE. */
- @Test
- @Ignore
- public void testFlinkKafkaProducerFailBeforeNotify() throws Exception {
- String topic = "flink-kafka-producer-fail-before-notify";
-
- final OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic);
-
- testHarness.setup();
- testHarness.open();
- testHarness.processElement(42, 0);
- testHarness.snapshot(0, 1);
- testHarness.processElement(43, 2);
- OperatorSubtaskState snapshot = testHarness.snapshot(1, 3);
-
- int leaderId = kafkaServer.getLeaderToShutDown(topic);
- failBroker(leaderId);
-
- assertThatThrownBy(
- () -> {
- testHarness.processElement(44, 4);
- testHarness.snapshot(2, 5);
- })
- .isInstanceOf(Exception.class);
-
- try {
- testHarness.close();
- } catch (Exception ex) {
- }
-
- kafkaServer.restartBroker(leaderId);
-
- final OneInputStreamOperatorTestHarness<Integer, Object> testHarness2 =
- createTestHarness(topic);
- testHarness2.setup();
- testHarness2.initializeState(snapshot);
- testHarness2.close();
-
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 43));
-
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- /**
- * This tests checks whether FlinkKafkaProducer correctly aborts lingering transactions after a
- * failure. If such transactions were left alone lingering it consumers would be unable to read
- * committed records that were created after this lingering transaction.
- */
- @Test
- public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception {
- String topic = "flink-kafka-producer-fail-before-notify";
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness1 = createTestHarness(topic);
- checkProducerLeak();
- testHarness1.setup();
- testHarness1.open();
- testHarness1.processElement(42, 0);
- testHarness1.snapshot(0, 1);
- testHarness1.processElement(43, 2);
- OperatorSubtaskState snapshot1 = testHarness1.snapshot(1, 3);
-
- testHarness1.processElement(44, 4);
- testHarness1.snapshot(2, 5);
- testHarness1.processElement(45, 6);
-
- // do not close previous testHarness to make sure that closing do not clean up something (in
- // case of failure
- // there might not be any close)
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness2 = createTestHarness(topic);
- testHarness2.setup();
- // restore from snapshot1, transactions with records 44 and 45 should be aborted
- testHarness2.initializeState(snapshot1);
- testHarness2.open();
-
- // write and commit more records, after potentially lingering transactions
- testHarness2.processElement(46, 7);
- testHarness2.snapshot(4, 8);
- testHarness2.processElement(47, 9);
- testHarness2.notifyOfCompletedCheckpoint(4);
-
- // now we should have:
- // - records 42 and 43 in committed transactions
- // - aborted transactions with records 44 and 45
- // - committed transaction with record 46
- // - pending transaction with record 47
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 43, 46));
-
- try {
- testHarness1.close();
- } catch (Exception e) {
- // The only acceptable exception is ProducerFencedException because testHarness2 uses
- // the same
- // transactional ID.
- if (!(e.getCause() instanceof ProducerFencedException)) {
- fail("Received unexpected exception " + e);
- }
- }
- testHarness2.close();
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- @Test
- public void testFailAndRecoverSameCheckpointTwice() throws Exception {
- String topic = "flink-kafka-producer-fail-and-recover-same-checkpoint-twice";
-
- OperatorSubtaskState snapshot1;
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic)) {
- testHarness.setup();
- testHarness.open();
- testHarness.processElement(42, 0);
- testHarness.snapshot(0, 1);
- testHarness.processElement(43, 2);
- snapshot1 = testHarness.snapshot(1, 3);
-
- testHarness.processElement(44, 4);
- }
-
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic)) {
- testHarness.setup();
- // restore from snapshot1, transactions with records 44 and 45 should be aborted
- testHarness.initializeState(snapshot1);
- testHarness.open();
-
- // write and commit more records, after potentially lingering transactions
- testHarness.processElement(44, 7);
- testHarness.snapshot(2, 8);
- testHarness.processElement(45, 9);
- }
-
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic)) {
- testHarness.setup();
- // restore from snapshot1, transactions with records 44 and 45 should be aborted
- testHarness.initializeState(snapshot1);
- testHarness.open();
-
- // write and commit more records, after potentially lingering transactions
- testHarness.processElement(44, 7);
- testHarness.snapshot(3, 8);
- testHarness.processElement(45, 9);
- }
-
- // now we should have:
- // - records 42 and 43 in committed transactions
- // - aborted transactions with records 44 and 45
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 43));
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- /**
- * This tests checks whether FlinkKafkaProducer correctly aborts lingering transactions after a
- * failure, which happened before first checkpoint and was followed up by reducing the
- * parallelism. If such transactions were left alone lingering it consumers would be unable to
- * read committed records that were created after this lingering transaction.
- */
- @Test
- public void testScaleDownBeforeFirstCheckpoint() throws Exception {
- String topic = "scale-down-before-first-checkpoint";
-
- List<AutoCloseable> operatorsToClose = new ArrayList<>();
- int preScaleDownParallelism = Math.max(2, FlinkKafkaProducer.SAFE_SCALE_DOWN_FACTOR);
- for (int subtaskIndex = 0; subtaskIndex < preScaleDownParallelism; subtaskIndex++) {
- OneInputStreamOperatorTestHarness<Integer, Object> preScaleDownOperator =
- createTestHarness(
- topic,
- preScaleDownParallelism,
- preScaleDownParallelism,
- subtaskIndex,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
-
- preScaleDownOperator.setup();
- preScaleDownOperator.open();
- preScaleDownOperator.processElement(subtaskIndex * 2, 0);
- preScaleDownOperator.snapshot(0, 1);
- preScaleDownOperator.processElement(subtaskIndex * 2 + 1, 2);
-
- operatorsToClose.add(preScaleDownOperator);
- }
-
- // do not close previous testHarnesses to make sure that closing do not clean up something
- // (in case of failure
- // there might not be any close)
-
- // After previous failure simulate restarting application with smaller parallelism
- OneInputStreamOperatorTestHarness<Integer, Object> postScaleDownOperator1 =
- createTestHarness(topic, 1, 1, 0, FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
-
- postScaleDownOperator1.setup();
- postScaleDownOperator1.open();
-
- // write and commit more records, after potentially lingering transactions
- postScaleDownOperator1.processElement(46, 7);
- postScaleDownOperator1.snapshot(4, 8);
- postScaleDownOperator1.processElement(47, 9);
- postScaleDownOperator1.notifyOfCompletedCheckpoint(4);
-
- // now we should have:
- // - records 42, 43, 44 and 45 in aborted transactions
- // - committed transaction with record 46
- // - pending transaction with record 47
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(46));
-
- postScaleDownOperator1.close();
- // ignore ProducerFencedExceptions, because postScaleDownOperator1 could reuse transactional
- // ids.
- for (AutoCloseable operatorToClose : operatorsToClose) {
- closeIgnoringProducerFenced(operatorToClose);
- }
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- /**
- * Each instance of FlinkKafkaProducer uses it's own pool of transactional ids. After the
- * restore from checkpoint transactional ids are redistributed across the subtasks. In case of
- * scale down, the surplus transactional ids are dropped. In case of scale up, new one are
- * generated (for the new subtasks). This test make sure that sequence of scaling down and up
- * again works fine. Especially it checks whether the newly generated ids in scaling up do not
- * overlap with ids that were used before scaling down. For example we start with 4 ids and
- * parallelism 4: [1], [2], [3], [4] - one assigned per each subtask we scale down to
- * parallelism 2: [1, 2], [3, 4] - first subtask got id 1 and 2, second got ids 3 and 4 surplus
- * ids are dropped from the pools and we scale up to parallelism 3: [1 or 2], [3 or 4], [???]
- * new subtask have to generate new id(s), but he can not use ids that are potentially in use,
- * so it has to generate new ones that are greater then 4.
- */
- @Test
- public void testScaleUpAfterScalingDown() throws Exception {
- String topic = "scale-up-after-scaling-down";
-
- final int parallelism1 = 4;
- final int parallelism2 = 2;
- final int parallelism3 = 3;
- final int maxParallelism = Math.max(parallelism1, Math.max(parallelism2, parallelism3));
-
- OperatorSubtaskState operatorSubtaskState =
- repartitionAndExecute(
- topic,
- OperatorSubtaskState.builder().build(),
- parallelism1,
- parallelism1,
- maxParallelism,
- IntStream.range(0, parallelism1).boxed().iterator());
-
- operatorSubtaskState =
- repartitionAndExecute(
- topic,
- operatorSubtaskState,
- parallelism1,
- parallelism2,
- maxParallelism,
- IntStream.range(parallelism1, parallelism1 + parallelism2)
- .boxed()
- .iterator());
-
- operatorSubtaskState =
- repartitionAndExecute(
- topic,
- operatorSubtaskState,
- parallelism2,
- parallelism3,
- maxParallelism,
- IntStream.range(
- parallelism1 + parallelism2,
- parallelism1 + parallelism2 + parallelism3)
- .boxed()
- .iterator());
-
- // After each previous repartitionAndExecute call, we are left with some lingering
- // transactions, that would
- // not allow us to read all committed messages from the topic. Thus we initialize operators
- // from
- // OperatorSubtaskState once more, but without any new data. This should terminate all
- // ongoing transactions.
-
- repartitionAndExecute(
- topic,
- operatorSubtaskState,
- parallelism3,
- 1,
- maxParallelism,
- Collections.emptyIterator());
-
- assertExactlyOnceForTopic(
- createProperties(),
- topic,
- IntStream.range(0, parallelism1 + parallelism2 + parallelism3)
- .boxed()
- .collect(Collectors.toList()));
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- private OperatorSubtaskState repartitionAndExecute(
- String topic,
- OperatorSubtaskState inputStates,
- int oldParallelism,
- int newParallelism,
- int maxParallelism,
- Iterator<Integer> inputData)
- throws Exception {
-
- List<OperatorSubtaskState> outputStates = new ArrayList<>();
- List<OneInputStreamOperatorTestHarness<Integer, Object>> testHarnesses = new ArrayList<>();
-
- for (int subtaskIndex = 0; subtaskIndex < newParallelism; subtaskIndex++) {
- OperatorSubtaskState initState =
- AbstractStreamOperatorTestHarness.repartitionOperatorState(
- inputStates,
- maxParallelism,
- oldParallelism,
- newParallelism,
- subtaskIndex);
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(
- topic,
- maxParallelism,
- newParallelism,
- subtaskIndex,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- testHarnesses.add(testHarness);
-
- testHarness.setup();
-
- testHarness.initializeState(initState);
- testHarness.open();
-
- if (inputData.hasNext()) {
- int nextValue = inputData.next();
- testHarness.processElement(nextValue, 0);
- OperatorSubtaskState snapshot = testHarness.snapshot(0, 0);
-
- outputStates.add(snapshot);
- checkState(
- snapshot.getRawOperatorState().isEmpty(), "Unexpected raw operator state");
- checkState(
- snapshot.getManagedKeyedState().isEmpty(),
- "Unexpected managed keyed state");
- checkState(snapshot.getRawKeyedState().isEmpty(), "Unexpected raw keyed state");
-
- for (int i = 1; i < FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE - 1; i++) {
- testHarness.processElement(-nextValue, 0);
- testHarness.snapshot(i, 0);
- }
- }
- }
-
- for (OneInputStreamOperatorTestHarness<Integer, Object> testHarness : testHarnesses) {
- testHarness.close();
- }
-
- return AbstractStreamOperatorTestHarness.repackageState(
- outputStates.toArray(new OperatorSubtaskState[outputStates.size()]));
- }
-
- @Test
- public void testRecoverCommittedTransaction() throws Exception {
- String topic = "flink-kafka-producer-recover-committed-transaction";
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness = createTestHarness(topic);
-
- testHarness.setup();
- testHarness.open(); // producerA - start transaction (txn) 0
- testHarness.processElement(42, 0); // producerA - write 42 in txn 0
- OperatorSubtaskState checkpoint0 =
- testHarness.snapshot(0, 1); // producerA - pre commit txn 0, producerB - start txn 1
- testHarness.processElement(43, 2); // producerB - write 43 in txn 1
- testHarness.notifyOfCompletedCheckpoint(
- 0); // producerA - commit txn 0 and return to the pool
- testHarness.snapshot(1, 3); // producerB - pre txn 1, producerA - start txn 2
- testHarness.processElement(44, 4); // producerA - write 44 in txn 2
- testHarness.close(); // producerA - abort txn 2
-
- testHarness = createTestHarness(topic);
- testHarness.initializeState(
- checkpoint0); // recover state 0 - producerA recover and commit txn 0
- testHarness.close();
-
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42));
-
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- @Test
- public void testRunOutOfProducersInThePool() throws Exception {
- String topic = "flink-kafka-run-out-of-producers";
-
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic)) {
-
- testHarness.setup();
- testHarness.open();
-
- for (int i = 0; i < FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE * 2; i++) {
- testHarness.processElement(i, i * 2);
- testHarness.snapshot(i, i * 2 + 1);
- }
- } catch (Exception ex) {
- if (!ex.getCause().getMessage().startsWith("Too many ongoing")) {
- throw ex;
- }
- }
- deleteTestTopic(topic);
- checkProducerLeak();
- }
-
- @Test
- public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception {
- String topic = "testMigrateFromAtLeastOnceToExactlyOnce";
- testRecoverWithChangeSemantics(
- topic,
- FlinkKafkaProducer.Semantic.AT_LEAST_ONCE,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 43, 44, 45));
- deleteTestTopic(topic);
- }
-
- @Test
- public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception {
- String topic = "testMigrateFromExactlyOnceToAtLeastOnce";
- testRecoverWithChangeSemantics(
- topic,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE,
- FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 43, 45, 46, 47));
- deleteTestTopic(topic);
- }
-
- @Test
- public void testDefaultTransactionalIdPrefix() throws Exception {
- Properties properties = createProperties();
- String topic = "testCustomizeTransactionalIdPrefix";
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- topic,
- integerKeyedSerializationSchema,
- properties,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
-
- final String taskName = "MyTask";
- final OperatorID operatorID = new OperatorID();
-
- String transactionalIdUsed;
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- IntSerializer.INSTANCE,
- taskName,
- operatorID)) {
- testHarness.setup();
- testHarness.open();
- testHarness.processElement(2, 0);
- testHarness.snapshot(0, 1);
-
- transactionalIdUsed = kafkaProducer.getTransactionalId();
- }
-
- deleteTestTopic(topic);
- checkProducerLeak();
-
- assertThat(transactionalIdUsed).isNotNull();
- String expectedTransactionalIdPrefix = taskName + "-" + operatorID.toHexString();
- assertThat(transactionalIdUsed).startsWith(expectedTransactionalIdPrefix);
- }
-
- @Test
- public void testCustomizeTransactionalIdPrefix() throws Exception {
- String transactionalIdPrefix = "my-prefix";
-
- Properties properties = createProperties();
- String topic = "testCustomizeTransactionalIdPrefix";
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- topic,
- integerKeyedSerializationSchema,
- properties,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- kafkaProducer.setTransactionalIdPrefix(transactionalIdPrefix);
-
- String transactionalIdUsed;
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer), IntSerializer.INSTANCE)) {
- testHarness.setup();
- testHarness.open();
- testHarness.processElement(1, 0);
- testHarness.snapshot(0, 1);
-
- transactionalIdUsed = kafkaProducer.getTransactionalId();
- }
-
- deleteTestTopic(topic);
- checkProducerLeak();
-
- assertThat(transactionalIdUsed).isNotNull();
- assertThat(transactionalIdUsed).startsWith(transactionalIdPrefix);
- }
-
- @Test
- public void testRestoreUsingDifferentTransactionalIdPrefix() throws Exception {
- String topic = "testCustomizeTransactionalIdPrefix";
- Properties properties = createProperties();
-
- final String transactionalIdPrefix1 = "my-prefix1";
- FlinkKafkaProducer<Integer> kafkaProducer1 =
- new FlinkKafkaProducer<>(
- topic,
- integerKeyedSerializationSchema,
- properties,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- kafkaProducer1.setTransactionalIdPrefix(transactionalIdPrefix1);
- OperatorSubtaskState snapshot;
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness1 =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer1), IntSerializer.INSTANCE)) {
- testHarness1.setup();
- testHarness1.open();
- testHarness1.processElement(42, 0);
- snapshot = testHarness1.snapshot(0, 1);
-
- testHarness1.processElement(43, 2);
- }
-
- final String transactionalIdPrefix2 = "my-prefix2";
- FlinkKafkaProducer<Integer> kafkaProducer2 =
- new FlinkKafkaProducer<>(
- topic,
- integerKeyedSerializationSchema,
- properties,
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- kafkaProducer2.setTransactionalIdPrefix(transactionalIdPrefix2);
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness2 =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer2), IntSerializer.INSTANCE)) {
- testHarness2.setup();
- // restore from the previous snapshot, transactions with records 43 should be aborted
- testHarness2.initializeState(snapshot);
- testHarness2.open();
-
- testHarness2.processElement(44, 3);
- testHarness2.snapshot(1, 4);
- testHarness2.processElement(45, 5);
- testHarness2.notifyOfCompletedCheckpoint(1);
- testHarness2.processElement(46, 6);
- }
-
- assertExactlyOnceForTopic(createProperties(), topic, Arrays.asList(42, 44));
- checkProducerLeak();
- }
-
- private void testRecoverWithChangeSemantics(
- String topic,
- FlinkKafkaProducer.Semantic fromSemantic,
- FlinkKafkaProducer.Semantic toSemantic)
- throws Exception {
- OperatorSubtaskState producerSnapshot;
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic, fromSemantic)) {
- testHarness.setup();
- testHarness.open();
- testHarness.processElement(42, 0);
- testHarness.snapshot(0, 1);
- testHarness.processElement(43, 2);
- testHarness.notifyOfCompletedCheckpoint(0);
- producerSnapshot = testHarness.snapshot(1, 3);
- testHarness.processElement(44, 4);
- }
-
- try (OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- createTestHarness(topic, toSemantic)) {
- testHarness.setup();
- testHarness.initializeState(producerSnapshot);
- testHarness.open();
- testHarness.processElement(45, 7);
- testHarness.snapshot(2, 8);
- testHarness.processElement(46, 9);
- testHarness.notifyOfCompletedCheckpoint(2);
- testHarness.processElement(47, 9);
- }
- checkProducerLeak();
- }
-
- // -----------------------------------------------------------------------------------------------------------------
-
- // shut down a Kafka broker
- private void failBroker(int brokerId) throws Exception {
- kafkaServer.stopBroker(brokerId);
- }
-
- private void closeIgnoringProducerFenced(AutoCloseable autoCloseable) throws Exception {
- try {
- autoCloseable.close();
- } catch (Exception ex) {
- if (!(ex.getCause() instanceof ProducerFencedException)) {
- throw ex;
- }
- }
- }
-
- private OneInputStreamOperatorTestHarness<Integer, Object> createTestHarness(String topic)
- throws Exception {
- return createTestHarness(topic, FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- }
-
- private OneInputStreamOperatorTestHarness<Integer, Object> createTestHarness(
- String topic, FlinkKafkaProducer.Semantic semantic) throws Exception {
- return createTestHarness(topic, 1, 1, 0, semantic);
- }
-
- private OneInputStreamOperatorTestHarness<Integer, Object> createTestHarness(
- String topic,
- int maxParallelism,
- int parallelism,
- int subtaskIndex,
- FlinkKafkaProducer.Semantic semantic)
- throws Exception {
- Properties properties = createProperties();
-
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- topic, integerKeyedSerializationSchema, properties, semantic);
-
- return new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- maxParallelism,
- parallelism,
- subtaskIndex,
- IntSerializer.INSTANCE,
- new OperatorID(42, 44));
- }
-
- private Properties createProperties() {
- Properties properties = new Properties();
- properties.putAll(standardProps);
- properties.putAll(secureProps);
- properties.put(FlinkKafkaProducer.KEY_DISABLE_METRICS, "true");
- return properties;
- }
-
- private boolean isCausedBy(FlinkKafkaErrorCode expectedErrorCode, Throwable ex) {
- Optional<FlinkKafkaException> cause = findThrowable(ex, FlinkKafkaException.class);
- if (cause.isPresent()) {
- return cause.get().getErrorCode().equals(expectedErrorCode);
- }
- return false;
- }
-
- private void checkProducerLeak() {
- for (Thread t : Thread.getAllStackTraces().keySet()) {
- if (t.getName().contains("kafka-producer-network-thread")) {
- fail("Detected producer leak. Thread name: " + t.getName());
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java
deleted file mode 100644
index 5de210d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.FlinkVersion;
-
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-/**
- * Migration test from FlinkKafkaProducer011 operator. This test depends on the resource generated
- * by removed {@code FlinkKafkaProducer011MigrationTest#writeSnapshot()}.
- *
- * <p>Warning: We need to rename the generated resource based on the file naming pattern specified
- * by the {@link #getOperatorSnapshotPath(FlinkVersion)} method then copy the resource to the path
- * also specified by the {@link #getOperatorSnapshotPath(FlinkVersion)} method.
- */
-public class FlinkKafkaProducerMigrationOperatorTest extends FlinkKafkaProducerMigrationTest {
- @Parameterized.Parameters(name = "Migration Savepoint: {0}")
- public static Collection<FlinkVersion> parameters() {
- return Arrays.asList(
- FlinkVersion.v1_8, FlinkVersion.v1_9, FlinkVersion.v1_10, FlinkVersion.v1_11);
- }
-
- public FlinkKafkaProducerMigrationOperatorTest(FlinkVersion testMigrateVersion) {
- super(testMigrateVersion);
- }
-
- @Override
- public String getOperatorSnapshotPath(FlinkVersion version) {
- return "src/test/resources/kafka-0.11-migration-kafka-producer-flink-"
- + version
- + "-snapshot";
- }
-
- @Override
- public void writeSnapshot(FlinkVersion targetVersion) throws Exception {
- // There is no need to generate snapshots for these tests in newer versions.
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java
deleted file mode 100644
index 53a94b3..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.FlinkVersion;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.test.util.MigrationTest;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Collection;
-import java.util.Properties;
-
-/**
- * Tests for checking whether {@link FlinkKafkaProducer} can restore from snapshots that were done
- * using previous Flink versions' {@link FlinkKafkaProducer}.
- */
-@RunWith(Parameterized.class)
-public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase {
- @Parameterized.Parameters(name = "Migration Savepoint: {0}")
- public static Collection<FlinkVersion> parameters() {
- return FlinkVersion.rangeOf(
- FlinkVersion.v1_8, MigrationTest.getMostRecentlyPublishedVersion());
- }
-
- public FlinkKafkaProducerMigrationTest(FlinkVersion testMigrateVersion) {
- super(testMigrateVersion);
- }
-
- @Override
- protected Properties createProperties() {
- Properties properties = new Properties();
- properties.putAll(standardProps);
- properties.putAll(secureProps);
- properties.put(ProducerConfig.CLIENT_ID_CONFIG, "producer-client-id");
- properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "producer-transaction-id");
- properties.put(FlinkKafkaProducer.KEY_DISABLE_METRICS, "true");
- return properties;
- }
-
- @Override
- protected OneInputStreamOperatorTestHarness<Integer, Object> createTestHarness()
- throws Exception {
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- TOPIC,
- integerKeyedSerializationSchema,
- createProperties(),
- FlinkKafkaProducer.Semantic.EXACTLY_ONCE)
- .ignoreFailuresAfterTransactionTimeout();
-
- return new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- 1,
- 1,
- 0,
- IntSerializer.INSTANCE,
- new OperatorID(1, 1));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerStateSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerStateSerializerTest.java
deleted file mode 100644
index 26d43f1..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerStateSerializerTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.api.common.typeutils.SerializerTestBase;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.TransactionHolder;
-
-import java.util.Collections;
-import java.util.Optional;
-
-/** A test for the {@link TypeSerializer TypeSerializers} used for the Kafka producer state. */
-class FlinkKafkaProducerStateSerializerTest
- extends SerializerTestBase<
- TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>> {
-
- @Override
- protected TypeSerializer<
- TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>>
- createSerializer() {
- return new TwoPhaseCommitSinkFunction.StateSerializer<>(
- new FlinkKafkaProducer.TransactionStateSerializer(),
- new FlinkKafkaProducer.ContextStateSerializer());
- }
-
- @Override
- protected Class<
- TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>>
- getTypeClass() {
- return (Class) TwoPhaseCommitSinkFunction.State.class;
- }
-
- @Override
- protected int getLength() {
- return -1;
- }
-
- @Override
- protected TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>
- [] getTestData() {
- //noinspection unchecked
- return new TwoPhaseCommitSinkFunction.State[] {
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0),
- Collections.emptyList(),
- Optional.empty()),
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 2711),
- Collections.singletonList(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 42)),
- Optional.empty()),
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0),
- Collections.emptyList(),
- Optional.of(
- new FlinkKafkaProducer.KafkaTransactionContext(
- Collections.emptySet()))),
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0),
- Collections.emptyList(),
- Optional.of(
- new FlinkKafkaProducer.KafkaTransactionContext(
- Collections.singleton("hello")))),
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0),
- Collections.singletonList(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0)),
- Optional.of(
- new FlinkKafkaProducer.KafkaTransactionContext(
- Collections.emptySet()))),
- new TwoPhaseCommitSinkFunction.State<
- FlinkKafkaProducer.KafkaTransactionState,
- FlinkKafkaProducer.KafkaTransactionContext>(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0),
- Collections.singletonList(
- new TransactionHolder(
- new FlinkKafkaProducer.KafkaTransactionState(
- "fake", 1L, (short) 42, null),
- 0)),
- Optional.of(
- new FlinkKafkaProducer.KafkaTransactionContext(
- Collections.singleton("hello"))))
- };
- }
-
- @Override
- public void testInstantiate() {
- // this serializer does not support instantiation
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java
deleted file mode 100644
index 6fedcc4..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-
-import java.util.Optional;
-import java.util.Properties;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link FlinkKafkaProducer}. */
-public class FlinkKafkaProducerTest {
- @Test
- public void testOpenSerializationSchemaProducer() throws Exception {
- OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema();
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>("localhost:9092", "test-topic", schema);
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- 1,
- 1,
- 0,
- IntSerializer.INSTANCE,
- new OperatorID(1, 1));
-
- testHarness.open();
-
- assertThat(schema.openCalled).isTrue();
- }
-
- @Test
- public void testOpenKafkaSerializationSchemaProducer() throws Exception {
- OpenTestingKafkaSerializationSchema schema = new OpenTestingKafkaSerializationSchema();
- Properties properties = new Properties();
- properties.put("bootstrap.servers", "localhost:9092");
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- "test-topic",
- schema,
- properties,
- FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- 1,
- 1,
- 0,
- IntSerializer.INSTANCE,
- new OperatorID(1, 1));
-
- testHarness.open();
-
- assertThat(schema.openCalled).isTrue();
- }
-
- @Test
- public void testOpenKafkaCustomPartitioner() throws Exception {
- CustomPartitioner<Integer> partitioner = new CustomPartitioner<>();
- Properties properties = new Properties();
- properties.put("bootstrap.servers", "localhost:9092");
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- "test-topic",
- new OpenTestingSerializationSchema(),
- properties,
- Optional.of(partitioner));
-
- OneInputStreamOperatorTestHarness<Integer, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(
- new StreamSink<>(kafkaProducer),
- 1,
- 1,
- 0,
- IntSerializer.INSTANCE,
- new OperatorID(1, 1));
-
- testHarness.open();
-
- assertThat(partitioner.openCalled).isTrue();
- }
-
- @Test(expected = NullPointerException.class)
- public void testProvidedNullTransactionalIdPrefix() {
- FlinkKafkaProducer<Integer> kafkaProducer =
- new FlinkKafkaProducer<>(
- "localhost:9092", "test-topic", new OpenTestingSerializationSchema());
- kafkaProducer.setTransactionalIdPrefix(null);
- }
-
- private static class CustomPartitioner<T> extends FlinkKafkaPartitioner<T> {
- private boolean openCalled;
-
- @Override
- public void open(int parallelInstanceId, int parallelInstances) {
- super.open(parallelInstanceId, parallelInstances);
- openCalled = true;
- }
-
- @Override
- public int partition(
- T record, byte[] key, byte[] value, String targetTopic, int[] partitions) {
- return 0;
- }
- }
-
- private static class OpenTestingKafkaSerializationSchema
- implements KafkaSerializationSchema<Integer> {
- private boolean openCalled;
-
- @Override
- public void open(SerializationSchema.InitializationContext context) throws Exception {
- openCalled = true;
- }
-
- @Override
- public ProducerRecord<byte[], byte[]> serialize(Integer element, @Nullable Long timestamp) {
- return null;
- }
- }
-
- private static class OpenTestingSerializationSchema implements SerializationSchema<Integer> {
- private boolean openCalled;
-
- @Override
- public void open(SerializationSchema.InitializationContext context) throws Exception {
- openCalled = true;
- }
-
- @Override
- public byte[] serialize(Integer element) {
- return new byte[0];
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
deleted file mode 100644
index ddbcf1c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.testutils.formats.DummyInitializationContext;
-import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema;
-import org.apache.flink.util.jackson.JacksonMapperFactory;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the{@link JSONKeyValueDeserializationSchema}. */
-public class JSONKeyValueDeserializationSchemaTest {
-
- private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper();
-
- @Test
- public void testDeserializeWithoutMetadata() throws Exception {
- ObjectNode initialKey = OBJECT_MAPPER.createObjectNode();
- initialKey.put("index", 4);
- byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey);
-
- ObjectNode initialValue = OBJECT_MAPPER.createObjectNode();
- initialValue.put("word", "world");
- byte[] serializedValue = OBJECT_MAPPER.writeValueAsBytes(initialValue);
-
- JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false);
- schema.open(new DummyInitializationContext());
- ObjectNode deserializedValue =
- schema.deserialize(newConsumerRecord(serializedKey, serializedValue));
-
- assertThat(deserializedValue.get("metadata")).isNull();
- assertThat(deserializedValue.get("key").get("index").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("value").get("word").asText()).isEqualTo("world");
- }
-
- @Test
- public void testDeserializeWithoutKey() throws Exception {
- byte[] serializedKey = null;
-
- ObjectNode initialValue = OBJECT_MAPPER.createObjectNode();
- initialValue.put("word", "world");
- byte[] serializedValue = OBJECT_MAPPER.writeValueAsBytes(initialValue);
-
- JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false);
- schema.open(new DummyInitializationContext());
- ObjectNode deserializedValue =
- schema.deserialize(newConsumerRecord(serializedKey, serializedValue));
-
- assertThat(deserializedValue.get("metadata")).isNull();
- assertThat(deserializedValue.get("key")).isNull();
- assertThat(deserializedValue.get("value").get("word").asText()).isEqualTo("world");
- }
-
- private static ConsumerRecord<byte[], byte[]> newConsumerRecord(
- byte[] serializedKey, byte[] serializedValue) {
- return newConsumerRecord("", 0, 0L, serializedKey, serializedValue);
- }
-
- private static ConsumerRecord<byte[], byte[]> newConsumerRecord(
- String topic,
- int partition,
- long offset,
- byte[] serializedKey,
- byte[] serializedValue) {
-
- return new ConsumerRecord<>(topic, partition, offset, serializedKey, serializedValue);
- }
-
- @Test
- public void testDeserializeWithoutValue() throws Exception {
- ObjectNode initialKey = OBJECT_MAPPER.createObjectNode();
- initialKey.put("index", 4);
- byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey);
-
- byte[] serializedValue = null;
-
- JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(false);
- schema.open(new DummyInitializationContext());
- ObjectNode deserializedValue =
- schema.deserialize(newConsumerRecord(serializedKey, serializedValue));
-
- assertThat(deserializedValue.get("metadata")).isNull();
- assertThat(deserializedValue.get("key").get("index").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("value")).isNull();
- }
-
- @Test
- public void testDeserializeWithMetadata() throws Exception {
- ObjectNode initialKey = OBJECT_MAPPER.createObjectNode();
- initialKey.put("index", 4);
- byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey);
-
- ObjectNode initialValue = OBJECT_MAPPER.createObjectNode();
- initialValue.put("word", "world");
- byte[] serializedValue = OBJECT_MAPPER.writeValueAsBytes(initialValue);
-
- JSONKeyValueDeserializationSchema schema = new JSONKeyValueDeserializationSchema(true);
- schema.open(new DummyInitializationContext());
- final ConsumerRecord<byte[], byte[]> consumerRecord =
- newConsumerRecord("topic#1", 3, 4L, serializedKey, serializedValue);
- ObjectNode deserializedValue = schema.deserialize(consumerRecord);
-
- assertThat(deserializedValue.get("key").get("index").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("value").get("word").asText()).isEqualTo("world");
- assertThat(deserializedValue.get("metadata").get("topic").asText()).isEqualTo("topic#1");
- assertThat(deserializedValue.get("metadata").get("offset").asInt()).isEqualTo(4);
- assertThat(deserializedValue.get("metadata").get("partition").asInt()).isEqualTo(3);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
deleted file mode 100644
index a8596ee..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ /dev/null
@@ -1,2728 +0,0 @@
-/*
- * 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.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.JobStatus;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.client.JobCancellationException;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
-import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
-import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
-import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2FlinkPartitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.testutils.junit.RetryOnException;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
-
-import org.apache.commons.io.output.ByteArrayOutputStream;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.NotLeaderForPartitionException;
-import org.apache.kafka.common.errors.TimeoutException;
-import org.junit.Before;
-
-import javax.annotation.Nullable;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.flink.streaming.connectors.kafka.testutils.ClusterCommunicationUtils.getRunningJobs;
-import static org.apache.flink.streaming.connectors.kafka.testutils.ClusterCommunicationUtils.waitUntilJobIsRunning;
-import static org.apache.flink.streaming.connectors.kafka.testutils.ClusterCommunicationUtils.waitUntilNoJobIsRunning;
-import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult;
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Abstract test base for all Kafka consumer tests. */
-@SuppressWarnings("serial")
-public abstract class KafkaConsumerTestBase extends KafkaTestBaseWithFlink {
- protected final boolean useNewSource;
-
- private ClusterClient<?> client;
-
- protected KafkaConsumerTestBase() {
- this(false);
- }
-
- protected KafkaConsumerTestBase(boolean useNewSource) {
- this.useNewSource = useNewSource;
- }
-
- // ------------------------------------------------------------------------
- // Common Test Preparation
- // ------------------------------------------------------------------------
-
- /**
- * Makes sure that no job is on the JobManager any more from any previous tests that use the
- * same mini cluster. Otherwise, missing slots may happen.
- */
- @Before
- public void setClientAndEnsureNoJobIsLingering() throws Exception {
- client = flink.getClusterClient();
- waitUntilNoJobIsRunning(client);
- }
-
- // ------------------------------------------------------------------------
- // Suite of Tests
- //
- // The tests here are all not activated (by an @Test tag), but need
- // to be invoked from the extending classes. That way, the classes can
- // select which tests to run.
- // ------------------------------------------------------------------------
-
- /**
- * Test that ensures the KafkaConsumer is properly failing if the topic doesn't exist and a
- * wrong broker was specified.
- *
- * @throws Exception
- */
- public void runFailOnNoBrokerTest() throws Exception {
- try {
- Properties properties = new Properties();
-
- StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
- see.setRestartStrategy(RestartStrategies.noRestart());
- see.setParallelism(1);
-
- // use wrong ports for the consumers
- properties.setProperty("bootstrap.servers", "localhost:80");
- properties.setProperty("group.id", "test");
- properties.setProperty("request.timeout.ms", "3000"); // let the test fail fast
- properties.setProperty("socket.timeout.ms", "3000");
- properties.setProperty("session.timeout.ms", "2000");
- properties.setProperty("fetch.max.wait.ms", "2000");
- properties.setProperty("heartbeat.interval.ms", "1000");
- properties.putAll(secureProps);
- DataStream<String> stream =
- getStream(see, "doesntexist", new SimpleStringSchema(), properties);
- stream.print();
- see.execute("No broker test");
- } catch (JobExecutionException jee) {
- final Optional<TimeoutException> optionalTimeoutException =
- ExceptionUtils.findThrowable(jee, TimeoutException.class);
- assertThat(optionalTimeoutException).isPresent();
-
- final TimeoutException timeoutException = optionalTimeoutException.get();
- if (useNewSource) {
- assertThat(timeoutException)
- .hasMessageContaining("Timed out waiting for a node assignment.");
- } else {
- assertThat(timeoutException)
- .hasMessage("Timeout expired while fetching topic metadata");
- }
- }
- }
-
- /**
- * Ensures that the committed offsets to Kafka are the offsets of "the next record to process".
- */
- public void runCommitOffsetsToKafka() throws Exception {
- // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition
- // should be 50)
- final int parallelism = 3;
- final int recordsInEachPartition = 50;
-
- final String topicName =
- writeSequence(
- "testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- env.setParallelism(parallelism);
- env.enableCheckpointing(200);
-
- DataStream<String> stream =
- getStream(env, topicName, new SimpleStringSchema(), standardProps);
- stream.addSink(new DiscardingSink<String>());
-
- final AtomicReference<Throwable> errorRef = new AtomicReference<>();
- final Thread runner =
- new Thread("runner") {
- @Override
- public void run() {
- try {
- env.execute();
- } catch (Throwable t) {
- if (!(t instanceof JobCancellationException)) {
- errorRef.set(t);
- }
- }
- }
- };
- runner.start();
-
- final Long l50 = 50L; // the final committed offset in Kafka should be 50
- final long deadline = 30_000_000_000L + System.nanoTime();
-
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
-
- do {
- Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
- Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
- Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-
- if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
- break;
- }
-
- Thread.sleep(100);
- } while (System.nanoTime() < deadline);
-
- // cancel the job & wait for the job to finish
- client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get();
- runner.join();
-
- final Throwable t = errorRef.get();
- if (t != null) {
- throw new RuntimeException("Job failed with an exception", t);
- }
-
- // final check to see if offsets are correctly in Kafka
- Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
- Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
- Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
- assertThat(o1).isEqualTo(Long.valueOf(50L));
- assertThat(o2).isEqualTo(Long.valueOf(50L));
- assertThat(o3).isEqualTo(Long.valueOf(50L));
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
- }
-
- /**
- * This test ensures that when the consumers retrieve some start offset from kafka (earliest,
- * latest), that this offset is committed to Kafka, even if some partitions are not read.
- *
- * <p>Test: - Create 3 partitions - write 50 messages into each. - Start three consumers with
- * auto.offset.reset='latest' and wait until they committed into Kafka. - Check if the offsets
- * in Kafka are set to 50 for the three partitions
- *
- * <p>See FLINK-3440 as well
- */
- public void runAutoOffsetRetrievalAndCommitToKafka() throws Exception {
- // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition
- // should be 50)
- final int parallelism = 3;
- final int recordsInEachPartition = 50;
-
- final String topicName =
- writeSequence(
- "testAutoOffsetRetrievalAndCommitToKafkaTopic",
- recordsInEachPartition,
- parallelism,
- 1);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- env.setParallelism(parallelism);
- env.enableCheckpointing(200);
-
- Properties readProps = new Properties();
- readProps.putAll(standardProps);
- readProps.setProperty(
- "auto.offset.reset",
- "latest"); // set to reset to latest, so that partitions are initially not read
-
- DataStream<String> stream = getStream(env, topicName, new SimpleStringSchema(), readProps);
- stream.addSink(new DiscardingSink<String>());
-
- final AtomicReference<Throwable> errorRef = new AtomicReference<>();
- final Thread runner =
- new Thread("runner") {
- @Override
- public void run() {
- try {
- env.execute();
- } catch (Throwable t) {
- if (!(t instanceof JobCancellationException)) {
- errorRef.set(t);
- }
- }
- }
- };
- runner.start();
-
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
-
- final Long l50 = 50L; // the final committed offset in Kafka should be 50
- final long deadline = 30_000_000_000L + System.nanoTime();
- do {
- Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
- Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
- Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
-
- if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) {
- break;
- }
-
- Thread.sleep(100);
- } while (System.nanoTime() < deadline);
-
- // cancel the job & wait for the job to finish
- client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get();
- runner.join();
-
- final Throwable t = errorRef.get();
- if (t != null) {
- throw new RuntimeException("Job failed with an exception", t);
- }
-
- // final check to see if offsets are correctly in Kafka
- Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
- Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
- Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
- assertThat(o1).isEqualTo(Long.valueOf(50L));
- assertThat(o2).isEqualTo(Long.valueOf(50L));
- assertThat(o3).isEqualTo(Long.valueOf(50L));
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
- }
-
- /**
- * This test ensures that when explicitly set to start from earliest record, the consumer
- * ignores the "auto.offset.reset" behaviour as well as any committed group offsets in Kafka.
- */
- public void runStartFromEarliestOffsets() throws Exception {
- // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition
- // should be 50)
- final int parallelism = 3;
- final int recordsInEachPartition = 50;
-
- final String topicName =
- writeSequence(
- "testStartFromEarliestOffsetsTopic",
- recordsInEachPartition,
- parallelism,
- 1);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
-
- Properties readProps = new Properties();
- readProps.putAll(standardProps);
- readProps.setProperty("auto.offset.reset", "latest"); // this should be ignored
-
- // the committed offsets should be ignored
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
- kafkaOffsetHandler.setCommittedOffset(topicName, 0, 23);
- kafkaOffsetHandler.setCommittedOffset(topicName, 1, 31);
- kafkaOffsetHandler.setCommittedOffset(topicName, 2, 43);
-
- readSequence(
- env,
- StartupMode.EARLIEST,
- null,
- null,
- readProps,
- parallelism,
- topicName,
- recordsInEachPartition,
- 0);
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
- }
-
- /**
- * This test ensures that when explicitly set to start from latest record, the consumer ignores
- * the "auto.offset.reset" behaviour as well as any committed group offsets in Kafka.
- */
- public void runStartFromLatestOffsets() throws Exception {
- // 50 records written to each of 3 partitions before launching a latest-starting consuming
- // job
- final int parallelism = 3;
- final int recordsInEachPartition = 50;
-
- // each partition will be written an extra 200 records
- final int extraRecordsInEachPartition = 200;
-
- // all already existing data in the topic, before the consuming topology has started, should
- // be ignored
- final String topicName =
- writeSequence(
- "testStartFromLatestOffsetsTopic", recordsInEachPartition, parallelism, 1);
-
- // the committed offsets should be ignored
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
- kafkaOffsetHandler.setCommittedOffset(topicName, 0, 23);
- kafkaOffsetHandler.setCommittedOffset(topicName, 1, 31);
- kafkaOffsetHandler.setCommittedOffset(topicName, 2, 43);
-
- // job names for the topologies for writing and consuming the extra records
- final String consumeExtraRecordsJobName = "Consume Extra Records Job";
- final String writeExtraRecordsJobName = "Write Extra Records Job";
-
- // serialization / deserialization schemas for writing and consuming the extra records
- final TypeInformation<Tuple2<Integer, Integer>> resultType =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
-
- final SerializationSchema<Tuple2<Integer, Integer>> serSchema =
- new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig());
-
- final KafkaDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
- new KafkaDeserializationSchemaWrapper<>(
- new TypeInformationSerializationSchema<>(
- resultType, new ExecutionConfig()));
-
- // setup and run the latest-consuming job
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
-
- final Properties readProps = new Properties();
- readProps.putAll(standardProps);
- readProps.setProperty("auto.offset.reset", "earliest"); // this should be ignored
-
- DataStreamSource<Tuple2<Integer, Integer>> stream;
- if (useNewSource) {
- KafkaSource<Tuple2<Integer, Integer>> source =
- kafkaServer
- .getSourceBuilder(topicName, deserSchema, readProps)
- .setStartingOffsets(OffsetsInitializer.latest())
- .build();
- stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "KafkaSource");
- } else {
- FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> latestReadingConsumer =
- kafkaServer.getConsumer(topicName, deserSchema, readProps);
- latestReadingConsumer.setStartFromLatest();
- stream = env.addSource(latestReadingConsumer);
- }
-
- stream.setParallelism(parallelism)
- .flatMap(
- new FlatMapFunction<Tuple2<Integer, Integer>, Object>() {
- @Override
- public void flatMap(
- Tuple2<Integer, Integer> value, Collector<Object> out)
- throws Exception {
- if (value.f1 - recordsInEachPartition < 0) {
- throw new RuntimeException(
- "test failed; consumed a record that was previously written: "
- + value);
- }
- }
- })
- .setParallelism(1)
- .addSink(new DiscardingSink<>());
-
- JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph());
- final JobID consumeJobId = jobGraph.getJobID();
-
- final AtomicReference<Throwable> error = new AtomicReference<>();
- Thread consumeThread =
- new Thread(
- () -> {
- try {
- submitJobAndWaitForResult(
- client, jobGraph, getClass().getClassLoader());
- } catch (Throwable t) {
- if (!ExceptionUtils.findThrowable(t, JobCancellationException.class)
- .isPresent()) {
- error.set(t);
- }
- }
- });
- consumeThread.start();
-
- // wait until the consuming job has started, to be extra safe
- waitUntilJobIsRunning(client);
-
- // setup the extra records writing job
- final StreamExecutionEnvironment env2 =
- StreamExecutionEnvironment.getExecutionEnvironment();
-
- env2.setParallelism(parallelism);
-
- DataStream<Tuple2<Integer, Integer>> extraRecordsStream =
- env2.addSource(
- new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Integer, Integer>> ctx)
- throws Exception {
- int count =
- recordsInEachPartition; // the extra records should start
- // from the last written value
- int partition = getRuntimeContext().getIndexOfThisSubtask();
-
- while (running
- && count
- < recordsInEachPartition
- + extraRecordsInEachPartition) {
- ctx.collect(new Tuple2<>(partition, count));
- count++;
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
-
- kafkaServer.produceIntoKafka(extraRecordsStream, topicName, serSchema, readProps, null);
-
- try {
- env2.execute(writeExtraRecordsJobName);
- } catch (Exception e) {
- throw new RuntimeException("Writing extra records failed", e);
- }
-
- // cancel the consume job after all extra records are written
- client.cancel(consumeJobId).get();
- consumeThread.join();
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
-
- // check whether the consuming thread threw any test errors;
- // test will fail here if the consume job had incorrectly read any records other than the
- // extra records
- final Throwable consumerError = error.get();
- if (consumerError != null) {
- throw new Exception("Exception in the consuming thread", consumerError);
- }
- }
-
- /**
- * This test ensures that the consumer correctly uses group offsets in Kafka, and defaults to
- * "auto.offset.reset" behaviour when necessary, when explicitly configured to start from group
- * offsets.
- *
- * <p>The partitions and their committed group offsets are setup as: partition 0 --> committed
- * offset 23 partition 1 --> no commit offset partition 2 --> committed offset 43
- *
- * <p>When configured to start from group offsets, each partition should read: partition 0 -->
- * start from offset 23, read to offset 49 (27 records) partition 1 --> default to
- * "auto.offset.reset" (set to earliest), so start from offset 0, read to offset 49 (50 records)
- * partition 2 --> start from offset 43, read to offset 49 (7 records)
- */
- public void runStartFromGroupOffsets() throws Exception {
- // 3 partitions with 50 records each (offsets 0-49)
- final int parallelism = 3;
- final int recordsInEachPartition = 50;
-
- final String topicName =
- writeSequence(
- "testStartFromGroupOffsetsTopic", recordsInEachPartition, parallelism, 1);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
-
- Properties readProps = new Properties();
- readProps.putAll(standardProps);
- readProps.setProperty("auto.offset.reset", "earliest");
-
- // the committed group offsets should be used as starting points
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
-
- // only partitions 0 and 2 have group offsets committed
- kafkaOffsetHandler.setCommittedOffset(topicName, 0, 23);
- kafkaOffsetHandler.setCommittedOffset(topicName, 2, 43);
-
- Map<Integer, Tuple2<Integer, Integer>> partitionsToValueCountAndStartOffsets =
- new HashMap<>();
- partitionsToValueCountAndStartOffsets.put(
- 0, new Tuple2<>(27, 23)); // partition 0 should read offset 23-49
- partitionsToValueCountAndStartOffsets.put(
- 1, new Tuple2<>(50, 0)); // partition 1 should read offset 0-49
- partitionsToValueCountAndStartOffsets.put(
- 2, new Tuple2<>(7, 43)); // partition 2 should read offset 43-49
-
- readSequence(
- env,
- StartupMode.GROUP_OFFSETS,
- null,
- null,
- readProps,
- topicName,
- partitionsToValueCountAndStartOffsets);
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
- }
-
- /**
- * This test ensures that the consumer correctly uses user-supplied specific offsets when
- * explicitly configured to start from specific offsets. For partitions which a specific offset
- * can not be found for, the starting position for them should fallback to the group offsets
- * behaviour.
- *
- * <p>4 partitions will have 50 records with offsets 0 to 49. The supplied specific offsets map
- * is: partition 0 --> start from offset 19 partition 1 --> not set partition 2 --> start from
- * offset 22 partition 3 --> not set partition 4 --> start from offset 26 (this should be
- * ignored because the partition does not exist)
- *
- * <p>The partitions and their committed group offsets are setup as: partition 0 --> committed
- * offset 23 partition 1 --> committed offset 31 partition 2 --> committed offset 43 partition 3
- * --> no commit offset
- *
- * <p>When configured to start from these specific offsets, each partition should read:
- * partition 0 --> start from offset 19, read to offset 49 (31 records) partition 1 --> fallback
- * to group offsets, so start from offset 31, read to offset 49 (19 records) partition 2 -->
- * start from offset 22, read to offset 49 (28 records) partition 3 --> fallback to group
- * offsets, but since there is no group offset for this partition, will default to
- * "auto.offset.reset" (set to "earliest"), so start from offset 0, read to offset 49 (50
- * records)
- */
- public void runStartFromSpecificOffsets() throws Exception {
- // 4 partitions with 50 records each (offsets 0-49)
- final int parallelism = 4;
- final int recordsInEachPartition = 50;
-
- final String topicName =
- writeSequence(
- "testStartFromSpecificOffsetsTopic",
- recordsInEachPartition,
- parallelism,
- 1);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
-
- Properties readProps = new Properties();
- readProps.putAll(standardProps);
- readProps.setProperty(
- "auto.offset.reset",
- "earliest"); // partition 3 should default back to this behaviour
-
- Map<KafkaTopicPartition, Long> specificStartupOffsets = new HashMap<>();
- specificStartupOffsets.put(new KafkaTopicPartition(topicName, 0), 19L);
- specificStartupOffsets.put(new KafkaTopicPartition(topicName, 2), 22L);
- specificStartupOffsets.put(
- new KafkaTopicPartition(topicName, 4),
- 26L); // non-existing partition, should be ignored
-
- // only the committed offset for partition 1 should be used, because partition 1 has no
- // entry in specific offset map
- KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler =
- kafkaServer.createOffsetHandler();
- kafkaOffsetHandler.setCommittedOffset(topicName, 0, 23);
- kafkaOffsetHandler.setCommittedOffset(topicName, 1, 31);
- kafkaOffsetHandler.setCommittedOffset(topicName, 2, 43);
-
- Map<Integer, Tuple2<Integer, Integer>> partitionsToValueCountAndStartOffsets =
- new HashMap<>();
- partitionsToValueCountAndStartOffsets.put(
- 0, new Tuple2<>(31, 19)); // partition 0 should read offset 19-49
- partitionsToValueCountAndStartOffsets.put(
- 1, new Tuple2<>(19, 31)); // partition 1 should read offset 31-49
- partitionsToValueCountAndStartOffsets.put(
- 2, new Tuple2<>(28, 22)); // partition 2 should read offset 22-49
- partitionsToValueCountAndStartOffsets.put(
- 3, new Tuple2<>(50, 0)); // partition 3 should read offset 0-49
-
- readSequence(
- env,
- StartupMode.SPECIFIC_OFFSETS,
- specificStartupOffsets,
- null,
- readProps,
- topicName,
- partitionsToValueCountAndStartOffsets);
-
- kafkaOffsetHandler.close();
- deleteTestTopic(topicName);
- }
-
- /**
- * This test ensures that the consumer correctly uses user-supplied timestamp when explicitly
- * configured to start from timestamp.
- *
- * <p>The validated Kafka data is written in 2 steps: first, an initial 50 records is written to
- * each partition. After that, another 30 records is appended to each partition. Before each
- * step, a timestamp is recorded. For the validation, when the read job is configured to start
- * from the first timestamp, each partition should start from offset 0 and read a total of 80
- * records. When configured to start from the second timestamp, each partition should start from
- * offset 50 and read on the remaining 30 appended records.
- */
- public void runStartFromTimestamp() throws Exception {
- // 4 partitions with 50 records each
- final int parallelism = 4;
- final int initialRecordsInEachPartition = 50;
- final int appendRecordsInEachPartition = 30;
-
- // attempt to create an appended test sequence, where the timestamp of writing the appended
- // sequence
- // is assured to be larger than the timestamp of the original sequence.
- long firstTimestamp = System.currentTimeMillis();
- String topic =
- writeSequence(
- "runStartFromTimestamp", initialRecordsInEachPartition, parallelism, 1);
-
- long secondTimestamp = 0;
- while (secondTimestamp <= firstTimestamp) {
- Thread.sleep(1000);
- secondTimestamp = System.currentTimeMillis();
- }
- writeAppendSequence(
- topic, initialRecordsInEachPartition, appendRecordsInEachPartition, parallelism);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
-
- Properties readProps = new Properties();
- readProps.putAll(standardProps);
-
- readSequence(
- env,
- StartupMode.TIMESTAMP,
- null,
- firstTimestamp,
- readProps,
- parallelism,
- topic,
- initialRecordsInEachPartition + appendRecordsInEachPartition,
- 0);
- readSequence(
- env,
- StartupMode.TIMESTAMP,
- null,
- secondTimestamp,
- readProps,
- parallelism,
- topic,
- appendRecordsInEachPartition,
- initialRecordsInEachPartition);
-
- deleteTestTopic(topic);
- }
-
- /**
- * Ensure Kafka is working on both producer and consumer side. This executes a job that contains
- * two Flink pipelines.
- *
- * <pre>
- * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
- * </pre>
- *
- * <p>We need to externally retry this test. We cannot let Flink's retry mechanism do it,
- * because the Kafka producer does not guarantee exactly-once output. Hence a recovery would
- * introduce duplicates that cause the test to fail.
- *
- * <p>This test also ensures that FLINK-3156 doesn't happen again:
- *
- * <p>The following situation caused a NPE in the FlinkKafkaConsumer
- *
- * <p>topic-1 <-- elements are only produced into topic1. topic-2
- *
- * <p>Therefore, this test is consuming as well from an empty topic.
- */
- @RetryOnException(times = 2, exception = NotLeaderForPartitionException.class)
- public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
- final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID();
- final String additionalEmptyTopic = "additionalEmptyTopic_" + UUID.randomUUID();
-
- final int parallelism = 3;
- final int elementsPerPartition = 100;
- final int totalElements = parallelism * elementsPerPartition;
-
- createTestTopic(topic, parallelism, 1);
- createTestTopic(
- additionalEmptyTopic,
- parallelism,
- 1); // create an empty topic which will remain empty all the time
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
- env.enableCheckpointing(500);
- env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-
- TypeInformation<Tuple2<Long, String>> longStringType =
- TypeInformation.of(new TypeHint<Tuple2<Long, String>>() {});
-
- TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
- new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
- TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
- new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
- // ----------- add producer dataflow ----------
-
- DataStream<Tuple2<Long, String>> stream =
- env.addSource(
- new RichParallelSourceFunction<Tuple2<Long, String>>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Long, String>> ctx)
- throws InterruptedException {
- int cnt =
- getRuntimeContext().getIndexOfThisSubtask()
- * elementsPerPartition;
- int limit = cnt + elementsPerPartition;
-
- while (running && cnt < limit) {
- ctx.collect(new Tuple2<>(1000L + cnt, "kafka-" + cnt));
- cnt++;
- // we delay data generation a bit so that we are sure that some
- // checkpoints are
- // triggered (for FLINK-3156)
- Thread.sleep(50);
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
- producerProperties.setProperty("retries", "3");
- producerProperties.putAll(secureProps);
- kafkaServer.produceIntoKafka(stream, topic, sinkSchema, producerProperties, null);
-
- // ----------- add consumer dataflow ----------
-
- List<String> topics = new ArrayList<>();
- topics.add(topic);
- topics.add(additionalEmptyTopic);
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- DataStreamSource<Tuple2<Long, String>> consuming =
- getStream(env, topics, sourceSchema, props);
-
- consuming
- .addSink(
- new RichSinkFunction<Tuple2<Long, String>>() {
-
- private int elCnt = 0;
-
- private BitSet validator = new BitSet(totalElements);
-
- @Override
- public void invoke(Tuple2<Long, String> value) throws Exception {
- String[] sp = value.f1.split("-");
- int v = Integer.parseInt(sp[1]);
- assertThat((long) v).isEqualTo(value.f0 - 1000);
- assertThat(validator.get(v)).as("Received tuple twice").isFalse();
- validator.set(v);
- elCnt++;
- if (elCnt == totalElements) {
- // check if everything in the bitset is set to true
- int nc;
- if ((nc = validator.nextClearBit(0)) != totalElements) {
- fail(
- "The bitset was not set to 1 on all elements. Next clear:"
- + nc
- + " Set: "
- + validator);
- }
- throw new SuccessException();
- }
- }
-
- @Override
- public void close() throws Exception {
- super.close();
- }
- })
- .setParallelism(1);
-
- try {
- tryExecutePropagateExceptions(env, "runSimpleConcurrentProducerConsumerTopology");
- } catch (ProgramInvocationException | JobExecutionException e) {
- // look for NotLeaderForPartitionException
- Throwable cause = e.getCause();
-
- // search for nested SuccessExceptions
- int depth = 0;
- while (cause != null && depth++ < 20) {
- if (cause instanceof NotLeaderForPartitionException) {
- throw (Exception) cause;
- }
- cause = cause.getCause();
- }
- throw e;
- }
-
- deleteTestTopic(topic);
- }
-
- /**
- * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
- * Flink sources.
- */
- public void runOneToOneExactlyOnceTest() throws Exception {
-
- final String topic = "oneToOneTopic-" + UUID.randomUUID();
- final int parallelism = 5;
- final int numElementsPerPartition = 1000;
- final int totalElements = parallelism * numElementsPerPartition;
- final int failAfterElements = numElementsPerPartition / 3;
-
- createTestTopic(topic, parallelism, 1);
-
- DataGenerators.generateRandomizedIntegerSequence(
- StreamExecutionEnvironment.getExecutionEnvironment(),
- kafkaServer,
- topic,
- parallelism,
- numElementsPerPartition,
- true);
-
- // run the topology that fails and recovers
-
- DeserializationSchema<Integer> schema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.enableCheckpointing(500);
- env.setParallelism(parallelism);
- env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- getStream(env, topic, schema, props)
- .map(new PartitionValidatingMapper(parallelism, 1))
- .map(new FailingIdentityMapper<Integer>(failAfterElements))
- .addSink(new ValidatingExactlyOnceSink(totalElements))
- .setParallelism(1);
-
- FailingIdentityMapper.failedBefore = false;
- tryExecute(env, "One-to-one exactly once test");
-
- deleteTestTopic(topic);
- }
-
- /**
- * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so one
- * Flink source will read multiple Kafka partitions.
- */
- public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
- final String topic = "oneToManyTopic-" + UUID.randomUUID();
- final int numPartitions = 5;
- final int numElementsPerPartition = 1000;
- final int totalElements = numPartitions * numElementsPerPartition;
- final int failAfterElements = numElementsPerPartition / 3;
-
- final int parallelism = 2;
-
- createTestTopic(topic, numPartitions, 1);
-
- DataGenerators.generateRandomizedIntegerSequence(
- StreamExecutionEnvironment.getExecutionEnvironment(),
- kafkaServer,
- topic,
- numPartitions,
- numElementsPerPartition,
- true);
-
- // run the topology that fails and recovers
-
- DeserializationSchema<Integer> schema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.enableCheckpointing(500);
- env.setParallelism(parallelism);
- env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- getStream(env, topic, schema, props)
- .map(new PartitionValidatingMapper(numPartitions, 3))
- .map(new FailingIdentityMapper<Integer>(failAfterElements))
- .addSink(new ValidatingExactlyOnceSink(totalElements))
- .setParallelism(1);
-
- FailingIdentityMapper.failedBefore = false;
- tryExecute(env, "One-source-multi-partitions exactly once test");
-
- deleteTestTopic(topic);
- }
-
- /**
- * Tests the proper consumption when having more Flink sources than Kafka partitions, which
- * means that some Flink sources will read no partitions.
- */
- public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
- final String topic = "manyToOneTopic-" + UUID.randomUUID();
- final int numPartitions = 5;
- final int numElementsPerPartition = 1000;
- final int totalElements = numPartitions * numElementsPerPartition;
- final int failAfterElements = numElementsPerPartition / 3;
-
- final int parallelism = 8;
-
- createTestTopic(topic, numPartitions, 1);
-
- DataGenerators.generateRandomizedIntegerSequence(
- StreamExecutionEnvironment.getExecutionEnvironment(),
- kafkaServer,
- topic,
- numPartitions,
- numElementsPerPartition,
- true);
-
- // run the topology that fails and recovers
-
- DeserializationSchema<Integer> schema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.enableCheckpointing(500);
- env.setParallelism(parallelism);
- // set the number of restarts to one. The failing mapper will fail once, then it's only
- // success exceptions.
- env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
- env.setBufferTimeout(0);
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- getStream(env, topic, schema, props)
- .map(new PartitionValidatingMapper(numPartitions, 1))
- // Job only fails after a checkpoint is taken and the necessary number of elements
- // is seen
- .map(new FailingIdentityMapper<Integer>(failAfterElements))
- .addSink(new ValidatingExactlyOnceSink(totalElements, true))
- .setParallelism(1);
-
- FailingIdentityMapper.failedBefore = false;
- tryExecute(env, "multi-source-one-partitions exactly once test");
-
- deleteTestTopic(topic);
- }
-
- /** Tests that the source can be properly canceled when reading full partitions. */
- public void runCancelingOnFullInputTest() throws Exception {
- final String topic = "cancelingOnFullTopic-" + UUID.randomUUID();
-
- final int parallelism = 3;
- createTestTopic(topic, parallelism, 1);
-
- // launch a producer thread
- DataGenerators.InfiniteStringsGenerator generator =
- new DataGenerators.InfiniteStringsGenerator(kafkaServer, topic);
- generator.start();
-
- // launch a consumer asynchronously
-
- final AtomicReference<Throwable> jobError = new AtomicReference<>();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
- env.enableCheckpointing(100);
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- getStream(env, topic, new SimpleStringSchema(), props)
- .addSink(new DiscardingSink<String>());
-
- JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph());
- final JobID jobId = jobGraph.getJobID();
-
- final Runnable jobRunner =
- () -> {
- try {
- submitJobAndWaitForResult(client, jobGraph, getClass().getClassLoader());
- } catch (Throwable t) {
- jobError.set(t);
- }
- };
-
- Thread runnerThread = new Thread(jobRunner, "program runner thread");
- runnerThread.start();
-
- // wait a bit before canceling
- Thread.sleep(2000);
-
- Throwable failueCause = jobError.get();
- if (failueCause != null) {
- failueCause.printStackTrace();
- fail("Test failed prematurely with: " + failueCause.getMessage());
- }
-
- // cancel
- client.cancel(jobId).get();
-
- // wait for the program to be done and validate that we failed with the right exception
- runnerThread.join();
-
- assertThat(client.getJobStatus(jobId).get()).isEqualTo(JobStatus.CANCELED);
-
- if (generator.isAlive()) {
- generator.shutdown();
- generator.join();
- } else {
- Throwable t = generator.getError();
- if (t != null) {
- t.printStackTrace();
- fail("Generator failed: " + t.getMessage());
- } else {
- fail("Generator failed with no exception");
- }
- }
-
- deleteTestTopic(topic);
- }
-
- /** Tests that the source can be properly canceled when reading empty partitions. */
- public void runCancelingOnEmptyInputTest() throws Exception {
- final String topic = "cancelingOnEmptyInputTopic-" + UUID.randomUUID();
-
- final int parallelism = 3;
- createTestTopic(topic, parallelism, 1);
-
- final AtomicReference<Throwable> error = new AtomicReference<>();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
- env.enableCheckpointing(100);
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- getStream(env, topic, new SimpleStringSchema(), props)
- .addSink(new DiscardingSink<String>());
-
- JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph());
- final JobID jobId = jobGraph.getJobID();
-
- final Runnable jobRunner =
- () -> {
- try {
- submitJobAndWaitForResult(client, jobGraph, getClass().getClassLoader());
- } catch (Throwable t) {
- LOG.error("Job Runner failed with exception", t);
- error.set(t);
- }
- };
-
- Thread runnerThread = new Thread(jobRunner, "program runner thread");
- runnerThread.start();
-
- // wait a bit before canceling
- Thread.sleep(2000);
-
- Throwable failueCause = error.get();
- if (failueCause != null) {
- failueCause.printStackTrace();
- fail("Test failed prematurely with: " + failueCause.getMessage());
- }
- // cancel
- client.cancel(jobId).get();
-
- // wait for the program to be done and validate that we failed with the right exception
- runnerThread.join();
-
- assertThat(client.getJobStatus(jobId).get()).isEqualTo(JobStatus.CANCELED);
-
- deleteTestTopic(topic);
- }
-
- /**
- * Test producing and consuming into multiple topics.
- *
- * @throws Exception
- */
- public void runProduceConsumeMultipleTopics(boolean useLegacySchema) throws Exception {
- final String topicNamePrefix =
- "runProduceConsumeMultipleTopics-" + (useLegacySchema ? "legacy" : "");
-
- final int numTopics = 5;
- final int numElements = 20;
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- // create topics with content
- final List<String> topics = new ArrayList<>();
- for (int i = 0; i < numTopics; i++) {
- final String topic = topicNamePrefix + i + "-" + UUID.randomUUID();
- topics.add(topic);
- // create topic
- createTestTopic(topic, i + 1 /*partitions*/, 1);
- }
-
- // before FLINK-6078 the RemoteExecutionEnvironment set the parallelism to 1 as well
- env.setParallelism(1);
-
- // run first job, producing into all topics
- DataStream<Tuple3<Integer, Integer, String>> stream =
- env.addSource(
- new RichParallelSourceFunction<Tuple3<Integer, Integer, String>>() {
-
- @Override
- public void run(SourceContext<Tuple3<Integer, Integer, String>> ctx) {
- int partition = getRuntimeContext().getIndexOfThisSubtask();
-
- for (int topicId = 0; topicId < numTopics; topicId++) {
- for (int i = 0; i < numElements; i++) {
- ctx.collect(
- new Tuple3<>(partition, i, topics.get(topicId)));
- }
- }
- }
-
- @Override
- public void cancel() {}
- });
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- if (useLegacySchema) {
- Tuple2WithTopicSchema schema = new Tuple2WithTopicSchema(env.getConfig());
- kafkaServer.produceIntoKafka(stream, "dummy", schema, props, null);
- } else {
- TestDeserializer schema = new TestDeserializer(env.getConfig());
- kafkaServer.produceIntoKafka(stream, "dummy", schema, props);
- }
-
- env.execute("Write to topics");
-
- // run second job consuming from multiple topics
- env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- if (useLegacySchema) {
- Tuple2WithTopicSchema schema = new Tuple2WithTopicSchema(env.getConfig());
- stream = getStream(env, topics, schema, props);
- } else {
- TestDeserializer schema = new TestDeserializer(env.getConfig());
- stream = getStream(env, topics, schema, props);
- }
-
- stream.flatMap(
- new FlatMapFunction<Tuple3<Integer, Integer, String>, Integer>() {
- Map<String, Integer> countPerTopic = new HashMap<>(numTopics);
-
- @Override
- public void flatMap(
- Tuple3<Integer, Integer, String> value, Collector<Integer> out)
- throws Exception {
- Integer count = countPerTopic.get(value.f2);
- if (count == null) {
- count = 1;
- } else {
- count++;
- }
- countPerTopic.put(value.f2, count);
-
- // check map:
- for (Map.Entry<String, Integer> el : countPerTopic.entrySet()) {
- if (el.getValue() < numElements) {
- break; // not enough yet
- }
- if (el.getValue() > numElements) {
- throw new RuntimeException(
- "There is a failure in the test. I've read "
- + el.getValue()
- + " from topic "
- + el.getKey());
- }
- }
- // we've seen messages from all topics
- throw new SuccessException();
- }
- })
- .setParallelism(1);
-
- tryExecute(env, "Count elements from the topics");
-
- // delete all topics again
- for (String topic : topics) {
- deleteTestTopic(topic);
- }
- }
-
- /**
- * Test Flink's Kafka integration also with very big records (30MB).
- *
- * <p>see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
- */
- public void runBigRecordTestTopology() throws Exception {
-
- final String topic = "bigRecordTestTopic-" + UUID.randomUUID();
- final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
-
- createTestTopic(topic, parallelism, 1);
-
- final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo =
- TypeInformation.of(new TypeHint<Tuple2<Long, byte[]>>() {});
-
- final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
- new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setRestartStrategy(RestartStrategies.noRestart());
- env.enableCheckpointing(100);
- env.setParallelism(parallelism);
-
- // add consuming topology:
- Properties consumerProps = new Properties();
- consumerProps.putAll(standardProps);
- consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 14));
- consumerProps.setProperty(
- "max.partition.fetch.bytes",
- Integer.toString(1024 * 1024 * 14)); // for the new fetcher
- consumerProps.setProperty("queued.max.message.chunks", "1");
- consumerProps.putAll(secureProps);
-
- DataStreamSource<Tuple2<Long, byte[]>> consuming =
- getStream(env, topic, serSchema, consumerProps);
-
- consuming.addSink(
- new SinkFunction<Tuple2<Long, byte[]>>() {
-
- private int elCnt = 0;
-
- @Override
- public void invoke(Tuple2<Long, byte[]> value) throws Exception {
- elCnt++;
- if (value.f0 == -1) {
- // we should have seen 11 elements now.
- if (elCnt == 11) {
- throw new SuccessException();
- } else {
- throw new RuntimeException(
- "There have been " + elCnt + " elements");
- }
- }
- if (elCnt > 10) {
- throw new RuntimeException("More than 10 elements seen: " + elCnt);
- }
- }
- });
-
- // add producing topology
- Properties producerProps = new Properties();
- producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 15));
- producerProps.setProperty("retries", "3");
- producerProps.putAll(secureProps);
- producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings);
-
- DataStream<Tuple2<Long, byte[]>> stream =
- env.addSource(
- new RichSourceFunction<Tuple2<Long, byte[]>>() {
-
- private boolean running;
-
- @Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- running = true;
- }
-
- @Override
- public void run(SourceContext<Tuple2<Long, byte[]>> ctx)
- throws Exception {
- Random rnd = new Random();
- long cnt = 0;
- int sevenMb = 1024 * 1024 * 7;
-
- while (running) {
- byte[] wl = new byte[sevenMb + rnd.nextInt(sevenMb)];
- ctx.collect(new Tuple2<>(cnt++, wl));
-
- Thread.sleep(100);
-
- if (cnt == 10) {
- // signal end
- ctx.collect(new Tuple2<>(-1L, new byte[] {1}));
- break;
- }
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
-
- kafkaServer.produceIntoKafka(stream, topic, serSchema, producerProps, null);
-
- tryExecute(env, "big topology test");
- deleteTestTopic(topic);
- }
-
- public void runBrokerFailureTest() throws Exception {
- final String topic = "brokerFailureTestTopic";
-
- // Start a temporary multi-broker cluster.
- // This test case relies on stopping a broker and switching partition leader to another
- // during the test, so single-broker cluster (kafkaServer) could not fulfill the
- // requirement.
- KafkaTestEnvironment multiBrokerCluster = constructKafkaTestEnvironment();
- multiBrokerCluster.prepare(KafkaTestEnvironment.createConfig().setKafkaServersNumber(3));
-
- final int parallelism = 2;
- final int numElementsPerPartition = 1000;
- final int totalElements = parallelism * numElementsPerPartition;
- final int failAfterElements = numElementsPerPartition / 3;
-
- multiBrokerCluster.createTestTopic(topic, parallelism, 2);
-
- DataGenerators.generateRandomizedIntegerSequence(
- StreamExecutionEnvironment.getExecutionEnvironment(),
- multiBrokerCluster,
- topic,
- parallelism,
- numElementsPerPartition,
- true);
-
- // find leader to shut down
- int leaderId = multiBrokerCluster.getLeaderToShutDown(topic);
-
- LOG.info("Leader to shutdown {}", leaderId);
-
- // run the topology (the consumers must handle the failures)
-
- DeserializationSchema<Integer> schema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
- env.enableCheckpointing(500);
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- Properties props = new Properties();
- props.putAll(multiBrokerCluster.getStandardProperties());
- props.putAll(multiBrokerCluster.getSecureProperties());
-
- getStream(env, topic, schema, props)
- .map(new PartitionValidatingMapper(parallelism, 1))
- .map(new BrokerKillingMapper<>(multiBrokerCluster, leaderId, failAfterElements))
- .addSink(new ValidatingExactlyOnceSink(totalElements))
- .setParallelism(1);
-
- try {
- BrokerKillingMapper.killedLeaderBefore = false;
- tryExecute(env, "Broker failure once test");
- } finally {
- // Tear down the temporary cluster anyway
- multiBrokerCluster.shutdown();
- }
- }
-
- public void runKeyValueTest() throws Exception {
- final String topic = "keyvaluetest-" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
- final int elementCount = 5000;
-
- // ----------- Write some data into Kafka -------------------
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- DataStream<Tuple2<Long, PojoValue>> kvStream =
- env.addSource(
- new SourceFunction<Tuple2<Long, PojoValue>>() {
- @Override
- public void run(SourceContext<Tuple2<Long, PojoValue>> ctx)
- throws Exception {
- Random rnd = new Random(1337);
- for (long i = 0; i < elementCount; i++) {
- PojoValue pojo = new PojoValue();
- pojo.when = new Date(rnd.nextLong());
- pojo.lon = rnd.nextLong();
- pojo.lat = i;
- // make every second key null to ensure proper "null"
- // serialization
- Long key = (i % 2 == 0) ? null : i;
- ctx.collect(new Tuple2<>(key, pojo));
- }
- }
-
- @Override
- public void cancel() {}
- });
-
- KeyedSerializationSchema<Tuple2<Long, PojoValue>> schema =
- new TypeInformationKeyValueSerializationSchema<>(
- Long.class, PojoValue.class, env.getConfig());
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
- producerProperties.setProperty("retries", "3");
- kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
- env.execute("Write KV to Kafka");
-
- // ----------- Read the data again -------------------
-
- env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- KafkaDeserializationSchema<Tuple2<Long, PojoValue>> readSchema =
- new TypeInformationKeyValueSerializationSchema<>(
- Long.class, PojoValue.class, env.getConfig());
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- DataStream<Tuple2<Long, PojoValue>> fromKafka = getStream(env, topic, readSchema, props);
- fromKafka.flatMap(
- new RichFlatMapFunction<Tuple2<Long, PojoValue>, Object>() {
-
- long counter = 0;
-
- @Override
- public void flatMap(Tuple2<Long, PojoValue> value, Collector<Object> out)
- throws Exception {
- // the elements should be in order.
- assertThat(value.f1.lat)
- .as("Wrong value " + value.f1.lat)
- .isEqualTo(counter);
- if (value.f1.lat % 2 == 0) {
- assertThat(value.f0).as("key was not null").isNull();
- } else {
- assertThat(value.f0).as("Wrong value " + value.f0).isEqualTo(counter);
- }
- counter++;
- if (counter == elementCount) {
- // we got the right number of elements
- throw new SuccessException();
- }
- }
- });
-
- tryExecute(env, "Read KV from Kafka");
-
- deleteTestTopic(topic);
- }
-
- private static class PojoValue {
- public Date when;
- public long lon;
- public long lat;
-
- public PojoValue() {}
- }
-
- /**
- * Test delete behavior and metrics for producer.
- *
- * @throws Exception
- */
- public void runAllDeletesTest() throws Exception {
- final String topic = "alldeletestest-" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
- final int elementCount = 300;
-
- // ----------- Write some data into Kafka -------------------
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- DataStream<Tuple2<byte[], PojoValue>> kvStream =
- env.addSource(
- new SourceFunction<Tuple2<byte[], PojoValue>>() {
- @Override
- public void run(SourceContext<Tuple2<byte[], PojoValue>> ctx)
- throws Exception {
- Random rnd = new Random(1337);
- for (long i = 0; i < elementCount; i++) {
- final byte[] key = new byte[200];
- rnd.nextBytes(key);
- ctx.collect(new Tuple2<>(key, (PojoValue) null));
- }
- }
-
- @Override
- public void cancel() {}
- });
-
- TypeInformationKeyValueSerializationSchema<byte[], PojoValue> schema =
- new TypeInformationKeyValueSerializationSchema<>(
- byte[].class, PojoValue.class, env.getConfig());
-
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
- producerProperties.setProperty("retries", "3");
- producerProperties.putAll(secureProps);
- kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null);
-
- env.execute("Write deletes to Kafka");
-
- // ----------- Read the data again -------------------
-
- env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- DataStream<Tuple2<byte[], PojoValue>> fromKafka = getStream(env, topic, schema, props);
-
- fromKafka.flatMap(
- new RichFlatMapFunction<Tuple2<byte[], PojoValue>, Object>() {
-
- long counter = 0;
-
- @Override
- public void flatMap(Tuple2<byte[], PojoValue> value, Collector<Object> out)
- throws Exception {
- // ensure that deleted messages are passed as nulls
- assertThat(value.f1).isNull();
- counter++;
- if (counter == elementCount) {
- // we got the right number of elements
- throw new SuccessException();
- }
- }
- });
-
- tryExecute(env, "Read deletes from Kafka");
-
- deleteTestTopic(topic);
- }
-
- /**
- * Test that ensures that DeserializationSchema.isEndOfStream() is properly evaluated.
- *
- * @throws Exception
- */
- public void runEndOfStreamTest() throws Exception {
-
- final int elementCount = 300;
- final String topic = writeSequence("testEndOfStream", elementCount, 1, 1);
-
- // read using custom schema
- final StreamExecutionEnvironment env1 =
- StreamExecutionEnvironment.getExecutionEnvironment();
- env1.setParallelism(1);
- env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- DataStream<Tuple2<Integer, Integer>> fromKafka =
- getStream(env1, topic, new FixedNumberDeserializationSchema(elementCount), props);
-
- fromKafka.flatMap(
- new FlatMapFunction<Tuple2<Integer, Integer>, Void>() {
- @Override
- public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out)
- throws Exception {
- // noop ;)
- }
- });
-
- tryExecute(env1, "Consume " + elementCount + " elements from Kafka");
-
- deleteTestTopic(topic);
- }
-
- /**
- * Test that ensures that DeserializationSchema can emit multiple records via a Collector.
- *
- * @throws Exception
- */
- public void runCollectingSchemaTest() throws Exception {
-
- final int elementCount = 20;
- final String topic = writeSequence("testCollectingSchema", elementCount, 1, 1);
-
- // read using custom schema
- final StreamExecutionEnvironment env1 =
- StreamExecutionEnvironment.getExecutionEnvironment();
- env1.setParallelism(1);
- env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
-
- DataStream<Tuple2<Integer, String>> fromKafka =
- env1.addSource(
- kafkaServer
- .getConsumer(
- topic,
- new CollectingDeserializationSchema(elementCount),
- props)
- .assignTimestampsAndWatermarks(
- new AscendingTimestampExtractor<Tuple2<Integer, String>>() {
- @Override
- public long extractAscendingTimestamp(
- Tuple2<Integer, String> element) {
- String string = element.f1;
- return Long.parseLong(
- string.substring(0, string.length() - 1));
- }
- }));
- fromKafka
- .keyBy(t -> t.f0)
- .process(
- new KeyedProcessFunction<Integer, Tuple2<Integer, String>, Void>() {
- private boolean registered = false;
-
- @Override
- public void processElement(
- Tuple2<Integer, String> value, Context ctx, Collector<Void> out)
- throws Exception {
- if (!registered) {
- ctx.timerService().registerEventTimeTimer(elementCount - 2);
- registered = true;
- }
- }
-
- @Override
- public void onTimer(
- long timestamp, OnTimerContext ctx, Collector<Void> out)
- throws Exception {
- throw new SuccessException();
- }
- });
-
- tryExecute(env1, "Consume " + elementCount + " elements from Kafka");
-
- deleteTestTopic(topic);
- }
-
- /**
- * Test metrics reporting for consumer.
- *
- * @throws Exception
- */
- public void runMetricsTest() throws Throwable {
-
- // create a stream with 5 topics
- final String topic = "metricsStream-" + UUID.randomUUID();
- createTestTopic(topic, 5, 1);
-
- final Tuple1<Throwable> error = new Tuple1<>(null);
-
- // start job writing & reading data.
- final StreamExecutionEnvironment env1 =
- StreamExecutionEnvironment.getExecutionEnvironment();
- env1.setParallelism(1);
- env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- env1.disableOperatorChaining(); // let the source read everything into the network buffers
-
- TypeInformationSerializationSchema<Tuple2<Integer, Integer>> schema =
- new TypeInformationSerializationSchema<>(
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {}),
- env1.getConfig());
-
- DataStream<Tuple2<Integer, Integer>> fromKafka =
- getStream(env1, topic, schema, standardProps);
- fromKafka.flatMap(
- new FlatMapFunction<Tuple2<Integer, Integer>, Void>() {
- @Override
- public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out)
- throws Exception { // no op
- }
- });
-
- DataStream<Tuple2<Integer, Integer>> fromGen =
- env1.addSource(
- new RichSourceFunction<Tuple2<Integer, Integer>>() {
- boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Integer, Integer>> ctx)
- throws Exception {
- int i = 0;
- while (running) {
- ctx.collect(
- Tuple2.of(
- i++,
- getRuntimeContext().getIndexOfThisSubtask()));
- Thread.sleep(1);
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
-
- kafkaServer.produceIntoKafka(fromGen, topic, schema, standardProps, null);
-
- JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env1.getStreamGraph());
- final JobID jobId = jobGraph.getJobID();
-
- Thread jobThread =
- new Thread(
- () -> {
- try {
- submitJobAndWaitForResult(
- client, jobGraph, getClass().getClassLoader());
- } catch (Throwable t) {
- if (!ExceptionUtils.findThrowable(t, JobCancellationException.class)
- .isPresent()) {
- LOG.warn("Got exception during execution", t);
- error.f0 = t;
- }
- }
- });
- jobThread.start();
-
- try {
- // connect to JMX
- MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
- // wait until we've found all 5 offset metrics
- Set<ObjectName> offsetMetrics =
- mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
- while (offsetMetrics.size()
- < 5) { // test will time out if metrics are not properly working
- if (error.f0 != null) {
- // fail test early
- throw error.f0;
- }
- offsetMetrics = mBeanServer.queryNames(new ObjectName("*current-offsets*:*"), null);
- Thread.sleep(50);
- }
- assertThat(offsetMetrics).hasSize(5);
- // we can't rely on the consumer to have touched all the partitions already
- // that's why we'll wait until all five partitions have a positive offset.
- // The test will fail if we never meet the condition
- while (true) {
- int numPosOffsets = 0;
- // check that offsets are correctly reported
- for (ObjectName object : offsetMetrics) {
- Object offset = mBeanServer.getAttribute(object, "Value");
- if ((long) offset >= 0) {
- numPosOffsets++;
- }
- }
- if (numPosOffsets == 5) {
- break;
- }
- // wait for the consumer to consume on all partitions
- Thread.sleep(50);
- }
-
- // check if producer metrics are also available.
- Set<ObjectName> producerMetrics =
- mBeanServer.queryNames(new ObjectName("*KafkaProducer*:*"), null);
- assertThat(producerMetrics.size()).as("No producer metrics found").isGreaterThan(30);
-
- LOG.info("Found all JMX metrics. Cancelling job.");
- } finally {
- // cancel
- client.cancel(jobId).get();
- // wait for the job to finish (it should due to the cancel command above)
- jobThread.join();
- }
-
- if (error.f0 != null) {
- throw error.f0;
- }
-
- deleteTestTopic(topic);
- }
-
- private static class CollectingDeserializationSchema
- implements KafkaDeserializationSchema<Tuple2<Integer, String>> {
-
- final int finalCount;
-
- TypeInformation<Tuple2<Integer, String>> ti =
- TypeInformation.of(new TypeHint<Tuple2<Integer, String>>() {});
- TypeSerializer<Tuple2<Integer, Integer>> ser =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {})
- .createSerializer(new ExecutionConfig());
-
- public CollectingDeserializationSchema(int finalCount) {
- this.finalCount = finalCount;
- }
-
- @Override
- public boolean isEndOfStream(Tuple2<Integer, String> nextElement) {
- return false;
- }
-
- @Override
- public Tuple2<Integer, String> deserialize(ConsumerRecord<byte[], byte[]> record)
- throws Exception {
- throw new UnsupportedOperationException("Should not be called");
- }
-
- @Override
- public void deserialize(
- ConsumerRecord<byte[], byte[]> message, Collector<Tuple2<Integer, String>> out)
- throws Exception {
- DataInputView in =
- new DataInputViewStreamWrapper(new ByteArrayInputStream(message.value()));
- Tuple2<Integer, Integer> tuple = ser.deserialize(in);
- out.collect(Tuple2.of(tuple.f0, tuple.f1 + "a"));
- out.collect(Tuple2.of(tuple.f0, tuple.f1 + "b"));
- }
-
- @Override
- public TypeInformation<Tuple2<Integer, String>> getProducedType() {
- return ti;
- }
- }
-
- private static class FixedNumberDeserializationSchema
- implements DeserializationSchema<Tuple2<Integer, Integer>> {
-
- final int finalCount;
- int count = 0;
-
- TypeInformation<Tuple2<Integer, Integer>> ti =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
- TypeSerializer<Tuple2<Integer, Integer>> ser = ti.createSerializer(new ExecutionConfig());
-
- public FixedNumberDeserializationSchema(int finalCount) {
- this.finalCount = finalCount;
- }
-
- @Override
- public Tuple2<Integer, Integer> deserialize(byte[] message) throws IOException {
- DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(message));
- return ser.deserialize(in);
- }
-
- @Override
- public boolean isEndOfStream(Tuple2<Integer, Integer> nextElement) {
- return ++count >= finalCount;
- }
-
- @Override
- public TypeInformation<Tuple2<Integer, Integer>> getProducedType() {
- return ti;
- }
- }
-
- // ------------------------------------------------------------------------
- // Reading writing test data sets
- // ------------------------------------------------------------------------
-
- /**
- * Runs a job using the provided environment to read a sequence of records from a single Kafka
- * topic. The method allows to individually specify the expected starting offset and total read
- * value count of each partition. The job will be considered successful only if all partition
- * read results match the start offset and value count criteria.
- */
- protected void readSequence(
- final StreamExecutionEnvironment env,
- final StartupMode startupMode,
- final Map<KafkaTopicPartition, Long> specificStartupOffsets,
- final Long startupTimestamp,
- final Properties cc,
- final String topicName,
- final Map<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset)
- throws Exception {
- final int sourceParallelism = partitionsToValuesCountAndStartOffset.keySet().size();
-
- int finalCountTmp = 0;
- for (Map.Entry<Integer, Tuple2<Integer, Integer>> valuesCountAndStartOffset :
- partitionsToValuesCountAndStartOffset.entrySet()) {
- finalCountTmp += valuesCountAndStartOffset.getValue().f0;
- }
- final int finalCount = finalCountTmp;
-
- final TypeInformation<Tuple2<Integer, Integer>> intIntTupleType =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
-
- final TypeInformationSerializationSchema<Tuple2<Integer, Integer>> deser =
- new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig());
-
- // create the consumer
- cc.putAll(secureProps);
- DataStreamSource<Tuple2<Integer, Integer>> source;
- if (useNewSource) {
- KafkaSourceBuilder<Tuple2<Integer, Integer>> sourceBuilder =
- kafkaServer.getSourceBuilder(topicName, deser, cc);
- Map<TopicPartition, Long> startOffsets = new HashMap<>();
- if (specificStartupOffsets != null) {
- specificStartupOffsets.forEach(
- (ktp, offset) ->
- startOffsets.put(
- new TopicPartition(ktp.getTopic(), ktp.getPartition()),
- offset));
- }
- setKafkaSourceOffset(startupMode, sourceBuilder, startOffsets, startupTimestamp);
- source =
- env.fromSource(
- sourceBuilder.build(), WatermarkStrategy.noWatermarks(), "KafkaSource");
- } else {
- FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer =
- kafkaServer.getConsumer(topicName, deser, cc);
- setKafkaConsumerOffset(startupMode, consumer, specificStartupOffsets, startupTimestamp);
-
- source = env.addSource(consumer);
- }
-
- source.setParallelism(sourceParallelism)
- .map(new ThrottledMapper<>(20))
- .setParallelism(sourceParallelism)
- .flatMap(
- new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
- private HashMap<Integer, BitSet> partitionsToValueCheck;
- private int count = 0;
-
- @Override
- public void open(Configuration parameters) throws Exception {
- partitionsToValueCheck = new HashMap<>();
- for (Integer partition :
- partitionsToValuesCountAndStartOffset.keySet()) {
- partitionsToValueCheck.put(partition, new BitSet());
- }
- }
-
- @Override
- public void flatMap(
- Tuple2<Integer, Integer> value, Collector<Integer> out)
- throws Exception {
- int partition = value.f0;
- int val = value.f1;
-
- BitSet bitSet = partitionsToValueCheck.get(partition);
- if (bitSet == null) {
- throw new RuntimeException(
- "Got a record from an unknown partition");
- } else {
- bitSet.set(
- val
- - partitionsToValuesCountAndStartOffset.get(
- partition)
- .f1);
- }
-
- count++;
-
- LOG.info("Received message {}, total {} messages", value, count);
-
- // verify if we've seen everything
- if (count == finalCount) {
- for (Map.Entry<Integer, BitSet> partitionsToValueCheck :
- this.partitionsToValueCheck.entrySet()) {
- BitSet check = partitionsToValueCheck.getValue();
- int expectedValueCount =
- partitionsToValuesCountAndStartOffset.get(
- partitionsToValueCheck.getKey())
- .f0;
-
- if (check.cardinality() != expectedValueCount) {
- throw new RuntimeException(
- "Expected cardinality to be "
- + expectedValueCount
- + ", but was "
- + check.cardinality());
- } else if (check.nextClearBit(0) != expectedValueCount) {
- throw new RuntimeException(
- "Expected next clear bit to be "
- + expectedValueCount
- + ", but was "
- + check.cardinality());
- }
- }
-
- // test has passed
- throw new SuccessException();
- }
- }
- })
- .setParallelism(1);
-
- tryExecute(env, "Read data from Kafka");
-
- LOG.info("Successfully read sequence for verification");
- }
-
- /**
- * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, StartupMode,
- * Map, Long, Properties, String, Map)} to expect reading from the same start offset and the
- * same value count for all partitions of a single Kafka topic.
- */
- protected void readSequence(
- final StreamExecutionEnvironment env,
- final StartupMode startupMode,
- final Map<KafkaTopicPartition, Long> specificStartupOffsets,
- final Long startupTimestamp,
- final Properties cc,
- final int sourceParallelism,
- final String topicName,
- final int valuesCount,
- final int startFrom)
- throws Exception {
- HashMap<Integer, Tuple2<Integer, Integer>> partitionsToValuesCountAndStartOffset =
- new HashMap<>();
- for (int i = 0; i < sourceParallelism; i++) {
- partitionsToValuesCountAndStartOffset.put(i, new Tuple2<>(valuesCount, startFrom));
- }
- readSequence(
- env,
- startupMode,
- specificStartupOffsets,
- startupTimestamp,
- cc,
- topicName,
- partitionsToValuesCountAndStartOffset);
- }
-
- protected void setKafkaConsumerOffset(
- final StartupMode startupMode,
- final FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer,
- final Map<KafkaTopicPartition, Long> specificStartupOffsets,
- final Long startupTimestamp) {
- switch (startupMode) {
- case EARLIEST:
- consumer.setStartFromEarliest();
- break;
- case LATEST:
- consumer.setStartFromLatest();
- break;
- case SPECIFIC_OFFSETS:
- consumer.setStartFromSpecificOffsets(specificStartupOffsets);
- break;
- case GROUP_OFFSETS:
- consumer.setStartFromGroupOffsets();
- break;
- case TIMESTAMP:
- consumer.setStartFromTimestamp(startupTimestamp);
- break;
- }
- }
-
- protected void setKafkaSourceOffset(
- final StartupMode startupMode,
- final KafkaSourceBuilder<?> kafkaSourceBuilder,
- final Map<TopicPartition, Long> specificStartupOffsets,
- final Long startupTimestamp) {
- switch (startupMode) {
- case EARLIEST:
- kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.earliest());
- break;
- case LATEST:
- kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.latest());
- break;
- case SPECIFIC_OFFSETS:
- kafkaSourceBuilder.setStartingOffsets(
- OffsetsInitializer.offsets(specificStartupOffsets));
- break;
- case GROUP_OFFSETS:
- kafkaSourceBuilder.setStartingOffsets(
- OffsetsInitializer.committedOffsets(OffsetResetStrategy.EARLIEST));
- break;
- case TIMESTAMP:
- kafkaSourceBuilder.setStartingOffsets(
- OffsetsInitializer.timestamp(startupTimestamp));
- break;
- }
- }
-
- protected String writeSequence(
- String baseTopicName,
- final int numElements,
- final int parallelism,
- final int replicationFactor)
- throws Exception {
- LOG.info(
- "\n===================================\n"
- + "== Writing sequence of "
- + numElements
- + " into "
- + baseTopicName
- + " with p="
- + parallelism
- + "\n"
- + "===================================");
-
- final TypeInformation<Tuple2<Integer, Integer>> resultType =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
-
- final SerializationSchema<Tuple2<Integer, Integer>> serSchema =
- new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig());
-
- final KafkaDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
- new KafkaDeserializationSchemaWrapper<>(
- new TypeInformationSerializationSchema<>(
- resultType, new ExecutionConfig()));
-
- final int maxNumAttempts = 10;
-
- for (int attempt = 1; attempt <= maxNumAttempts; attempt++) {
-
- final String topicName = baseTopicName + '-' + attempt + '-' + UUID.randomUUID();
-
- LOG.info("Writing attempt #" + attempt);
-
- // -------- Write the Sequence --------
-
- createTestTopic(topicName, parallelism, replicationFactor);
-
- StreamExecutionEnvironment writeEnv =
- StreamExecutionEnvironment.getExecutionEnvironment();
- writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- DataStream<Tuple2<Integer, Integer>> stream =
- writeEnv.addSource(
- new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Integer, Integer>> ctx)
- throws Exception {
- int cnt = 0;
- int partition =
- getRuntimeContext().getIndexOfThisSubtask();
-
- while (running && cnt < numElements) {
- ctx.collect(new Tuple2<>(partition, cnt));
- cnt++;
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- })
- .setParallelism(parallelism);
-
- // the producer must not produce duplicates
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
- producerProperties.setProperty("retries", "0");
- producerProperties.putAll(secureProps);
-
- kafkaServer
- .produceIntoKafka(
- stream,
- topicName,
- serSchema,
- producerProperties,
- new Tuple2FlinkPartitioner(parallelism))
- .setParallelism(parallelism);
-
- try {
- writeEnv.execute("Write sequence");
- } catch (Exception e) {
- LOG.error("Write attempt failed, trying again", e);
- deleteTestTopic(topicName);
- waitUntilNoJobIsRunning(client);
- continue;
- }
-
- LOG.info("Finished writing sequence");
-
- // -------- Validate the Sequence --------
-
- // we need to validate the sequence, because kafka's producers are not exactly once
- LOG.info("Validating sequence");
-
- waitUntilNoJobIsRunning(client);
-
- if (validateSequence(topicName, parallelism, deserSchema, numElements)) {
- // everything is good!
- return topicName;
- } else {
- deleteTestTopic(topicName);
- // fall through the loop
- }
- }
-
- throw new Exception(
- "Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts");
- }
-
- protected void writeAppendSequence(
- String topicName,
- final int originalNumElements,
- final int numElementsToAppend,
- final int parallelism)
- throws Exception {
-
- LOG.info(
- "\n===================================\n"
- + "== Appending sequence of "
- + numElementsToAppend
- + " into "
- + topicName
- + "===================================");
-
- final TypeInformation<Tuple2<Integer, Integer>> resultType =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
-
- final SerializationSchema<Tuple2<Integer, Integer>> serSchema =
- new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig());
-
- final KafkaDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
- new KafkaDeserializationSchemaWrapper<>(
- new TypeInformationSerializationSchema<>(
- resultType, new ExecutionConfig()));
-
- // -------- Write the append sequence --------
-
- StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- DataStream<Tuple2<Integer, Integer>> stream =
- writeEnv.addSource(
- new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Integer, Integer>> ctx)
- throws Exception {
- int cnt = originalNumElements;
- int partition = getRuntimeContext().getIndexOfThisSubtask();
-
- while (running
- && cnt
- < numElementsToAppend
- + originalNumElements) {
- ctx.collect(new Tuple2<>(partition, cnt));
- cnt++;
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- })
- .setParallelism(parallelism);
-
- // the producer must not produce duplicates
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
- producerProperties.setProperty("retries", "0");
- producerProperties.putAll(secureProps);
-
- kafkaServer
- .produceIntoKafka(
- stream,
- topicName,
- serSchema,
- producerProperties,
- new Tuple2FlinkPartitioner(parallelism))
- .setParallelism(parallelism);
-
- try {
- writeEnv.execute("Write sequence");
- } catch (Exception e) {
- throw new Exception("Failed to append sequence to Kafka; append job failed.", e);
- }
-
- LOG.info("Finished writing append sequence");
-
- // we need to validate the sequence, because kafka's producers are not exactly once
- LOG.info("Validating sequence");
- while (!getRunningJobs(client).isEmpty()) {
- Thread.sleep(50);
- }
-
- if (!validateSequence(
- topicName, parallelism, deserSchema, originalNumElements + numElementsToAppend)) {
- throw new Exception("Could not append a valid sequence to Kafka.");
- }
- }
-
- private boolean validateSequence(
- final String topic,
- final int parallelism,
- KafkaDeserializationSchema<Tuple2<Integer, Integer>> deserSchema,
- final int totalNumElements)
- throws Exception {
-
- final StreamExecutionEnvironment readEnv =
- StreamExecutionEnvironment.getExecutionEnvironment();
- readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
- readEnv.setParallelism(parallelism);
-
- Properties readProps = (Properties) standardProps.clone();
- readProps.setProperty("group.id", "flink-tests-validator");
- readProps.putAll(secureProps);
- DataStreamSource<Tuple2<Integer, Integer>> dataStreamSource;
-
- if (useNewSource) {
- KafkaSource<Tuple2<Integer, Integer>> source =
- kafkaServer
- .getSourceBuilder(topic, deserSchema, readProps)
- .setStartingOffsets(OffsetsInitializer.earliest())
- .build();
- dataStreamSource =
- readEnv.fromSource(source, WatermarkStrategy.noWatermarks(), "KafkaSource");
- } else {
- FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer =
- kafkaServer.getConsumer(topic, deserSchema, readProps);
- consumer.setStartFromEarliest();
- dataStreamSource = readEnv.addSource(consumer);
- }
-
- dataStreamSource
- .map(
- new RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>() {
-
- private final int totalCount = parallelism * totalNumElements;
- private int count = 0;
-
- @Override
- public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value)
- throws Exception {
- if (++count == totalCount) {
- throw new SuccessException();
- } else {
- return value;
- }
- }
- })
- .setParallelism(1)
- .addSink(new DiscardingSink<>())
- .setParallelism(1);
-
- final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-
- JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(readEnv.getStreamGraph());
- final JobID jobId = jobGraph.getJobID();
-
- Thread runner =
- new Thread(
- () -> {
- try {
- submitJobAndWaitForResult(
- client, jobGraph, getClass().getClassLoader());
- tryExecute(readEnv, "sequence validation");
- } catch (Throwable t) {
- if (!ExceptionUtils.findThrowable(t, SuccessException.class)
- .isPresent()) {
- errorRef.set(t);
- }
- }
- });
- runner.start();
-
- final long deadline = System.nanoTime() + 10_000_000_000L;
- long delay;
- while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) {
- runner.join(delay / 1_000_000L);
- }
-
- boolean success;
-
- if (runner.isAlive()) {
- // did not finish in time, maybe the producer dropped one or more records and
- // the validation did not reach the exit point
- success = false;
- client.cancel(jobId).get();
- } else {
- Throwable error = errorRef.get();
- if (error != null) {
- success = false;
- LOG.info("Sequence validation job failed with exception", error);
- } else {
- success = true;
- }
- }
-
- waitUntilNoJobIsRunning(client);
-
- return success;
- }
-
- private <T> DataStreamSource<T> getStream(
- StreamExecutionEnvironment env,
- String topic,
- DeserializationSchema<T> schema,
- Properties props) {
- return getStream(env, Collections.singletonList(topic), schema, props);
- }
-
- private <T> DataStreamSource<T> getStream(
- StreamExecutionEnvironment env,
- String topic,
- KafkaDeserializationSchema<T> schema,
- Properties props) {
- return getStream(env, Collections.singletonList(topic), schema, props);
- }
-
- private <T> DataStreamSource<T> getStream(
- StreamExecutionEnvironment env,
- List<String> topics,
- DeserializationSchema<T> schema,
- Properties props) {
- if (useNewSource) {
- KafkaSource<T> kafkaSource =
- kafkaServer.getSourceBuilder(topics, schema, props).build();
- return env.fromSource(kafkaSource, WatermarkStrategy.noWatermarks(), "KafkaSource");
- } else {
- FlinkKafkaConsumerBase<T> flinkKafkaConsumer =
- kafkaServer.getConsumer(topics, schema, props);
- return env.addSource(flinkKafkaConsumer);
- }
- }
-
- private <T> DataStreamSource<T> getStream(
- StreamExecutionEnvironment env,
- List<String> topics,
- KafkaDeserializationSchema<T> schema,
- Properties props) {
- if (useNewSource) {
- KafkaSource<T> kafkaSource =
- kafkaServer.getSourceBuilder(topics, schema, props).build();
- return env.fromSource(kafkaSource, WatermarkStrategy.noWatermarks(), "KafkaSource");
- } else {
- FlinkKafkaConsumerBase<T> flinkKafkaConsumer =
- kafkaServer.getConsumer(topics, schema, props);
- return env.addSource(flinkKafkaConsumer);
- }
- }
-
- // ------------------------------------------------------------------------
- // Debugging utilities
- // ------------------------------------------------------------------------
-
- private static class BrokerKillingMapper<T> extends RichMapFunction<T, T>
- implements ListCheckpointed<Integer>, CheckpointListener {
-
- private static final long serialVersionUID = 6334389850158707313L;
-
- public static volatile boolean killedLeaderBefore;
- public static volatile boolean hasBeenCheckpointedBeforeFailure;
-
- private static KafkaTestEnvironment kafkaServerToKill;
- private final int shutdownBrokerId;
- private final int failCount;
- private int numElementsTotal;
-
- private boolean failer;
- private boolean hasBeenCheckpointed;
-
- public BrokerKillingMapper(
- KafkaTestEnvironment kafkaServer, int shutdownBrokerId, int failCount) {
- kafkaServerToKill = kafkaServer;
- this.shutdownBrokerId = shutdownBrokerId;
- this.failCount = failCount;
- }
-
- @Override
- public void open(Configuration parameters) {
- failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
- }
-
- @Override
- public T map(T value) throws Exception {
- numElementsTotal++;
-
- if (!killedLeaderBefore) {
- Thread.sleep(10);
-
- if (failer && numElementsTotal >= failCount) {
- // shut down a Kafka broker
- kafkaServerToKill.stopBroker(shutdownBrokerId);
- hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
- killedLeaderBefore = true;
- }
- }
- return value;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) {
- hasBeenCheckpointed = true;
- }
-
- @Override
- public void notifyCheckpointAborted(long checkpointId) {}
-
- @Override
- public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
- return Collections.singletonList(this.numElementsTotal);
- }
-
- @Override
- public void restoreState(List<Integer> state) throws Exception {
- if (state.isEmpty() || state.size() > 1) {
- throw new RuntimeException(
- "Test failed due to unexpected recovered state size " + state.size());
- }
- this.numElementsTotal = state.get(0);
- }
- }
-
- private abstract static class AbstractTestDeserializer
- implements KafkaDeserializationSchema<Tuple3<Integer, Integer, String>> {
-
- protected final TypeSerializer<Tuple2<Integer, Integer>> ts;
-
- public AbstractTestDeserializer(ExecutionConfig ec) {
- ts =
- TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {})
- .createSerializer(ec);
- }
-
- @Override
- public Tuple3<Integer, Integer, String> deserialize(ConsumerRecord<byte[], byte[]> record)
- throws Exception {
- DataInputView in =
- new DataInputViewStreamWrapper(new ByteArrayInputStream(record.value()));
- Tuple2<Integer, Integer> t2 = ts.deserialize(in);
- return new Tuple3<>(t2.f0, t2.f1, record.topic());
- }
-
- @Override
- public boolean isEndOfStream(Tuple3<Integer, Integer, String> nextElement) {
- return false;
- }
-
- @Override
- public TypeInformation<Tuple3<Integer, Integer, String>> getProducedType() {
- return TypeInformation.of(new TypeHint<Tuple3<Integer, Integer, String>>() {});
- }
- }
-
- private static class Tuple2WithTopicSchema extends AbstractTestDeserializer
- implements KeyedSerializationSchema<Tuple3<Integer, Integer, String>> {
-
- public Tuple2WithTopicSchema(ExecutionConfig ec) {
- super(ec);
- }
-
- @Override
- public byte[] serializeKey(Tuple3<Integer, Integer, String> element) {
- return null;
- }
-
- @Override
- public byte[] serializeValue(Tuple3<Integer, Integer, String> element) {
- ByteArrayOutputStream by = new ByteArrayOutputStream();
- DataOutputView out = new DataOutputViewStreamWrapper(by);
- try {
- ts.serialize(new Tuple2<>(element.f0, element.f1), out);
- } catch (IOException e) {
- throw new RuntimeException("Error", e);
- }
- return by.toByteArray();
- }
-
- @Override
- public String getTargetTopic(Tuple3<Integer, Integer, String> element) {
- return element.f2;
- }
- }
-
- private static class TestDeserializer extends AbstractTestDeserializer
- implements KafkaSerializationSchema<Tuple3<Integer, Integer, String>> {
-
- public TestDeserializer(ExecutionConfig ec) {
- super(ec);
- }
-
- @Override
- public ProducerRecord<byte[], byte[]> serialize(
- Tuple3<Integer, Integer, String> element, @Nullable Long timestamp) {
- ByteArrayOutputStream by = new ByteArrayOutputStream();
- DataOutputView out = new DataOutputViewStreamWrapper(by);
- try {
- ts.serialize(new Tuple2<>(element.f0, element.f1), out);
- } catch (IOException e) {
- throw new RuntimeException("Error", e);
- }
- byte[] serializedValue = by.toByteArray();
-
- return new ProducerRecord<>(element.f2, serializedValue);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
deleted file mode 100644
index 68db691..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ /dev/null
@@ -1,380 +0,0 @@
-/*
- * 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.api.common.ExecutionConfig;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.util.Optional;
-import java.util.UUID;
-
-/** IT cases for Kafka. */
-public class KafkaITCase extends KafkaConsumerTestBase {
-
- @BeforeClass
- public static void prepare() throws Exception {
- KafkaProducerTestBase.prepare();
- ((KafkaTestEnvironmentImpl) kafkaServer)
- .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
- }
-
- // ------------------------------------------------------------------------
- // Suite of Tests
- // ------------------------------------------------------------------------
-
- @Test(timeout = 120000)
- public void testFailOnNoBroker() throws Exception {
- runFailOnNoBrokerTest();
- }
-
- @Test(timeout = 60000)
- public void testConcurrentProducerConsumerTopology() throws Exception {
- runSimpleConcurrentProducerConsumerTopology();
- }
-
- @Test(timeout = 60000)
- public void testKeyValueSupport() throws Exception {
- runKeyValueTest();
- }
-
- // --- canceling / failures ---
-
- @Test(timeout = 60000)
- public void testCancelingEmptyTopic() throws Exception {
- runCancelingOnEmptyInputTest();
- }
-
- @Test(timeout = 60000)
- public void testCancelingFullTopic() throws Exception {
- runCancelingOnFullInputTest();
- }
-
- // --- source to partition mappings and exactly once ---
-
- @Test(timeout = 60000)
- public void testOneToOneSources() throws Exception {
- runOneToOneExactlyOnceTest();
- }
-
- @Test(timeout = 60000)
- public void testOneSourceMultiplePartitions() throws Exception {
- runOneSourceMultiplePartitionsExactlyOnceTest();
- }
-
- @Test(timeout = 60000)
- public void testMultipleSourcesOnePartition() throws Exception {
- runMultipleSourcesOnePartitionExactlyOnceTest();
- }
-
- // --- broker failure ---
-
- @Test(timeout = 60000)
- public void testBrokerFailure() throws Exception {
- runBrokerFailureTest();
- }
-
- // --- special executions ---
-
- @Test(timeout = 60000)
- public void testBigRecordJob() throws Exception {
- runBigRecordTestTopology();
- }
-
- @Test(timeout = 60000)
- public void testMultipleTopicsWithLegacySerializer() throws Exception {
- runProduceConsumeMultipleTopics(true);
- }
-
- @Test(timeout = 60000)
- public void testMultipleTopicsWithKafkaSerializer() throws Exception {
- runProduceConsumeMultipleTopics(false);
- }
-
- @Test(timeout = 60000)
- public void testAllDeletes() throws Exception {
- runAllDeletesTest();
- }
-
- @Test(timeout = 60000)
- public void testMetricsAndEndOfStream() throws Exception {
- runEndOfStreamTest();
- }
-
- // --- startup mode ---
-
- @Test(timeout = 60000)
- public void testStartFromEarliestOffsets() throws Exception {
- runStartFromEarliestOffsets();
- }
-
- @Test(timeout = 60000)
- public void testStartFromLatestOffsets() throws Exception {
- runStartFromLatestOffsets();
- }
-
- @Test(timeout = 60000)
- public void testStartFromGroupOffsets() throws Exception {
- runStartFromGroupOffsets();
- }
-
- @Test(timeout = 60000)
- public void testStartFromSpecificOffsets() throws Exception {
- runStartFromSpecificOffsets();
- }
-
- @Test(timeout = 60000)
- public void testStartFromTimestamp() throws Exception {
- runStartFromTimestamp();
- }
-
- // --- offset committing ---
-
- @Test(timeout = 60000)
- public void testCommitOffsetsToKafka() throws Exception {
- runCommitOffsetsToKafka();
- }
-
- @Test(timeout = 60000)
- public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception {
- runAutoOffsetRetrievalAndCommitToKafka();
- }
-
- @Test(timeout = 60000)
- public void testCollectingSchema() throws Exception {
- runCollectingSchemaTest();
- }
-
- /** Kafka 20 specific test, ensuring Timestamps are properly written to and read from Kafka. */
- @Test(timeout = 60000)
- public void testTimestamps() throws Exception {
-
- final String topic = "tstopic-" + UUID.randomUUID();
- createTestTopic(topic, 3, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- DataStream<Long> streamWithTimestamps =
- env.addSource(
- new SourceFunction<Long>() {
- private static final long serialVersionUID = -2255115836471289626L;
- boolean running = true;
-
- @Override
- public void run(SourceContext<Long> ctx) throws Exception {
- long i = 0;
- while (running) {
- ctx.collectWithTimestamp(i, i * 2);
- if (i++ == 1110L) {
- running = false;
- }
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
-
- final TypeInformationSerializationSchema<Long> longSer =
- new TypeInformationSerializationSchema<>(Types.LONG, env.getConfig());
- FlinkKafkaProducer<Long> prod =
- new FlinkKafkaProducer<>(
- topic,
- new KeyedSerializationSchemaWrapper<>(longSer),
- standardProps,
- Optional.of(
- new FlinkKafkaPartitioner<Long>() {
- private static final long serialVersionUID =
- -6730989584364230617L;
-
- @Override
- public int partition(
- Long next,
- byte[] key,
- byte[] value,
- String targetTopic,
- int[] partitions) {
- return (int) (next % 3);
- }
- }));
- prod.setWriteTimestampToKafka(true);
-
- streamWithTimestamps.addSink(prod).setParallelism(3);
-
- env.execute("Produce some");
-
- // ---------- Consume stream from Kafka -------------------
-
- env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- FlinkKafkaConsumer<Long> kafkaSource =
- new FlinkKafkaConsumer<>(
- topic, new KafkaITCase.LimitedLongDeserializer(), standardProps);
- kafkaSource.assignTimestampsAndWatermarks(
- new AssignerWithPunctuatedWatermarks<Long>() {
- private static final long serialVersionUID = -4834111173247835189L;
-
- @Nullable
- @Override
- public Watermark checkAndGetNextWatermark(
- Long lastElement, long extractedTimestamp) {
- if (lastElement % 11 == 0) {
- return new Watermark(lastElement);
- }
- return null;
- }
-
- @Override
- public long extractTimestamp(Long element, long previousElementTimestamp) {
- return previousElementTimestamp;
- }
- });
-
- DataStream<Long> stream = env.addSource(kafkaSource);
- GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
- stream.transform(
- "timestamp validating operator",
- objectTypeInfo,
- new TimestampValidatingOperator())
- .setParallelism(1);
-
- env.execute("Consume again");
-
- deleteTestTopic(topic);
- }
-
- private static class TimestampValidatingOperator extends StreamSink<Long> {
-
- private static final long serialVersionUID = 1353168781235526806L;
-
- public TimestampValidatingOperator() {
- super(
- new SinkFunction<Long>() {
- private static final long serialVersionUID = -6676565693361786524L;
-
- @Override
- public void invoke(Long value) throws Exception {
- throw new RuntimeException("Unexpected");
- }
- });
- }
-
- long elCount = 0;
- long wmCount = 0;
- long lastWM = Long.MIN_VALUE;
-
- @Override
- public void processElement(StreamRecord<Long> element) throws Exception {
- elCount++;
- if (element.getValue() * 2 != element.getTimestamp()) {
- throw new RuntimeException("Invalid timestamp: " + element);
- }
- }
-
- @Override
- public void processWatermark(Watermark mark) throws Exception {
- wmCount++;
-
- if (lastWM <= mark.getTimestamp()) {
- lastWM = mark.getTimestamp();
- } else {
- throw new RuntimeException("Received watermark higher than the last one");
- }
-
- if (mark.getTimestamp() % 11 != 0 && mark.getTimestamp() != Long.MAX_VALUE) {
- throw new RuntimeException("Invalid watermark: " + mark.getTimestamp());
- }
- }
-
- @Override
- public void close() throws Exception {
- super.close();
- if (elCount != 1110L) {
- throw new RuntimeException("Wrong final element count " + elCount);
- }
-
- if (wmCount <= 2) {
- throw new RuntimeException("Almost no watermarks have been sent " + wmCount);
- }
- }
- }
-
- private static class LimitedLongDeserializer implements KafkaDeserializationSchema<Long> {
-
- private static final long serialVersionUID = 6966177118923713521L;
- private final TypeInformation<Long> ti;
- private final TypeSerializer<Long> ser;
- long cnt = 0;
-
- public LimitedLongDeserializer() {
- this.ti = Types.LONG;
- this.ser = ti.createSerializer(new ExecutionConfig());
- }
-
- @Override
- public TypeInformation<Long> getProducedType() {
- return ti;
- }
-
- @Override
- public Long deserialize(ConsumerRecord<byte[], byte[]> record) throws IOException {
- cnt++;
- DataInputView in =
- new DataInputViewStreamWrapper(new ByteArrayInputStream(record.value()));
- Long e = ser.deserialize(in);
- return e;
- }
-
- @Override
- public boolean isEndOfStream(Long nextElement) {
- return cnt > 1110L;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java
deleted file mode 100644
index 540c625..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * 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.FlinkVersion;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.test.util.MigrationTest;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** The base class with migration tests for the Kafka Exactly-Once Producer. */
-@SuppressWarnings("serial")
-public abstract class KafkaMigrationTestBase extends KafkaTestBase implements MigrationTest {
-
- protected static final Logger LOG = LoggerFactory.getLogger(KafkaMigrationTestBase.class);
- protected static final String TOPIC = "flink-kafka-producer-migration-test";
-
- protected final FlinkVersion testMigrateVersion;
- protected final TypeInformationSerializationSchema<Integer> integerSerializationSchema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
- protected final KeyedSerializationSchema<Integer> integerKeyedSerializationSchema =
- new KeyedSerializationSchemaWrapper<>(integerSerializationSchema);
-
- public KafkaMigrationTestBase(FlinkVersion testMigrateVersion) {
- this.testMigrateVersion = checkNotNull(testMigrateVersion);
- }
-
- public String getOperatorSnapshotPath() {
- return getOperatorSnapshotPath(testMigrateVersion);
- }
-
- public String getOperatorSnapshotPath(FlinkVersion version) {
- return "src/test/resources/kafka-migration-kafka-producer-flink-" + version + "-snapshot";
- }
-
- /**
- * Override {@link KafkaTestBase}. Kafka Migration Tests are starting up Kafka/ZooKeeper cluster
- * manually
- */
- @BeforeClass
- public static void prepare() throws Exception {}
-
- /**
- * Override {@link KafkaTestBase}. Kafka Migration Tests are starting up Kafka/ZooKeeper cluster
- * manually
- */
- @AfterClass
- public static void shutDownServices() throws Exception {}
-
- @SnapshotsGenerator
- public void writeSnapshot(FlinkVersion flinkGenerateSavepointVersion) throws Exception {
- try {
- startClusters();
-
- OperatorSubtaskState snapshot = initializeTestState();
- OperatorSnapshotUtil.writeStateHandle(
- snapshot, getOperatorSnapshotPath(flinkGenerateSavepointVersion));
- } finally {
- shutdownClusters();
- }
- }
-
- private OperatorSubtaskState initializeTestState() throws Exception {
- try (OneInputStreamOperatorTestHarness testHarness = createTestHarness()) {
- testHarness.setup();
- testHarness.open();
-
- // Create a committed transaction
- testHarness.processElement(42, 0L);
-
- // TODO: when stop with savepoint is available, replace this code with it (with stop
- // with savepoint
- // there won't be any pending transactions)
- OperatorSubtaskState snapshot = testHarness.snapshot(0L, 1L);
- // We kind of simulate stop with savepoint by making sure that
- // notifyOfCompletedCheckpoint is called
- testHarness.notifyOfCompletedCheckpoint(0L);
-
- // Create a Pending transaction
- testHarness.processElement(43, 2L);
- return snapshot;
- }
- }
-
- @SuppressWarnings("warning")
- @Test
- public void testRestoreProducer() throws Exception {
- try {
- startClusters();
-
- initializeTestState();
-
- try (OneInputStreamOperatorTestHarness testHarness = createTestHarness()) {
- initializeState(testHarness);
-
- // Create a committed transaction
- testHarness.processElement(44, 4L);
- testHarness.snapshot(2L, 5L);
- testHarness.notifyOfCompletedCheckpoint(2L);
-
- // Create a pending transaction
- testHarness.processElement(45, 6L);
-
- // We should have:
- // - committed transaction 42
- // - transaction 43 aborted
- // - committed transaction 44
- // - transaction 45 pending
- assertExactlyOnceForTopic(createProperties(), TOPIC, Arrays.asList(42, 44));
- }
- } finally {
- shutdownClusters();
- }
- }
-
- protected abstract OneInputStreamOperatorTestHarness<Integer, Object> createTestHarness()
- throws Exception;
-
- protected abstract Properties createProperties();
-
- protected void initializeState(OneInputStreamOperatorTestHarness testHarness) throws Exception {
- testHarness.setup();
- testHarness.initializeState(getOperatorSnapshotPath());
- testHarness.open();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java
deleted file mode 100644
index aae2680..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.junit.BeforeClass;
-
-/** IT cases for the {@link FlinkKafkaProducer}. */
-@SuppressWarnings("serial")
-public class KafkaProducerAtLeastOnceITCase extends KafkaProducerTestBase {
-
- @BeforeClass
- public static void prepare() throws Exception {
- KafkaProducerTestBase.prepare();
- ((KafkaTestEnvironmentImpl) kafkaServer)
- .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE);
- }
-
- @Override
- public void testExactlyOnceRegularSink() throws Exception {
- // disable test for at least once semantic
- }
-
- @Override
- public void testExactlyOnceCustomOperator() throws Exception {
- // disable test for at least once semantic
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java
deleted file mode 100644
index f8b20ae..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.junit.BeforeClass;
-import org.junit.Test;
-
-/** IT cases for the {@link FlinkKafkaProducer}. */
-@SuppressWarnings("serial")
-public class KafkaProducerExactlyOnceITCase extends KafkaProducerTestBase {
- @BeforeClass
- public static void prepare() throws Exception {
- KafkaProducerTestBase.prepare();
- ((KafkaTestEnvironmentImpl) kafkaServer)
- .setProducerSemantic(FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- }
-
- @Test
- public void testMultipleSinkOperators() throws Exception {
- testExactlyOnce(false, 2);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
deleted file mode 100644
index ff3ad2b..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * 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.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.IntegerSource;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.test.util.TestUtils;
-import org.apache.flink.util.Preconditions;
-
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Abstract test base for all Kafka producer tests. */
-@SuppressWarnings("serial")
-public abstract class KafkaProducerTestBase extends KafkaTestBaseWithFlink {
-
- /**
- * This tests verifies that custom partitioning works correctly, with a default topic and
- * dynamic topic. The number of partitions for each topic is deliberately different.
- *
- * <p>Test topology:
- *
- * <pre>
- * +------> (sink) --+--> [DEFAULT_TOPIC-1] --> (source) -> (map) -----+
- * / | | | |
- * | | | | ------+--> (sink)
- * +------> (sink) --+--> [DEFAULT_TOPIC-2] --> (source) -> (map) -----+
- * / |
- * | |
- * (source) ----------> (sink) --+--> [DYNAMIC_TOPIC-1] --> (source) -> (map) -----+
- * | | | | |
- * \ | | | |
- * +------> (sink) --+--> [DYNAMIC_TOPIC-2] --> (source) -> (map) -----+--> (sink)
- * | | | | |
- * \ | | | |
- * +------> (sink) --+--> [DYNAMIC_TOPIC-3] --> (source) -> (map) -----+
- * </pre>
- *
- * <p>Each topic has an independent mapper that validates the values come consistently from the
- * correct Kafka partition of the topic is is responsible of.
- *
- * <p>Each topic also has a final sink that validates that there are no duplicates and that all
- * partitions are present.
- */
- @Test
- public void testCustomPartitioning() {
- try {
- LOG.info("Starting KafkaProducerITCase.testCustomPartitioning()");
-
- final String defaultTopic = "defaultTopic-" + UUID.randomUUID();
- final int defaultTopicPartitions = 2;
-
- final String dynamicTopic = "dynamicTopic-" + UUID.randomUUID();
- final int dynamicTopicPartitions = 3;
-
- createTestTopic(defaultTopic, defaultTopicPartitions, 1);
- createTestTopic(dynamicTopic, dynamicTopicPartitions, 1);
-
- Map<String, Integer> expectedTopicsToNumPartitions = new HashMap<>(2);
- expectedTopicsToNumPartitions.put(defaultTopic, defaultTopicPartitions);
- expectedTopicsToNumPartitions.put(dynamicTopic, dynamicTopicPartitions);
-
- TypeInformation<Tuple2<Long, String>> longStringInfo =
- TypeInformation.of(new TypeHint<Tuple2<Long, String>>() {});
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- TypeInformationSerializationSchema<Tuple2<Long, String>> serSchema =
- new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
-
- TypeInformationSerializationSchema<Tuple2<Long, String>> deserSchema =
- new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
-
- // ------ producing topology ---------
-
- // source has DOP 1 to make sure it generates no duplicates
- DataStream<Tuple2<Long, String>> stream =
- env.addSource(
- new SourceFunction<Tuple2<Long, String>>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<Tuple2<Long, String>> ctx)
- throws Exception {
- long cnt = 0;
- while (running) {
- ctx.collect(
- new Tuple2<Long, String>(
- cnt, "kafka-" + cnt));
- cnt++;
- if (cnt % 100 == 0) {
- Thread.sleep(1);
- }
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- })
- .setParallelism(1);
-
- Properties props = new Properties();
- props.putAll(
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings));
- props.putAll(secureProps);
-
- // sink partitions into
- kafkaServer
- .produceIntoKafka(
- stream,
- defaultTopic,
- // this serialization schema will route between the default topic and
- // dynamic topic
- new CustomKeyedSerializationSchemaWrapper(
- serSchema, defaultTopic, dynamicTopic),
- props,
- new CustomPartitioner(expectedTopicsToNumPartitions))
- .setParallelism(Math.max(defaultTopicPartitions, dynamicTopicPartitions));
-
- // ------ consuming topology ---------
-
- Properties consumerProps = new Properties();
- consumerProps.putAll(standardProps);
- consumerProps.putAll(secureProps);
-
- FlinkKafkaConsumerBase<Tuple2<Long, String>> defaultTopicSource =
- kafkaServer.getConsumer(defaultTopic, deserSchema, consumerProps);
- FlinkKafkaConsumerBase<Tuple2<Long, String>> dynamicTopicSource =
- kafkaServer.getConsumer(dynamicTopic, deserSchema, consumerProps);
-
- env.addSource(defaultTopicSource)
- .setParallelism(defaultTopicPartitions)
- .map(new PartitionValidatingMapper(defaultTopicPartitions))
- .setParallelism(defaultTopicPartitions)
- .addSink(new PartitionValidatingSink(defaultTopicPartitions))
- .setParallelism(1);
-
- env.addSource(dynamicTopicSource)
- .setParallelism(dynamicTopicPartitions)
- .map(new PartitionValidatingMapper(dynamicTopicPartitions))
- .setParallelism(dynamicTopicPartitions)
- .addSink(new PartitionValidatingSink(dynamicTopicPartitions))
- .setParallelism(1);
-
- tryExecute(env, "custom partitioning test");
-
- deleteTestTopic(defaultTopic);
- deleteTestTopic(dynamicTopic);
-
- LOG.info("Finished KafkaProducerITCase.testCustomPartitioning()");
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- /** Tests the exactly-once semantic for the simple writes into Kafka. */
- @Test
- public void testExactlyOnceRegularSink() throws Exception {
- testExactlyOnce(true, 1);
- }
-
- /** Tests the exactly-once semantic for the simple writes into Kafka. */
- @Test
- public void testExactlyOnceCustomOperator() throws Exception {
- testExactlyOnce(false, 1);
- }
-
- /**
- * This test sets KafkaProducer so that it will automatically flush the data and fails the
- * broker to check whether flushed records since last checkpoint were not duplicated.
- */
- protected void testExactlyOnce(boolean regularSink, int sinksCount) throws Exception {
- final String topicNamePrefix =
- (regularSink ? "exactlyOnceTopicRegularSink" : "exactlyTopicCustomOperator")
- + sinksCount;
- final int partition = 0;
- final int numElements = 1000;
- final int failAfterElements = 333;
-
- final List<String> topics = new ArrayList<>();
- for (int i = 0; i < sinksCount; i++) {
- final String topic = topicNamePrefix + i + "-" + UUID.randomUUID();
- topics.add(topic);
- createTestTopic(topic, 1, 1);
- }
-
- TypeInformationSerializationSchema<Integer> schema =
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.enableCheckpointing(500);
- env.setParallelism(1);
- env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
-
- Properties properties = new Properties();
- properties.putAll(standardProps);
- properties.putAll(secureProps);
-
- // process exactly failAfterElements number of elements and then shutdown Kafka broker and
- // fail application
- List<Integer> expectedElements = getIntegersSequence(numElements);
-
- DataStream<Integer> inputStream =
- env.addSource(new IntegerSource(numElements))
- .map(new FailingIdentityMapper<Integer>(failAfterElements));
-
- for (int i = 0; i < sinksCount; i++) {
- FlinkKafkaPartitioner<Integer> partitioner =
- new FlinkKafkaPartitioner<Integer>() {
- @Override
- public int partition(
- Integer record,
- byte[] key,
- byte[] value,
- String targetTopic,
- int[] partitions) {
- return partition;
- }
- };
-
- if (regularSink) {
- StreamSink<Integer> kafkaSink =
- kafkaServer.getProducerSink(topics.get(i), schema, properties, partitioner);
- inputStream.addSink(kafkaSink.getUserFunction());
- } else {
- kafkaServer.produceIntoKafka(
- inputStream, topics.get(i), schema, properties, partitioner);
- }
- }
-
- FailingIdentityMapper.failedBefore = false;
- TestUtils.tryExecute(env, "Exactly once test");
-
- for (int i = 0; i < sinksCount; i++) {
- // assert that before failure we successfully snapshot/flushed all expected elements
- assertExactlyOnceForTopic(properties, topics.get(i), expectedElements);
- deleteTestTopic(topics.get(i));
- }
- }
-
- private List<Integer> getIntegersSequence(int size) {
- List<Integer> result = new ArrayList<>(size);
- for (int i = 0; i < size; i++) {
- result.add(i);
- }
- return result;
- }
-
- // ------------------------------------------------------------------------
-
- private static class CustomPartitioner extends FlinkKafkaPartitioner<Tuple2<Long, String>>
- implements Serializable {
-
- private final Map<String, Integer> expectedTopicsToNumPartitions;
-
- public CustomPartitioner(Map<String, Integer> expectedTopicsToNumPartitions) {
- this.expectedTopicsToNumPartitions = expectedTopicsToNumPartitions;
- }
-
- @Override
- public int partition(
- Tuple2<Long, String> next,
- byte[] serializedKey,
- byte[] serializedValue,
- String topic,
- int[] partitions) {
- assertThat(partitions).hasSize(expectedTopicsToNumPartitions.get(topic).intValue());
-
- return (int) (next.f0 % partitions.length);
- }
- }
-
- /**
- * A {@link KeyedSerializationSchemaWrapper} that supports routing serialized records to
- * different target topics.
- */
- public static class CustomKeyedSerializationSchemaWrapper
- extends KeyedSerializationSchemaWrapper<Tuple2<Long, String>> {
-
- private final String defaultTopic;
- private final String dynamicTopic;
-
- public CustomKeyedSerializationSchemaWrapper(
- SerializationSchema<Tuple2<Long, String>> serializationSchema,
- String defaultTopic,
- String dynamicTopic) {
-
- super(serializationSchema);
-
- this.defaultTopic = Preconditions.checkNotNull(defaultTopic);
- this.dynamicTopic = Preconditions.checkNotNull(dynamicTopic);
- }
-
- @Override
- public String getTargetTopic(Tuple2<Long, String> element) {
- return (element.f0 % 2 == 0) ? defaultTopic : dynamicTopic;
- }
- }
-
- /** Mapper that validates partitioning and maps to partition. */
- public static class PartitionValidatingMapper
- extends RichMapFunction<Tuple2<Long, String>, Integer> {
-
- private final int numPartitions;
-
- private int ourPartition = -1;
-
- public PartitionValidatingMapper(int numPartitions) {
- this.numPartitions = numPartitions;
- }
-
- @Override
- public Integer map(Tuple2<Long, String> value) throws Exception {
- int partition = value.f0.intValue() % numPartitions;
- if (ourPartition != -1) {
- assertThat(partition).as("inconsistent partitioning").isEqualTo(ourPartition);
- } else {
- ourPartition = partition;
- }
- return partition;
- }
- }
-
- /**
- * Sink that validates records received from each partition and checks that there are no
- * duplicates.
- */
- public static class PartitionValidatingSink implements SinkFunction<Integer> {
- private final int[] valuesPerPartition;
-
- public PartitionValidatingSink(int numPartitions) {
- this.valuesPerPartition = new int[numPartitions];
- }
-
- @Override
- public void invoke(Integer value) throws Exception {
- valuesPerPartition[value]++;
-
- boolean missing = false;
- for (int i : valuesPerPartition) {
- if (i < 100) {
- missing = true;
- break;
- }
- }
- if (!missing) {
- throw new SuccessException();
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java
deleted file mode 100644
index 9886885..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * 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.FlinkVersion;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerMatchers;
-import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
-import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase;
-import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer;
-
-import org.hamcrest.Matcher;
-import org.mockito.Mockito;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.hamcrest.Matchers.is;
-
-/**
- * A {@link TypeSerializerUpgradeTestBase} for {@link FlinkKafkaProducer.TransactionStateSerializer}
- * and {@link FlinkKafkaProducer.ContextStateSerializer}.
- */
-class KafkaSerializerUpgradeTest extends TypeSerializerUpgradeTestBase<Object, Object> {
-
- public Collection<TestSpecification<?, ?>> createTestSpecifications(FlinkVersion flinkVersion)
- throws Exception {
-
- ArrayList<TestSpecification<?, ?>> testSpecifications = new ArrayList<>();
- testSpecifications.add(
- new TestSpecification<>(
- "transaction-state-serializer",
- flinkVersion,
- TransactionStateSerializerSetup.class,
- TransactionStateSerializerVerifier.class));
- testSpecifications.add(
- new TestSpecification<>(
- "context-state-serializer",
- flinkVersion,
- ContextStateSerializerSetup.class,
- ContextStateSerializerVerifier.class));
- return testSpecifications;
- }
-
- // ----------------------------------------------------------------------------------------------
- // Specification for "transaction-state-serializer"
- // ----------------------------------------------------------------------------------------------
-
- /**
- * This class is only public to work with {@link
- * org.apache.flink.api.common.typeutils.ClassRelocator}.
- */
- public static final class TransactionStateSerializerSetup
- implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<
- FlinkKafkaProducer.KafkaTransactionState> {
- @Override
- public TypeSerializer<FlinkKafkaProducer.KafkaTransactionState> createPriorSerializer() {
- return new FlinkKafkaProducer.TransactionStateSerializer();
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionState createTestData() {
- @SuppressWarnings("unchecked")
- FlinkKafkaInternalProducer<byte[], byte[]> mock =
- Mockito.mock(FlinkKafkaInternalProducer.class);
- return new FlinkKafkaProducer.KafkaTransactionState("1234", 3456, (short) 789, mock);
- }
- }
-
- /**
- * This class is only public to work with {@link
- * org.apache.flink.api.common.typeutils.ClassRelocator}.
- */
- public static final class TransactionStateSerializerVerifier
- implements TypeSerializerUpgradeTestBase.UpgradeVerifier<
- FlinkKafkaProducer.KafkaTransactionState> {
- @Override
- public TypeSerializer<FlinkKafkaProducer.KafkaTransactionState> createUpgradedSerializer() {
- return new FlinkKafkaProducer.TransactionStateSerializer();
- }
-
- @Override
- public Matcher<FlinkKafkaProducer.KafkaTransactionState> testDataMatcher() {
- @SuppressWarnings("unchecked")
- FlinkKafkaInternalProducer<byte[], byte[]> mock =
- Mockito.mock(FlinkKafkaInternalProducer.class);
- return is(
- new FlinkKafkaProducer.KafkaTransactionState("1234", 3456, (short) 789, mock));
- }
-
- @Override
- public Matcher<TypeSerializerSchemaCompatibility<FlinkKafkaProducer.KafkaTransactionState>>
- schemaCompatibilityMatcher(FlinkVersion version) {
- return TypeSerializerMatchers.isCompatibleAsIs();
- }
- }
-
- // ----------------------------------------------------------------------------------------------
- // Specification for "context-state-serializer"
- // ----------------------------------------------------------------------------------------------
-
- /**
- * This class is only public to work with {@link
- * org.apache.flink.api.common.typeutils.ClassRelocator}.
- */
- public static final class ContextStateSerializerSetup
- implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<
- FlinkKafkaProducer.KafkaTransactionContext> {
- @Override
- public TypeSerializer<FlinkKafkaProducer.KafkaTransactionContext> createPriorSerializer() {
- return new FlinkKafkaProducer.ContextStateSerializer();
- }
-
- @Override
- public FlinkKafkaProducer.KafkaTransactionContext createTestData() {
- Set<String> transactionIds = new HashSet<>();
- transactionIds.add("123");
- transactionIds.add("456");
- transactionIds.add("789");
- return new FlinkKafkaProducer.KafkaTransactionContext(transactionIds);
- }
- }
-
- /**
- * This class is only public to work with {@link
- * org.apache.flink.api.common.typeutils.ClassRelocator}.
- */
- public static final class ContextStateSerializerVerifier
- implements TypeSerializerUpgradeTestBase.UpgradeVerifier<
- FlinkKafkaProducer.KafkaTransactionContext> {
- @Override
- public TypeSerializer<FlinkKafkaProducer.KafkaTransactionContext>
- createUpgradedSerializer() {
- return new FlinkKafkaProducer.ContextStateSerializer();
- }
-
- @Override
- public Matcher<FlinkKafkaProducer.KafkaTransactionContext> testDataMatcher() {
- Set<String> transactionIds = new HashSet<>();
- transactionIds.add("123");
- transactionIds.add("456");
- transactionIds.add("789");
- return is(new FlinkKafkaProducer.KafkaTransactionContext(transactionIds));
- }
-
- @Override
- public Matcher<
- TypeSerializerSchemaCompatibility<
- FlinkKafkaProducer.KafkaTransactionContext>>
- schemaCompatibilityMatcher(FlinkVersion version) {
- return TypeSerializerMatchers.isCompatibleAsIs();
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
deleted file mode 100644
index 9fb16d4..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.MemorySize;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.test.util.MiniClusterWithClientResource;
-import org.apache.flink.util.InstantiationUtil;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-
-/**
- * A class containing a special Kafka broker which has a log retention of only 250 ms. This way, we
- * can make sure our consumer is properly handling cases where we run into out of offset errors
- */
-@SuppressWarnings("serial")
-public class KafkaShortRetentionTestBase implements Serializable {
-
- protected static final Logger LOG = LoggerFactory.getLogger(KafkaShortRetentionTestBase.class);
-
- protected static final int NUM_TMS = 1;
-
- protected static final int TM_SLOTS = 8;
-
- protected static final int PARALLELISM = NUM_TMS * TM_SLOTS;
-
- private static KafkaTestEnvironment kafkaServer;
- private static Properties standardProps;
-
- @ClassRule
- public static MiniClusterWithClientResource flink =
- new MiniClusterWithClientResource(
- new MiniClusterResourceConfiguration.Builder()
- .setConfiguration(getConfiguration())
- .setNumberTaskManagers(NUM_TMS)
- .setNumberSlotsPerTaskManager(TM_SLOTS)
- .build());
-
- @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder();
-
- protected static Properties secureProps = new Properties();
-
- private static Configuration getConfiguration() {
- Configuration flinkConfig = new Configuration();
- flinkConfig.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("16m"));
- return flinkConfig;
- }
-
- @BeforeClass
- public static void prepare() throws Exception {
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Starting KafkaShortRetentionTestBase ");
- LOG.info("-------------------------------------------------------------------------");
-
- // dynamically load the implementation for the test
- Class<?> clazz =
- Class.forName(
- "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
- kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
-
- LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
-
- if (kafkaServer.isSecureRunSupported()) {
- secureProps = kafkaServer.getSecureProperties();
- }
-
- Properties specificProperties = new Properties();
- specificProperties.setProperty("log.retention.hours", "0");
- specificProperties.setProperty("log.retention.minutes", "0");
- specificProperties.setProperty("log.retention.ms", "250");
- specificProperties.setProperty("log.retention.check.interval.ms", "100");
- kafkaServer.prepare(
- kafkaServer.createConfig().setKafkaServerProperties(specificProperties));
-
- standardProps = kafkaServer.getStandardProperties();
- }
-
- @AfterClass
- public static void shutDownServices() throws Exception {
- kafkaServer.shutdown();
-
- secureProps.clear();
- }
-
- /**
- * This test is concurrently reading and writing from a kafka topic. The job will run for a
- * while In a special deserializationSchema, we make sure that the offsets from the topic are
- * non-continuous (because the data is expiring faster than its consumed --> with
- * auto.offset.reset = 'earliest', some offsets will not show up)
- */
- private static boolean stopProducer = false;
-
- public void runAutoOffsetResetTest() throws Exception {
- final String topic = "auto-offset-reset-test-" + UUID.randomUUID();
-
- final int parallelism = 1;
- final int elementsPerPartition = 50000;
-
- Properties tprops = new Properties();
- tprops.setProperty("retention.ms", "250");
- kafkaServer.createTestTopic(topic, parallelism, 1, tprops);
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(parallelism);
- env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-
- // ----------- add producer dataflow ----------
-
- DataStream<String> stream =
- env.addSource(
- new RichParallelSourceFunction<String>() {
-
- private boolean running = true;
-
- @Override
- public void run(SourceContext<String> ctx) throws InterruptedException {
- int cnt =
- getRuntimeContext().getIndexOfThisSubtask()
- * elementsPerPartition;
- int limit = cnt + elementsPerPartition;
-
- while (running && !stopProducer && cnt < limit) {
- ctx.collect("element-" + cnt);
- cnt++;
- Thread.sleep(10);
- }
- LOG.info("Stopping producer");
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(secureProps);
- kafkaServer.produceIntoKafka(stream, topic, new SimpleStringSchema(), props, null);
-
- // ----------- add consumer dataflow ----------
-
- NonContinousOffsetsDeserializationSchema deserSchema =
- new NonContinousOffsetsDeserializationSchema();
- FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer(topic, deserSchema, props);
-
- DataStreamSource<String> consuming = env.addSource(source);
- consuming.addSink(new DiscardingSink<String>());
-
- tryExecute(env, "run auto offset reset test");
-
- kafkaServer.deleteTestTopic(topic);
- }
-
- private class NonContinousOffsetsDeserializationSchema
- implements KafkaDeserializationSchema<String> {
- private int numJumps;
- long nextExpected = 0;
-
- @Override
- public String deserialize(ConsumerRecord<byte[], byte[]> record) {
- final long offset = record.offset();
- if (offset != nextExpected) {
- numJumps++;
- nextExpected = offset;
- LOG.info("Registered now jump at offset {}", offset);
- }
- nextExpected++;
- try {
- Thread.sleep(10); // slow down data consumption to trigger log eviction
- } catch (InterruptedException e) {
- throw new RuntimeException("Stopping it");
- }
- return "";
- }
-
- @Override
- public boolean isEndOfStream(String nextElement) {
- if (numJumps >= 5) {
- // we saw 5 jumps and no failures --> consumer can handle auto.offset.reset
- stopProducer = true;
- return true;
- }
- return false;
- }
-
- @Override
- public TypeInformation<String> getProducedType() {
- return Types.STRING;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
deleted file mode 100644
index b4fd54a..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ /dev/null
@@ -1,341 +0,0 @@
-/*
- * 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.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.MemorySize;
-import org.apache.flink.configuration.MetricOptions;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.metrics.jmx.JMXReporterFactory;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.testutils.junit.RetryOnFailure;
-import org.apache.flink.testutils.junit.RetryRule;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-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.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.assertj.core.api.Assertions.fail;
-
-/**
- * The base for the Kafka tests. It brings up:
- *
- * <ul>
- * <li>A ZooKeeper mini cluster
- * <li>Three Kafka Brokers (mini clusters)
- * <li>A Flink mini cluster
- * </ul>
- *
- * <p>Code in this test is based on the following GitHub repository: <a
- * href="https://github.com/sakserv/hadoop-mini-clusters">
- * https://github.com/sakserv/hadoop-mini-clusters</a> (ASL licensed), as per commit
- * <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i>
- *
- * <p>Tests inheriting from this class are known to be unstable due to the test setup. All tests
- * implemented in subclasses will be retried on failures.
- */
-@SuppressWarnings("serial")
-@RetryOnFailure(times = 3)
-public abstract class KafkaTestBase extends TestLogger {
-
- public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
-
- public static final int NUMBER_OF_KAFKA_SERVERS = 1;
-
- public static String brokerConnectionStrings;
-
- public static Properties standardProps;
-
- public static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
-
- public static KafkaTestEnvironment kafkaServer;
-
- @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder();
-
- public static Properties secureProps = new Properties();
-
- @Rule public final RetryRule retryRule = new RetryRule();
-
- // ------------------------------------------------------------------------
- // Setup and teardown of the mini clusters
- // ------------------------------------------------------------------------
-
- @BeforeClass
- public static void prepare() throws Exception {
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Starting KafkaTestBase ");
- LOG.info("-------------------------------------------------------------------------");
-
- startClusters(false);
- }
-
- @AfterClass
- public static void shutDownServices() throws Exception {
-
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" Shut down KafkaTestBase ");
- LOG.info("-------------------------------------------------------------------------");
-
- TestStreamEnvironment.unsetAsContext();
-
- shutdownClusters();
-
- LOG.info("-------------------------------------------------------------------------");
- LOG.info(" KafkaTestBase finished");
- LOG.info("-------------------------------------------------------------------------");
- }
-
- public static Configuration getFlinkConfiguration() {
- Configuration flinkConfig = new Configuration();
- flinkConfig.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("16m"));
- MetricOptions.forReporter(flinkConfig, "my_reporter")
- .set(MetricOptions.REPORTER_FACTORY_CLASS, JMXReporterFactory.class.getName());
- return flinkConfig;
- }
-
- public static void startClusters() throws Exception {
- startClusters(
- KafkaTestEnvironment.createConfig().setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS));
- }
-
- public static void startClusters(boolean secureMode) throws Exception {
- startClusters(
- KafkaTestEnvironment.createConfig()
- .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS)
- .setSecureMode(secureMode));
- }
-
- public static void startClusters(KafkaTestEnvironment.Config environmentConfig)
- throws Exception {
- kafkaServer = constructKafkaTestEnvironment();
-
- LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion());
-
- kafkaServer.prepare(environmentConfig);
-
- standardProps = kafkaServer.getStandardProperties();
-
- brokerConnectionStrings = kafkaServer.getBrokerConnectionString();
-
- if (environmentConfig.isSecureMode()) {
- if (!kafkaServer.isSecureRunSupported()) {
- throw new IllegalStateException(
- "Attempting to test in secure mode but secure mode not supported by the KafkaTestEnvironment.");
- }
- secureProps = kafkaServer.getSecureProperties();
- }
- }
-
- public static KafkaTestEnvironment constructKafkaTestEnvironment() throws Exception {
- Class<?> clazz =
- Class.forName(
- "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl");
- return (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz);
- }
-
- public static void shutdownClusters() throws Exception {
- if (secureProps != null) {
- secureProps.clear();
- }
-
- if (kafkaServer != null) {
- kafkaServer.shutdown();
- }
- }
-
- // ------------------------------------------------------------------------
- // Execution utilities
- // ------------------------------------------------------------------------
-
- public static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name)
- throws Exception {
- try {
- see.execute(name);
- } catch (ProgramInvocationException | JobExecutionException root) {
- Throwable cause = root.getCause();
-
- // search for nested SuccessExceptions
- int depth = 0;
- while (!(cause instanceof SuccessException)) {
- if (cause == null || depth++ == 20) {
- throw root;
- } else {
- cause = cause.getCause();
- }
- }
- }
- }
-
- public static void createTestTopic(
- String topic, int numberOfPartitions, int replicationFactor) {
- kafkaServer.createTestTopic(topic, numberOfPartitions, replicationFactor);
- }
-
- public static void deleteTestTopic(String topic) {
- kafkaServer.deleteTestTopic(topic);
- }
-
- public static <K, V> void produceToKafka(
- Collection<ProducerRecord<K, V>> records,
- Class<? extends org.apache.kafka.common.serialization.Serializer<K>> keySerializerClass,
- Class<? extends org.apache.kafka.common.serialization.Serializer<V>>
- valueSerializerClass)
- throws Throwable {
- Properties props = new Properties();
- props.putAll(standardProps);
- props.putAll(kafkaServer.getIdempotentProducerConfig());
- props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName());
- props.setProperty(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName());
-
- AtomicReference<Throwable> sendingError = new AtomicReference<>();
- Callback callback =
- (metadata, exception) -> {
- if (exception != null) {
- if (!sendingError.compareAndSet(null, exception)) {
- sendingError.get().addSuppressed(exception);
- }
- }
- };
- try (KafkaProducer<K, V> producer = new KafkaProducer<>(props)) {
- for (ProducerRecord<K, V> record : records) {
- producer.send(record, callback);
- }
- }
- if (sendingError.get() != null) {
- throw sendingError.get();
- }
- }
-
- /**
- * We manually handle the timeout instead of using JUnit's timeout to return failure instead of
- * timeout error. After timeout we assume that there are missing records and there is a bug, not
- * that the test has run out of time.
- */
- public void assertAtLeastOnceForTopic(
- Properties properties,
- String topic,
- int partition,
- Set<Integer> expectedElements,
- long timeoutMillis)
- throws Exception {
-
- long startMillis = System.currentTimeMillis();
- Set<Integer> actualElements = new HashSet<>();
-
- // until we timeout...
- while (System.currentTimeMillis() < startMillis + timeoutMillis) {
- properties.put(
- "key.deserializer",
- "org.apache.kafka.common.serialization.IntegerDeserializer");
- properties.put(
- "value.deserializer",
- "org.apache.kafka.common.serialization.IntegerDeserializer");
- // We need to set these two properties so that they are lower than request.timeout.ms.
- // This is
- // required for some old KafkaConsumer versions.
- properties.put("session.timeout.ms", "2000");
- properties.put("heartbeat.interval.ms", "500");
-
- // query kafka for new records ...
- Collection<ConsumerRecord<Integer, Integer>> records =
- kafkaServer.getAllRecordsFromTopic(properties, topic);
-
- for (ConsumerRecord<Integer, Integer> record : records) {
- actualElements.add(record.value());
- }
-
- // succeed if we got all expectedElements
- if (actualElements.containsAll(expectedElements)) {
- return;
- }
- }
-
- fail(
- String.format(
- "Expected to contain all of: <%s>, but was: <%s>",
- expectedElements, actualElements));
- }
-
- public void assertExactlyOnceForTopic(
- Properties properties, String topic, List<Integer> expectedElements) {
-
- List<Integer> actualElements = new ArrayList<>();
-
- Properties consumerProperties = new Properties();
- consumerProperties.putAll(properties);
- consumerProperties.put(
- "key.deserializer", "org.apache.kafka.common.serialization.IntegerDeserializer");
- consumerProperties.put(
- "value.deserializer", "org.apache.kafka.common.serialization.IntegerDeserializer");
- consumerProperties.put("isolation.level", "read_committed");
-
- // query kafka for new records ...
- Collection<ConsumerRecord<byte[], byte[]>> records =
- kafkaServer.getAllRecordsFromTopic(consumerProperties, topic);
-
- for (ConsumerRecord<byte[], byte[]> record : records) {
- actualElements.add(ByteBuffer.wrap(record.value()).getInt());
- }
-
- // succeed if we got all expectedElements
- if (actualElements.equals(expectedElements)) {
- return;
- }
-
- fail(
- String.format(
- "Expected %s, but was: %s",
- formatElements(expectedElements), formatElements(actualElements)));
- }
-
- private String formatElements(List<Integer> elements) {
- if (elements.size() > 50) {
- return String.format("number of elements: <%s>", elements.size());
- } else {
- return String.format("elements: <%s>", elements);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java
deleted file mode 100644
index 1c8d651..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.test.util.MiniClusterWithClientResource;
-
-import org.junit.ClassRule;
-
-/** The base for the Kafka tests with Flink's MiniCluster. */
-@SuppressWarnings("serial")
-public abstract class KafkaTestBaseWithFlink extends KafkaTestBase {
-
- protected static final int NUM_TMS = 1;
-
- protected static final int TM_SLOTS = 8;
-
- @ClassRule
- public static MiniClusterWithClientResource flink =
- new MiniClusterWithClientResource(
- new MiniClusterResourceConfiguration.Builder()
- .setConfiguration(getFlinkConfiguration())
- .setNumberTaskManagers(NUM_TMS)
- .setNumberSlotsPerTaskManager(TM_SLOTS)
- .build());
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
deleted file mode 100644
index ee38e85..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * 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.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/** Abstract class providing a Kafka test environment. */
-public abstract class KafkaTestEnvironment {
- /** Configuration class for {@link KafkaTestEnvironment}. */
- public static class Config {
- private int kafkaServersNumber = 1;
- private Properties kafkaServerProperties = null;
- private boolean secureMode = false;
-
- /** Please use {@link KafkaTestEnvironment#createConfig()} method. */
- private Config() {}
-
- public int getKafkaServersNumber() {
- return kafkaServersNumber;
- }
-
- public Config setKafkaServersNumber(int kafkaServersNumber) {
- this.kafkaServersNumber = kafkaServersNumber;
- return this;
- }
-
- public Properties getKafkaServerProperties() {
- return kafkaServerProperties;
- }
-
- public Config setKafkaServerProperties(Properties kafkaServerProperties) {
- this.kafkaServerProperties = kafkaServerProperties;
- return this;
- }
-
- public boolean isSecureMode() {
- return secureMode;
- }
-
- public Config setSecureMode(boolean secureMode) {
- this.secureMode = secureMode;
- return this;
- }
-
- public Config setHideKafkaBehindProxy(boolean hideKafkaBehindProxy) {
- return this;
- }
- }
-
- protected static final String KAFKA_HOST = "localhost";
-
- public static Config createConfig() {
- return new Config();
- }
-
- public abstract void prepare(Config config) throws Exception;
-
- public void shutdown() throws Exception {}
-
- public abstract void deleteTestTopic(String topic);
-
- public abstract void createTestTopic(
- String topic, int numberOfPartitions, int replicationFactor, Properties properties);
-
- public void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
- this.createTestTopic(topic, numberOfPartitions, replicationFactor, new Properties());
- }
-
- public abstract Properties getStandardProperties();
-
- public abstract Properties getSecureProperties();
-
- public abstract String getBrokerConnectionString();
-
- public abstract String getVersion();
-
- public Properties getIdempotentProducerConfig() {
- Properties props = new Properties();
- props.put("enable.idempotence", "true");
- props.put("acks", "all");
- props.put("retries", "3");
- return props;
- }
-
- // -- consumer / producer instances:
- public <T> FlinkKafkaConsumerBase<T> getConsumer(
- List<String> topics, DeserializationSchema<T> deserializationSchema, Properties props) {
- return getConsumer(
- topics, new KafkaDeserializationSchemaWrapper<T>(deserializationSchema), props);
- }
-
- public <T> FlinkKafkaConsumerBase<T> getConsumer(
- String topic, KafkaDeserializationSchema<T> readSchema, Properties props) {
- return getConsumer(Collections.singletonList(topic), readSchema, props);
- }
-
- public <T> FlinkKafkaConsumerBase<T> getConsumer(
- String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
- return getConsumer(Collections.singletonList(topic), deserializationSchema, props);
- }
-
- public abstract <T> FlinkKafkaConsumerBase<T> getConsumer(
- List<String> topics, KafkaDeserializationSchema<T> readSchema, Properties props);
-
- public <T> KafkaSourceBuilder<T> getSourceBuilder(
- List<String> topics, DeserializationSchema<T> deserializationSchema, Properties props) {
- return getSourceBuilder(
- topics, new KafkaDeserializationSchemaWrapper<T>(deserializationSchema), props);
- }
-
- public <T> KafkaSourceBuilder<T> getSourceBuilder(
- String topic, KafkaDeserializationSchema<T> readSchema, Properties props) {
- return getSourceBuilder(Collections.singletonList(topic), readSchema, props);
- }
-
- public <T> KafkaSourceBuilder<T> getSourceBuilder(
- String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
- return getSourceBuilder(Collections.singletonList(topic), deserializationSchema, props);
- }
-
- public abstract <T> KafkaSourceBuilder<T> getSourceBuilder(
- List<String> topics, KafkaDeserializationSchema<T> readSchema, Properties props);
-
- public abstract <K, V> Collection<ConsumerRecord<K, V>> getAllRecordsFromTopic(
- Properties properties, String topic);
-
- public abstract <T> StreamSink<T> getProducerSink(
- String topic,
- SerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner);
-
- @Deprecated
- public abstract <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- KeyedSerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner);
-
- public abstract <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- SerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner);
-
- public <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- KafkaSerializationSchema<T> serSchema,
- Properties props) {
- throw new RuntimeException(
- "KafkaSerializationSchema is only supported on the modern Kafka Connector.");
- }
-
- // -- offset handlers
-
- /** Simple interface to commit and retrieve offsets. */
- public interface KafkaOffsetHandler {
- Long getCommittedOffset(String topicName, int partition);
-
- void setCommittedOffset(String topicName, int partition, long offset);
-
- void close();
- }
-
- public abstract KafkaOffsetHandler createOffsetHandler();
-
- // -- leader failure simulation
-
- public abstract void restartBroker(int leaderId) throws Exception;
-
- public abstract void stopBroker(int brokerId) throws Exception;
-
- public abstract int getLeaderToShutDown(String topic) throws Exception;
-
- public abstract boolean isSecureRunSupported();
-
- protected void maybePrintDanglingThreadStacktrace(String threadNameKeyword) {
- for (Map.Entry<Thread, StackTraceElement[]> threadEntry :
- Thread.getAllStackTraces().entrySet()) {
- if (threadEntry.getKey().getName().contains(threadNameKeyword)) {
- System.out.println("Dangling thread found:");
- for (StackTraceElement ste : threadEntry.getValue()) {
- System.out.println(ste);
- }
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
deleted file mode 100644
index c5bc3b0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/*
- * 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.api.common.serialization.SerializationSchema;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceBuilder;
-import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
-import org.apache.flink.connector.kafka.testutils.KafkaUtil;
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
-import org.apache.flink.util.DockerImageVersions;
-
-import org.apache.commons.collections.list.UnmodifiableList;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.admin.TopicListing;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.DockerClientFactory;
-import org.testcontainers.containers.GenericContainer;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-import org.testcontainers.utility.DockerImageName;
-
-import javax.annotation.Nullable;
-
-import java.time.Duration;
-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.Optional;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
-
-import static org.assertj.core.api.Assertions.fail;
-
-/** An implementation of the KafkaServerProvider. */
-public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment {
-
- protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestEnvironmentImpl.class);
-
- private static final String ZOOKEEPER_HOSTNAME = "zookeeper";
- private static final int ZOOKEEPER_PORT = 2181;
-
- private final Map<Integer, KafkaContainer> brokers = new HashMap<>();
- private final Set<Integer> pausedBroker = new HashSet<>();
- private @Nullable GenericContainer<?> zookeeper;
- private @Nullable Network network;
- private String brokerConnectionString = "";
- private Properties standardProps;
- private FlinkKafkaProducer.Semantic producerSemantic = FlinkKafkaProducer.Semantic.EXACTLY_ONCE;
- // 6 seconds is default. Seems to be too small for travis. 30 seconds
- private int zkTimeout = 30000;
- private Config config;
- private static final int REQUEST_TIMEOUT_SECONDS = 30;
-
- public void setProducerSemantic(FlinkKafkaProducer.Semantic producerSemantic) {
- this.producerSemantic = producerSemantic;
- }
-
- @Override
- public void prepare(Config config) throws Exception {
- // increase the timeout since in Travis ZK connection takes long time for secure connection.
- if (config.isSecureMode()) {
- // run only one kafka server to avoid multiple ZK connections from many instances -
- // Travis timeout
- config.setKafkaServersNumber(1);
- zkTimeout = zkTimeout * 15;
- }
- this.config = config;
- brokers.clear();
-
- LOG.info("Starting KafkaServer");
- startKafkaContainerCluster(config.getKafkaServersNumber());
- LOG.info("KafkaServer started.");
-
- standardProps = new Properties();
- standardProps.setProperty("bootstrap.servers", brokerConnectionString);
- standardProps.setProperty("group.id", "flink-tests");
- standardProps.setProperty("enable.auto.commit", "false");
- standardProps.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
- standardProps.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
- standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
- standardProps.setProperty(
- "max.partition.fetch.bytes",
- "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
- }
-
- @Override
- public void deleteTestTopic(String topic) {
- LOG.info("Deleting topic {}", topic);
- Properties props = getSecureProperties();
- props.putAll(getStandardProperties());
- String clientId = Long.toString(new Random().nextLong());
- props.put("client.id", clientId);
- AdminClient adminClient = AdminClient.create(props);
- // We do not use a try-catch clause here so we can apply a timeout to the admin client
- // closure.
- try {
- tryDelete(adminClient, topic);
- } catch (Exception e) {
- e.printStackTrace();
- fail(String.format("Delete test topic : %s failed, %s", topic, e.getMessage()));
- } finally {
- adminClient.close(Duration.ofMillis(5000L));
- maybePrintDanglingThreadStacktrace(clientId);
- }
- }
-
- private void tryDelete(AdminClient adminClient, String topic) throws Exception {
- try {
- adminClient.deleteTopics(Collections.singleton(topic)).all().get();
- CommonTestUtils.waitUtil(
- () -> {
- try {
- return adminClient.listTopics().listings().get().stream()
- .map(TopicListing::name)
- .noneMatch((name) -> name.equals(topic));
- } catch (Exception e) {
- LOG.warn("Exception caught when listing Kafka topics", e);
- return false;
- }
- },
- Duration.ofSeconds(REQUEST_TIMEOUT_SECONDS),
- String.format("Topic \"%s\" was not deleted within timeout", topic));
- } catch (TimeoutException e) {
- LOG.info(
- "Did not receive delete topic response within {} seconds. Checking if it succeeded",
- REQUEST_TIMEOUT_SECONDS);
- if (adminClient.listTopics().names().get().contains(topic)) {
- throw new Exception("Topic still exists after timeout", e);
- }
- }
- }
-
- @Override
- public void createTestTopic(
- String topic, int numberOfPartitions, int replicationFactor, Properties properties) {
- LOG.info("Creating topic {}", topic);
- try (AdminClient adminClient = AdminClient.create(getStandardProperties())) {
- NewTopic topicObj = new NewTopic(topic, numberOfPartitions, (short) replicationFactor);
- adminClient.createTopics(Collections.singleton(topicObj)).all().get();
- CommonTestUtils.waitUtil(
- () -> {
- Map<String, TopicDescription> topicDescriptions;
- try {
- topicDescriptions =
- adminClient
- .describeTopics(Collections.singleton(topic))
- .allTopicNames()
- .get(REQUEST_TIMEOUT_SECONDS, TimeUnit.SECONDS);
- } catch (Exception e) {
- LOG.warn("Exception caught when describing Kafka topics", e);
- return false;
- }
- if (topicDescriptions == null || !topicDescriptions.containsKey(topic)) {
- return false;
- }
- TopicDescription topicDescription = topicDescriptions.get(topic);
- return topicDescription.partitions().size() == numberOfPartitions;
- },
- Duration.ofSeconds(30),
- String.format("New topic \"%s\" is not ready within timeout", topicObj));
- } catch (Exception e) {
- e.printStackTrace();
- fail("Create test topic : " + topic + " failed, " + e.getMessage());
- }
- }
-
- @Override
- public Properties getStandardProperties() {
- return standardProps;
- }
-
- @Override
- public Properties getSecureProperties() {
- Properties prop = new Properties();
- if (config.isSecureMode()) {
- prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT");
- prop.put("security.protocol", "SASL_PLAINTEXT");
- prop.put("sasl.kerberos.service.name", "kafka");
-
- // add special timeout for Travis
- prop.setProperty("zookeeper.session.timeout.ms", String.valueOf(zkTimeout));
- prop.setProperty("zookeeper.connection.timeout.ms", String.valueOf(zkTimeout));
- prop.setProperty("metadata.fetch.timeout.ms", "120000");
- }
- return prop;
- }
-
- @Override
- public String getBrokerConnectionString() {
- return brokerConnectionString;
- }
-
- @Override
- public String getVersion() {
- return DockerImageVersions.KAFKA;
- }
-
- @Override
- public <T> FlinkKafkaConsumerBase<T> getConsumer(
- List<String> topics, KafkaDeserializationSchema<T> readSchema, Properties props) {
- return new FlinkKafkaConsumer<T>(topics, readSchema, props);
- }
-
- @Override
- public <T> KafkaSourceBuilder<T> getSourceBuilder(
- List<String> topics, KafkaDeserializationSchema<T> schema, Properties props) {
- return KafkaSource.<T>builder()
- .setTopics(topics)
- .setDeserializer(KafkaRecordDeserializationSchema.of(schema))
- .setProperties(props);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <K, V> Collection<ConsumerRecord<K, V>> getAllRecordsFromTopic(
- Properties properties, String topic) {
- return UnmodifiableList.decorate(KafkaUtil.drainAllRecordsFromTopic(topic, properties));
- }
-
- @Override
- public <T> StreamSink<T> getProducerSink(
- String topic,
- SerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner) {
- return new StreamSink<>(
- new FlinkKafkaProducer<>(
- topic,
- serSchema,
- props,
- partitioner,
- producerSemantic,
- FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE));
- }
-
- @Override
- public <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- KeyedSerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner) {
- return stream.addSink(
- new FlinkKafkaProducer<T>(
- topic,
- serSchema,
- props,
- Optional.ofNullable(partitioner),
- producerSemantic,
- FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE));
- }
-
- @Override
- public <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- SerializationSchema<T> serSchema,
- Properties props,
- FlinkKafkaPartitioner<T> partitioner) {
- return stream.addSink(
- new FlinkKafkaProducer<T>(
- topic,
- serSchema,
- props,
- partitioner,
- producerSemantic,
- FlinkKafkaProducer.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE));
- }
-
- @Override
- public <T> DataStreamSink<T> produceIntoKafka(
- DataStream<T> stream,
- String topic,
- KafkaSerializationSchema<T> serSchema,
- Properties props) {
- return stream.addSink(new FlinkKafkaProducer<T>(topic, serSchema, props, producerSemantic));
- }
-
- @Override
- public KafkaOffsetHandler createOffsetHandler() {
- return new KafkaOffsetHandlerImpl();
- }
-
- @Override
- public void restartBroker(int leaderId) throws Exception {
- unpause(leaderId);
- }
-
- @Override
- public void stopBroker(int brokerId) throws Exception {
- pause(brokerId);
- }
-
- @Override
- public int getLeaderToShutDown(String topic) throws Exception {
- try (final AdminClient client = AdminClient.create(getStandardProperties())) {
- TopicDescription result =
- client.describeTopics(Collections.singleton(topic))
- .allTopicNames()
- .get()
- .get(topic);
- return result.partitions().get(0).leader().id();
- }
- }
-
- @Override
- public boolean isSecureRunSupported() {
- return true;
- }
-
- @Override
- public void shutdown() throws Exception {
- brokers.values().forEach(GenericContainer::stop);
- brokers.clear();
-
- if (zookeeper != null) {
- zookeeper.stop();
- }
-
- if (network != null) {
- network.close();
- }
- }
-
- private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler {
-
- private final KafkaConsumer<byte[], byte[]> offsetClient;
-
- public KafkaOffsetHandlerImpl() {
- Properties props = new Properties();
- props.putAll(standardProps);
- props.setProperty(
- "key.deserializer",
- "org.apache.kafka.common.serialization.ByteArrayDeserializer");
- props.setProperty(
- "value.deserializer",
- "org.apache.kafka.common.serialization.ByteArrayDeserializer");
-
- offsetClient = new KafkaConsumer<>(props);
- }
-
- @Override
- public Long getCommittedOffset(String topicName, int partition) {
- OffsetAndMetadata committed =
- offsetClient.committed(new TopicPartition(topicName, partition));
- return (committed != null) ? committed.offset() : null;
- }
-
- @Override
- public void setCommittedOffset(String topicName, int partition, long offset) {
- Map<TopicPartition, OffsetAndMetadata> partitionAndOffset = new HashMap<>();
- partitionAndOffset.put(
- new TopicPartition(topicName, partition), new OffsetAndMetadata(offset));
- offsetClient.commitSync(partitionAndOffset);
- }
-
- @Override
- public void close() {
- offsetClient.close();
- }
- }
-
- private void startKafkaContainerCluster(int numBrokers) {
- if (numBrokers > 1) {
- network = Network.newNetwork();
- zookeeper = createZookeeperContainer(network);
- zookeeper.start();
- LOG.info("Zookeeper container started");
- }
- for (int brokerID = 0; brokerID < numBrokers; brokerID++) {
- KafkaContainer broker = createKafkaContainer(brokerID, zookeeper);
- brokers.put(brokerID, broker);
- }
- new ArrayList<>(brokers.values()).parallelStream().forEach(GenericContainer::start);
- LOG.info("{} brokers started", numBrokers);
- brokerConnectionString =
- brokers.values().stream()
- .map(KafkaContainer::getBootstrapServers)
- // Here we have URL like "PLAINTEXT://127.0.0.1:15213", and we only keep the
- // "127.0.0.1:15213" part in broker connection string
- .map(server -> server.split("://")[1])
- .collect(Collectors.joining(","));
- }
-
- private GenericContainer<?> createZookeeperContainer(Network network) {
- return new GenericContainer<>(DockerImageName.parse(DockerImageVersions.ZOOKEEPER))
- .withNetwork(network)
- .withNetworkAliases(ZOOKEEPER_HOSTNAME)
- .withEnv("ZOOKEEPER_CLIENT_PORT", String.valueOf(ZOOKEEPER_PORT));
- }
-
- private KafkaContainer createKafkaContainer(
- int brokerID, @Nullable GenericContainer<?> zookeeper) {
- String brokerName = String.format("Kafka-%d", brokerID);
- KafkaContainer broker =
- KafkaUtil.createKafkaContainer(DockerImageVersions.KAFKA, LOG, brokerName)
- .withNetworkAliases(brokerName)
- .withEnv("KAFKA_BROKER_ID", String.valueOf(brokerID))
- .withEnv("KAFKA_MESSAGE_MAX_BYTES", String.valueOf(50 * 1024 * 1024))
- .withEnv("KAFKA_REPLICA_FETCH_MAX_BYTES", String.valueOf(50 * 1024 * 1024))
- .withEnv(
- "KAFKA_TRANSACTION_MAX_TIMEOUT_MS",
- Integer.toString(1000 * 60 * 60 * 2))
- // Disable log deletion to prevent records from being deleted during test
- // run
- .withEnv("KAFKA_LOG_RETENTION_MS", "-1")
- .withEnv("KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS", String.valueOf(zkTimeout))
- .withEnv(
- "KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS", String.valueOf(zkTimeout));
-
- if (zookeeper != null) {
- broker.dependsOn(zookeeper)
- .withNetwork(zookeeper.getNetwork())
- .withExternalZookeeper(
- String.format("%s:%d", ZOOKEEPER_HOSTNAME, ZOOKEEPER_PORT));
- } else {
- broker.withEmbeddedZookeeper();
- }
- return broker;
- }
-
- private void pause(int brokerId) {
- if (pausedBroker.contains(brokerId)) {
- LOG.warn("Broker {} is already paused. Skipping pause operation", brokerId);
- return;
- }
- DockerClientFactory.instance()
- .client()
- .pauseContainerCmd(brokers.get(brokerId).getContainerId())
- .exec();
- pausedBroker.add(brokerId);
- LOG.info("Broker {} is paused", brokerId);
- }
-
- private void unpause(int brokerId) throws Exception {
- if (!pausedBroker.contains(brokerId)) {
- LOG.warn("Broker {} is already running. Skipping unpause operation", brokerId);
- return;
- }
- DockerClientFactory.instance()
- .client()
- .unpauseContainerCmd(brokers.get(brokerId).getContainerId())
- .exec();
- try (AdminClient adminClient = AdminClient.create(getStandardProperties())) {
- CommonTestUtils.waitUtil(
- () -> {
- try {
- return adminClient.describeCluster().nodes().get().stream()
- .anyMatch((node) -> node.id() == brokerId);
- } catch (Exception e) {
- return false;
- }
- },
- Duration.ofSeconds(30),
- String.format(
- "The paused broker %d is not recovered within timeout", brokerId));
- }
- pausedBroker.remove(brokerId);
- LOG.info("Broker {} is resumed", brokerId);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/NextTransactionalIdHintSerializerTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/NextTransactionalIdHintSerializerTest.java
deleted file mode 100644
index 7632afe..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/NextTransactionalIdHintSerializerTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.api.common.typeutils.SerializerTestBase;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * A test for the {@link TypeSerializer TypeSerializers} used for {@link
- * FlinkKafkaProducer.NextTransactionalIdHint}.
- */
-class NextTransactionalIdHintSerializerTest
- extends SerializerTestBase<FlinkKafkaProducer.NextTransactionalIdHint> {
-
- @Override
- protected TypeSerializer<FlinkKafkaProducer.NextTransactionalIdHint> createSerializer() {
- return new FlinkKafkaProducer.NextTransactionalIdHintSerializer();
- }
-
- @Override
- protected int getLength() {
- return Long.BYTES + Integer.BYTES;
- }
-
- @Override
- protected Class<FlinkKafkaProducer.NextTransactionalIdHint> getTypeClass() {
- return (Class) FlinkKafkaProducer.NextTransactionalIdHint.class;
- }
-
- @Override
- protected FlinkKafkaProducer.NextTransactionalIdHint[] getTestData() {
- return new FlinkKafkaProducer.NextTransactionalIdHint[] {
- new FlinkKafkaProducer.NextTransactionalIdHint(1, 0L),
- new FlinkKafkaProducer.NextTransactionalIdHint(1, 1L),
- new FlinkKafkaProducer.NextTransactionalIdHint(1, -1L),
- new FlinkKafkaProducer.NextTransactionalIdHint(2, 0L),
- new FlinkKafkaProducer.NextTransactionalIdHint(2, 1L),
- new FlinkKafkaProducer.NextTransactionalIdHint(2, -1L),
- };
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java
deleted file mode 100644
index 92fc08e..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java
+++ /dev/null
@@ -1,279 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.core.testutils.CheckedThread;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.util.SerializedValue;
-
-import org.junit.Test;
-
-import javax.annotation.Nonnull;
-
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Queue;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the {@link AbstractFetcher}. */
-@SuppressWarnings("serial")
-public class AbstractFetcherTest {
-
- @Test
- public void testIgnorePartitionStateSentinelInSnapshot() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 1),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 2),
- KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 3),
- KafkaTopicPartitionStateSentinel.EARLIEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- null, /* watermark strategy */
- new TestProcessingTimeService(),
- 0);
-
- synchronized (sourceContext.getCheckpointLock()) {
- HashMap<KafkaTopicPartition, Long> currentState = fetcher.snapshotCurrentState();
- fetcher.commitInternalOffsetsToKafka(
- currentState,
- new KafkaCommitCallback() {
- @Override
- public void onSuccess() {}
-
- @Override
- public void onException(Throwable cause) {
- throw new RuntimeException("Callback failed", cause);
- }
- });
-
- assertThat(fetcher.getLastCommittedOffsets()).isPresent();
- assertThat(fetcher.getLastCommittedOffsets().get()).isEmpty();
- }
- }
-
- // ------------------------------------------------------------------------
- // Record emitting tests
- // ------------------------------------------------------------------------
-
- @Test
- public void testSkipCorruptedRecord() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 1),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- null, /* watermark strategy */
- new TestProcessingTimeService(),
- 0);
-
- final KafkaTopicPartitionState<Long, Object> partitionStateHolder =
- fetcher.subscribedPartitionStates().get(0);
-
- emitRecord(fetcher, 1L, partitionStateHolder, 1L);
- emitRecord(fetcher, 2L, partitionStateHolder, 2L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(2L);
- assertThat(partitionStateHolder.getOffset()).isEqualTo(2L);
-
- // emit no records
- fetcher.emitRecordsWithTimestamps(emptyQueue(), partitionStateHolder, 3L, Long.MIN_VALUE);
- assertThat(sourceContext.getLatestElement().getValue().longValue())
- .isEqualTo(2L); // the null record should be skipped
- assertThat(partitionStateHolder.getOffset())
- .isEqualTo(3L); // the offset in state still should have advanced
- }
-
- @Test
- public void testConcurrentPartitionsDiscoveryAndLoopFetching() throws Exception {
- // test data
- final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42);
-
- // ----- create the test fetcher -----
-
- SourceContext<String> sourceContext = new TestSourceContext<>();
- Map<KafkaTopicPartition, Long> partitionsWithInitialOffsets =
- Collections.singletonMap(
- testPartition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-
- final OneShotLatch fetchLoopWaitLatch = new OneShotLatch();
- final OneShotLatch stateIterationBlockLatch = new OneShotLatch();
-
- final TestFetcher<String> fetcher =
- new TestFetcher<>(
- sourceContext,
- partitionsWithInitialOffsets,
- null, /* watermark strategy */
- new TestProcessingTimeService(),
- 10,
- fetchLoopWaitLatch,
- stateIterationBlockLatch);
-
- // ----- run the fetcher -----
-
- final CheckedThread checkedThread =
- new CheckedThread() {
- @Override
- public void go() throws Exception {
- fetcher.runFetchLoop();
- }
- };
- checkedThread.start();
-
- // wait until state iteration begins before adding discovered partitions
- fetchLoopWaitLatch.await();
- fetcher.addDiscoveredPartitions(Collections.singletonList(testPartition));
-
- stateIterationBlockLatch.trigger();
- checkedThread.sync();
- }
-
- // ------------------------------------------------------------------------
- // Test mocks
- // ------------------------------------------------------------------------
-
- private static final class TestFetcher<T> extends AbstractFetcher<T, Object> {
- Map<KafkaTopicPartition, Long> lastCommittedOffsets = null;
-
- private final OneShotLatch fetchLoopWaitLatch;
- private final OneShotLatch stateIterationBlockLatch;
-
- TestFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval)
- throws Exception {
-
- this(
- sourceContext,
- assignedPartitionsWithStartOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- null,
- null);
- }
-
- TestFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval,
- OneShotLatch fetchLoopWaitLatch,
- OneShotLatch stateIterationBlockLatch)
- throws Exception {
-
- super(
- sourceContext,
- assignedPartitionsWithStartOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- TestFetcher.class.getClassLoader(),
- new UnregisteredMetricsGroup(),
- false);
-
- this.fetchLoopWaitLatch = fetchLoopWaitLatch;
- this.stateIterationBlockLatch = stateIterationBlockLatch;
- }
-
- /**
- * Emulation of partition's iteration which is required for {@link
- * AbstractFetcherTest#testConcurrentPartitionsDiscoveryAndLoopFetching}.
- */
- @Override
- public void runFetchLoop() throws Exception {
- if (fetchLoopWaitLatch != null) {
- for (KafkaTopicPartitionState<?, ?> ignored : subscribedPartitionStates()) {
- fetchLoopWaitLatch.trigger();
- stateIterationBlockLatch.await();
- }
- } else {
- throw new UnsupportedOperationException();
- }
- }
-
- @Override
- public void cancel() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Object createKafkaPartitionHandle(KafkaTopicPartition partition) {
- return new Object();
- }
-
- @Override
- protected void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets, @Nonnull KafkaCommitCallback callback) {
- lastCommittedOffsets = offsets;
- callback.onSuccess();
- }
-
- public Optional<Map<KafkaTopicPartition, Long>> getLastCommittedOffsets() {
- return Optional.ofNullable(lastCommittedOffsets);
- }
- }
-
- // ------------------------------------------------------------------------
-
- private static <T, KPH> void emitRecord(
- AbstractFetcher<T, KPH> fetcher,
- T record,
- KafkaTopicPartitionState<T, KPH> partitionState,
- long offset) {
- ArrayDeque<T> recordQueue = new ArrayDeque<>();
- recordQueue.add(record);
-
- fetcher.emitRecordsWithTimestamps(recordQueue, partitionState, offset, Long.MIN_VALUE);
- }
-
- private static <T> Queue<T> emptyQueue() {
- return new ArrayDeque<>();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java
deleted file mode 100644
index 6cd3b64..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java
+++ /dev/null
@@ -1,499 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.api.common.eventtime.WatermarkGenerator;
-import org.apache.flink.api.common.eventtime.WatermarkOutput;
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext;
-import org.apache.flink.streaming.runtime.operators.util.AssignerWithPeriodicWatermarksAdapter;
-import org.apache.flink.streaming.runtime.operators.util.AssignerWithPunctuatedWatermarksAdapter;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
-import org.apache.flink.util.SerializedValue;
-
-import org.junit.Test;
-import org.junit.experimental.runners.Enclosed;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-
-import java.util.ArrayDeque;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the watermarking behaviour of {@link AbstractFetcher}. */
-@SuppressWarnings("serial")
-@RunWith(Enclosed.class)
-public class AbstractFetcherWatermarksTest {
-
- /** Tests with watermark generators that have a periodic nature. */
- @RunWith(Parameterized.class)
- public static class PeriodicWatermarksSuite {
-
- @Parameterized.Parameters
- public static Collection<WatermarkStrategy<Long>> getParams() {
- return Arrays.asList(
- new AssignerWithPeriodicWatermarksAdapter.Strategy<>(
- new PeriodicTestExtractor()),
- WatermarkStrategy.forGenerator((ctx) -> new PeriodicTestWatermarkGenerator())
- .withTimestampAssigner((event, previousTimestamp) -> event));
- }
-
- @Parameterized.Parameter public WatermarkStrategy<Long> testWmStrategy;
-
- @Test
- public void testPeriodicWatermarks() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 7),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 13),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 21),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestProcessingTimeService processingTimeService = new TestProcessingTimeService();
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- new SerializedValue<>(testWmStrategy),
- processingTimeService,
- 10);
-
- final KafkaTopicPartitionState<Long, Object> part1 =
- fetcher.subscribedPartitionStates().get(0);
- final KafkaTopicPartitionState<Long, Object> part2 =
- fetcher.subscribedPartitionStates().get(1);
- final KafkaTopicPartitionState<Long, Object> part3 =
- fetcher.subscribedPartitionStates().get(2);
-
- // elements generate a watermark if the timestamp is a multiple of three
-
- // elements for partition 1
- emitRecord(fetcher, 1L, part1, 1L);
- emitRecord(fetcher, 1L, part1, 1L);
- emitRecord(fetcher, 2L, part1, 2L);
- emitRecord(fetcher, 3L, part1, 3L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
-
- // elements for partition 2
- emitRecord(fetcher, 12L, part2, 1L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(12L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(12L);
-
- // elements for partition 3
- emitRecord(fetcher, 101L, part3, 1L);
- emitRecord(fetcher, 102L, part3, 2L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(102L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(102L);
-
- processingTimeService.setCurrentTime(10);
-
- // now, we should have a watermark (this blocks until the periodic thread emitted the
- // watermark)
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(3L);
-
- // advance partition 3
- emitRecord(fetcher, 1003L, part3, 3L);
- emitRecord(fetcher, 1004L, part3, 4L);
- emitRecord(fetcher, 1005L, part3, 5L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(1005L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(1005L);
-
- // advance partition 1 beyond partition 2 - this bumps the watermark
- emitRecord(fetcher, 30L, part1, 4L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(30L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(30L);
-
- processingTimeService.setCurrentTime(20);
-
- // this blocks until the periodic thread emitted the watermark
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(12L);
-
- // advance partition 2 again - this bumps the watermark
- emitRecord(fetcher, 13L, part2, 2L);
- emitRecord(fetcher, 14L, part2, 3L);
- emitRecord(fetcher, 15L, part2, 3L);
-
- processingTimeService.setCurrentTime(30);
- // this blocks until the periodic thread emitted the watermark
- long watermarkTs = sourceContext.getLatestWatermark().getTimestamp();
- assertThat(watermarkTs >= 13L && watermarkTs <= 15L).isTrue();
- }
-
- @Test
- public void testSkipCorruptedRecordWithPeriodicWatermarks() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 1),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- new SerializedValue<>(testWmStrategy),
- processingTimeProvider,
- 10);
-
- final KafkaTopicPartitionState<Long, Object> partitionStateHolder =
- fetcher.subscribedPartitionStates().get(0);
-
- // elements generate a watermark if the timestamp is a multiple of three
- emitRecord(fetcher, 1L, partitionStateHolder, 1L);
- emitRecord(fetcher, 2L, partitionStateHolder, 2L);
- emitRecord(fetcher, 3L, partitionStateHolder, 3L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
- assertThat(partitionStateHolder.getOffset()).isEqualTo(3L);
-
- // advance timer for watermark emitting
- processingTimeProvider.setCurrentTime(10L);
- assertThat(sourceContext.hasWatermark()).isTrue();
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(3L);
-
- // emit no records
- fetcher.emitRecordsWithTimestamps(
- emptyQueue(), partitionStateHolder, 4L, Long.MIN_VALUE);
-
- // no elements should have been collected
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
- // the offset in state still should have advanced
- assertThat(partitionStateHolder.getOffset()).isEqualTo(4L);
-
- // no watermarks should be collected
- processingTimeProvider.setCurrentTime(20L);
- assertThat(sourceContext.hasWatermark()).isFalse();
- }
-
- @Test
- public void testPeriodicWatermarksWithNoSubscribedPartitionsShouldYieldNoWatermarks()
- throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- new SerializedValue<>(testWmStrategy),
- processingTimeProvider,
- 10);
-
- processingTimeProvider.setCurrentTime(10);
- // no partitions; when the periodic watermark emitter fires, no watermark should be
- // emitted
- assertThat(sourceContext.hasWatermark()).isFalse();
-
- // counter-test that when the fetcher does actually have partitions,
- // when the periodic watermark emitter fires again, a watermark really is emitted
- fetcher.addDiscoveredPartitions(
- Collections.singletonList(new KafkaTopicPartition(testTopic, 0)));
- emitRecord(fetcher, 100L, fetcher.subscribedPartitionStates().get(0), 3L);
- processingTimeProvider.setCurrentTime(20);
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(100);
- }
- }
-
- /** Tests with watermark generators that have a punctuated nature. */
- public static class PunctuatedWatermarksSuite {
-
- @Test
- public void testSkipCorruptedRecordWithPunctuatedWatermarks() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 1),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
-
- AssignerWithPunctuatedWatermarksAdapter.Strategy<Long> testWmStrategy =
- new AssignerWithPunctuatedWatermarksAdapter.Strategy<>(
- new PunctuatedTestExtractor());
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- new SerializedValue<>(testWmStrategy),
- processingTimeProvider,
- 0);
-
- final KafkaTopicPartitionState<Long, Object> partitionStateHolder =
- fetcher.subscribedPartitionStates().get(0);
-
- // elements generate a watermark if the timestamp is a multiple of three
- emitRecord(fetcher, 1L, partitionStateHolder, 1L);
- emitRecord(fetcher, 2L, partitionStateHolder, 2L);
- emitRecord(fetcher, 3L, partitionStateHolder, 3L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
- assertThat(sourceContext.hasWatermark()).isTrue();
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(3L);
- assertThat(partitionStateHolder.getOffset()).isEqualTo(3L);
-
- // emit no records
- fetcher.emitRecordsWithTimestamps(emptyQueue(), partitionStateHolder, 4L, -1L);
-
- // no elements or watermarks should have been collected
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
- assertThat(sourceContext.hasWatermark()).isFalse();
- // the offset in state still should have advanced
- assertThat(partitionStateHolder.getOffset()).isEqualTo(4L);
- }
-
- @Test
- public void testPunctuatedWatermarks() throws Exception {
- final String testTopic = "test topic name";
- Map<KafkaTopicPartition, Long> originalPartitions = new HashMap<>();
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 7),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 13),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
- originalPartitions.put(
- new KafkaTopicPartition(testTopic, 21),
- KafkaTopicPartitionStateSentinel.LATEST_OFFSET);
-
- TestSourceContext<Long> sourceContext = new TestSourceContext<>();
-
- TestProcessingTimeService processingTimeProvider = new TestProcessingTimeService();
-
- AssignerWithPunctuatedWatermarksAdapter.Strategy<Long> testWmStrategy =
- new AssignerWithPunctuatedWatermarksAdapter.Strategy<>(
- new PunctuatedTestExtractor());
-
- TestFetcher<Long> fetcher =
- new TestFetcher<>(
- sourceContext,
- originalPartitions,
- new SerializedValue<>(testWmStrategy),
- processingTimeProvider,
- 0);
-
- final KafkaTopicPartitionState<Long, Object> part1 =
- fetcher.subscribedPartitionStates().get(0);
- final KafkaTopicPartitionState<Long, Object> part2 =
- fetcher.subscribedPartitionStates().get(1);
- final KafkaTopicPartitionState<Long, Object> part3 =
- fetcher.subscribedPartitionStates().get(2);
-
- // elements generate a watermark if the timestamp is a multiple of three
-
- // elements for partition 1
- emitRecords(fetcher, Arrays.asList(1L, 2L), part1, 1L);
- emitRecord(fetcher, 2L, part1, 2L);
- emitRecords(fetcher, Arrays.asList(2L, 3L), part1, 3L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(3L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(3L);
- assertThat(sourceContext.hasWatermark()).isFalse();
-
- // elements for partition 2
- emitRecord(fetcher, 12L, part2, 1L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(12L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(12L);
- assertThat(sourceContext.hasWatermark()).isFalse();
-
- // elements for partition 3
- emitRecord(fetcher, 101L, part3, 1L);
- emitRecord(fetcher, 102L, part3, 2L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(102L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(102L);
-
- // now, we should have a watermark
- assertThat(sourceContext.hasWatermark()).isTrue();
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(3L);
-
- // advance partition 3
- emitRecord(fetcher, 1003L, part3, 3L);
- emitRecord(fetcher, 1004L, part3, 4L);
- emitRecord(fetcher, 1005L, part3, 5L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(1005L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(1005L);
-
- // advance partition 1 beyond partition 2 - this bumps the watermark
- emitRecord(fetcher, 30L, part1, 4L);
- assertThat(sourceContext.getLatestElement().getValue().longValue()).isEqualTo(30L);
- assertThat(sourceContext.getLatestElement().getTimestamp()).isEqualTo(30L);
- assertThat(sourceContext.hasWatermark()).isTrue();
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(12L);
-
- // advance partition 2 again - this bumps the watermark
- emitRecord(fetcher, 13L, part2, 2L);
- assertThat(sourceContext.hasWatermark()).isFalse();
- emitRecord(fetcher, 14L, part2, 3L);
- assertThat(sourceContext.hasWatermark()).isFalse();
- emitRecord(fetcher, 15L, part2, 3L);
- assertThat(sourceContext.hasWatermark()).isTrue();
- assertThat(sourceContext.getLatestWatermark().getTimestamp()).isEqualTo(15L);
- }
- }
-
- private static final class TestFetcher<T> extends AbstractFetcher<T, Object> {
- TestFetcher(
- SourceContext<T> sourceContext,
- Map<KafkaTopicPartition, Long> assignedPartitionsWithStartOffsets,
- SerializedValue<WatermarkStrategy<T>> watermarkStrategy,
- ProcessingTimeService processingTimeProvider,
- long autoWatermarkInterval)
- throws Exception {
- super(
- sourceContext,
- assignedPartitionsWithStartOffsets,
- watermarkStrategy,
- processingTimeProvider,
- autoWatermarkInterval,
- TestFetcher.class.getClassLoader(),
- new UnregisteredMetricsGroup(),
- false);
- }
-
- public void runFetchLoop() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void cancel() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- protected void doCommitInternalOffsetsToKafka(
- Map<KafkaTopicPartition, Long> offsets,
- @Nonnull KafkaCommitCallback commitCallback) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- protected Object createKafkaPartitionHandle(KafkaTopicPartition partition) {
- return new Object();
- }
- }
-
- private static <T, KPH> void emitRecord(
- AbstractFetcher<T, KPH> fetcher,
- T record,
- KafkaTopicPartitionState<T, KPH> partitionState,
- long offset) {
- ArrayDeque<T> recordQueue = new ArrayDeque<>();
- recordQueue.add(record);
-
- fetcher.emitRecordsWithTimestamps(recordQueue, partitionState, offset, Long.MIN_VALUE);
- }
-
- private static <T, KPH> void emitRecords(
- AbstractFetcher<T, KPH> fetcher,
- List<T> records,
- KafkaTopicPartitionState<T, KPH> partitionState,
- long offset) {
- ArrayDeque<T> recordQueue = new ArrayDeque<>(records);
-
- fetcher.emitRecordsWithTimestamps(recordQueue, partitionState, offset, Long.MIN_VALUE);
- }
-
- private static <T> Queue<T> emptyQueue() {
- return new ArrayDeque<>();
- }
-
- @SuppressWarnings("deprecation")
- private static class PeriodicTestExtractor implements AssignerWithPeriodicWatermarks<Long> {
-
- private volatile long maxTimestamp = Long.MIN_VALUE;
-
- @Override
- public long extractTimestamp(Long element, long previousElementTimestamp) {
- maxTimestamp = Math.max(maxTimestamp, element);
- return element;
- }
-
- @Nullable
- @Override
- public Watermark getCurrentWatermark() {
- return new Watermark(maxTimestamp);
- }
- }
-
- @SuppressWarnings("deprecation")
- private static class PunctuatedTestExtractor implements AssignerWithPunctuatedWatermarks<Long> {
-
- @Override
- public long extractTimestamp(Long element, long previousElementTimestamp) {
- return element;
- }
-
- @Nullable
- @Override
- public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
- return extractedTimestamp % 3 == 0 ? new Watermark(extractedTimestamp) : null;
- }
- }
-
- private static class PeriodicTestWatermarkGenerator implements WatermarkGenerator<Long> {
-
- private volatile long maxTimestamp = Long.MIN_VALUE;
-
- @Override
- public void onEvent(Long event, long eventTimestamp, WatermarkOutput output) {
- maxTimestamp = Math.max(maxTimestamp, event);
- }
-
- @Override
- public void onPeriodicEmit(WatermarkOutput output) {
- output.emitWatermark(new org.apache.flink.api.common.eventtime.Watermark(maxTimestamp));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java
deleted file mode 100644
index b475345..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java
+++ /dev/null
@@ -1,561 +0,0 @@
-/*
- * 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.internals;
-
-import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/**
- * Tests that the partition assignment in the partition discoverer is deterministic and stable, with
- * both fixed and growing partitions.
- */
-@RunWith(Parameterized.class)
-public class AbstractPartitionDiscovererTest {
-
- private static final String TEST_TOPIC = "test-topic";
- private static final String TEST_TOPIC_PATTERN = "^" + TEST_TOPIC + "[0-9]*$";
-
- private final KafkaTopicsDescriptor topicsDescriptor;
-
- public AbstractPartitionDiscovererTest(KafkaTopicsDescriptor topicsDescriptor) {
- this.topicsDescriptor = topicsDescriptor;
- }
-
- @Parameterized.Parameters(name = "KafkaTopicsDescriptor = {0}")
- @SuppressWarnings("unchecked")
- public static Collection<KafkaTopicsDescriptor[]> timeCharacteristic() {
- return Arrays.asList(
- new KafkaTopicsDescriptor[] {
- new KafkaTopicsDescriptor(Collections.singletonList(TEST_TOPIC), null)
- },
- new KafkaTopicsDescriptor[] {
- new KafkaTopicsDescriptor(null, Pattern.compile(TEST_TOPIC_PATTERN))
- });
- }
-
- @Test
- public void testPartitionsEqualConsumersFixedPartitions() throws Exception {
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn =
- Arrays.asList(
- new KafkaTopicPartition(TEST_TOPIC, 0),
- new KafkaTopicPartition(TEST_TOPIC, 1),
- new KafkaTopicPartition(TEST_TOPIC, 2),
- new KafkaTopicPartition(TEST_TOPIC, 3));
-
- int numSubtasks = mockGetAllPartitionsForTopicsReturn.size();
-
- // get the start index; the assertions below will fail if the assignment logic does not meet
- // correct contracts
- int numConsumers =
- KafkaTopicPartitionAssigner.assign(
- mockGetAllPartitionsForTopicsReturn.get(0), numSubtasks);
-
- for (int subtaskIndex = 0;
- subtaskIndex < mockGetAllPartitionsForTopicsReturn.size();
- subtaskIndex++) {
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- subtaskIndex,
- mockGetAllPartitionsForTopicsReturn.size(),
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockGetAllPartitionsForTopicsReturn));
- partitionDiscoverer.open();
-
- List<KafkaTopicPartition> initialDiscovery = partitionDiscoverer.discoverPartitions();
- assertThat(initialDiscovery).hasSize(1);
- assertThat(
- contains(
- mockGetAllPartitionsForTopicsReturn,
- initialDiscovery.get(0).getPartition()))
- .isTrue();
- assertThat(subtaskIndex)
- .isEqualTo(
- getExpectedSubtaskIndex(
- initialDiscovery.get(0), numConsumers, numSubtasks));
-
- // subsequent discoveries should not find anything
- List<KafkaTopicPartition> secondDiscovery = partitionDiscoverer.discoverPartitions();
- List<KafkaTopicPartition> thirdDiscovery = partitionDiscoverer.discoverPartitions();
- assertThat(secondDiscovery).isEmpty();
- assertThat(thirdDiscovery).isEmpty();
- }
- }
-
- @Test
- public void testMultiplePartitionsPerConsumersFixedPartitions() {
- try {
- final int[] partitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
-
- final List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn = new ArrayList<>();
- final Set<KafkaTopicPartition> allPartitions = new HashSet<>();
-
- for (int p : partitionIDs) {
- KafkaTopicPartition part = new KafkaTopicPartition(TEST_TOPIC, p);
- mockGetAllPartitionsForTopicsReturn.add(part);
- allPartitions.add(part);
- }
-
- final int numConsumers = 3;
- final int minPartitionsPerConsumer =
- mockGetAllPartitionsForTopicsReturn.size() / numConsumers;
- final int maxPartitionsPerConsumer =
- mockGetAllPartitionsForTopicsReturn.size() / numConsumers + 1;
-
- // get the start index; the assertions below will fail if the assignment logic does not
- // meet correct contracts
- int startIndex =
- KafkaTopicPartitionAssigner.assign(
- mockGetAllPartitionsForTopicsReturn.get(0), numConsumers);
-
- for (int subtaskIndex = 0; subtaskIndex < numConsumers; subtaskIndex++) {
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- subtaskIndex,
- numConsumers,
- TestPartitionDiscoverer
- .createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockGetAllPartitionsForTopicsReturn));
- partitionDiscoverer.open();
-
- List<KafkaTopicPartition> initialDiscovery =
- partitionDiscoverer.discoverPartitions();
- assertThat(initialDiscovery)
- .hasSizeGreaterThanOrEqualTo(minPartitionsPerConsumer)
- .hasSizeLessThanOrEqualTo(maxPartitionsPerConsumer);
-
- for (KafkaTopicPartition p : initialDiscovery) {
- // check that the element was actually contained
- assertThat(allPartitions.remove(p)).isTrue();
- assertThat(subtaskIndex)
- .isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- // subsequent discoveries should not find anything
- List<KafkaTopicPartition> secondDiscovery =
- partitionDiscoverer.discoverPartitions();
- List<KafkaTopicPartition> thirdDiscovery = partitionDiscoverer.discoverPartitions();
- assertThat(secondDiscovery).isEmpty();
- assertThat(thirdDiscovery).isEmpty();
- }
-
- // all partitions must have been assigned
- assertThat(allPartitions).isEmpty();
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testPartitionsFewerThanConsumersFixedPartitions() {
- try {
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn =
- Arrays.asList(
- new KafkaTopicPartition(TEST_TOPIC, 0),
- new KafkaTopicPartition(TEST_TOPIC, 1),
- new KafkaTopicPartition(TEST_TOPIC, 2),
- new KafkaTopicPartition(TEST_TOPIC, 3));
-
- final Set<KafkaTopicPartition> allPartitions = new HashSet<>();
- allPartitions.addAll(mockGetAllPartitionsForTopicsReturn);
-
- final int numConsumers = 2 * mockGetAllPartitionsForTopicsReturn.size() + 3;
-
- // get the start index; the assertions below will fail if the assignment logic does not
- // meet correct contracts
- int startIndex =
- KafkaTopicPartitionAssigner.assign(
- mockGetAllPartitionsForTopicsReturn.get(0), numConsumers);
-
- for (int subtaskIndex = 0; subtaskIndex < numConsumers; subtaskIndex++) {
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- subtaskIndex,
- numConsumers,
- TestPartitionDiscoverer
- .createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockGetAllPartitionsForTopicsReturn));
- partitionDiscoverer.open();
-
- List<KafkaTopicPartition> initialDiscovery =
- partitionDiscoverer.discoverPartitions();
- assertThat(initialDiscovery).hasSizeLessThanOrEqualTo(1);
-
- for (KafkaTopicPartition p : initialDiscovery) {
- // check that the element was actually contained
- assertThat(allPartitions.remove(p)).isTrue();
- assertThat(subtaskIndex)
- .isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- // subsequent discoveries should not find anything
- List<KafkaTopicPartition> secondDiscovery =
- partitionDiscoverer.discoverPartitions();
- List<KafkaTopicPartition> thirdDiscovery = partitionDiscoverer.discoverPartitions();
- assertThat(secondDiscovery).isEmpty();
- assertThat(thirdDiscovery).isEmpty();
- }
-
- // all partitions must have been assigned
- assertThat(allPartitions).isEmpty();
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testGrowingPartitions() {
- try {
- final int[] newPartitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
- List<KafkaTopicPartition> allPartitions = new ArrayList<>(11);
-
- for (int p : newPartitionIDs) {
- KafkaTopicPartition part = new KafkaTopicPartition(TEST_TOPIC, p);
- allPartitions.add(part);
- }
-
- // first discovery returns an initial subset of the partitions; second returns all
- // partitions
- List<List<KafkaTopicPartition>> mockGetAllPartitionsForTopicsReturnSequence =
- Arrays.asList(new ArrayList<>(allPartitions.subList(0, 7)), allPartitions);
-
- final Set<KafkaTopicPartition> allNewPartitions = new HashSet<>(allPartitions);
- final Set<KafkaTopicPartition> allInitialPartitions =
- new HashSet<>(mockGetAllPartitionsForTopicsReturnSequence.get(0));
-
- final int numConsumers = 3;
- final int minInitialPartitionsPerConsumer =
- mockGetAllPartitionsForTopicsReturnSequence.get(0).size() / numConsumers;
- final int maxInitialPartitionsPerConsumer =
- mockGetAllPartitionsForTopicsReturnSequence.get(0).size() / numConsumers + 1;
- final int minNewPartitionsPerConsumer = allPartitions.size() / numConsumers;
- final int maxNewPartitionsPerConsumer = allPartitions.size() / numConsumers + 1;
-
- // get the start index; the assertions below will fail if the assignment logic does not
- // meet correct contracts
- int startIndex = KafkaTopicPartitionAssigner.assign(allPartitions.get(0), numConsumers);
-
- TestPartitionDiscoverer partitionDiscovererSubtask0 =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- 0,
- numConsumers,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- deepClone(mockGetAllPartitionsForTopicsReturnSequence));
- partitionDiscovererSubtask0.open();
-
- TestPartitionDiscoverer partitionDiscovererSubtask1 =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- 1,
- numConsumers,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- deepClone(mockGetAllPartitionsForTopicsReturnSequence));
- partitionDiscovererSubtask1.open();
-
- TestPartitionDiscoverer partitionDiscovererSubtask2 =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- 2,
- numConsumers,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList(TEST_TOPIC)),
- deepClone(mockGetAllPartitionsForTopicsReturnSequence));
- partitionDiscovererSubtask2.open();
-
- List<KafkaTopicPartition> initialDiscoverySubtask0 =
- partitionDiscovererSubtask0.discoverPartitions();
- List<KafkaTopicPartition> initialDiscoverySubtask1 =
- partitionDiscovererSubtask1.discoverPartitions();
- List<KafkaTopicPartition> initialDiscoverySubtask2 =
- partitionDiscovererSubtask2.discoverPartitions();
-
- assertThat(initialDiscoverySubtask0)
- .hasSizeGreaterThanOrEqualTo(minInitialPartitionsPerConsumer)
- .hasSizeLessThanOrEqualTo(maxInitialPartitionsPerConsumer);
- assertThat(initialDiscoverySubtask1)
- .hasSizeGreaterThanOrEqualTo(minInitialPartitionsPerConsumer)
- .hasSizeLessThanOrEqualTo(maxInitialPartitionsPerConsumer);
- assertThat(initialDiscoverySubtask2)
- .hasSizeGreaterThanOrEqualTo(minInitialPartitionsPerConsumer)
- .hasSizeLessThanOrEqualTo(maxInitialPartitionsPerConsumer);
-
- for (KafkaTopicPartition p : initialDiscoverySubtask0) {
- // check that the element was actually contained
- assertThat(allInitialPartitions.remove(p)).isTrue();
- assertThat(0).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : initialDiscoverySubtask1) {
- // check that the element was actually contained
- assertThat(allInitialPartitions.remove(p)).isTrue();
- assertThat(1).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : initialDiscoverySubtask2) {
- // check that the element was actually contained
- assertThat(allInitialPartitions.remove(p)).isTrue();
- assertThat(2).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- // all partitions must have been assigned
- assertThat(allInitialPartitions).isEmpty();
-
- // now, execute discover again (should find the extra new partitions)
- List<KafkaTopicPartition> secondDiscoverySubtask0 =
- partitionDiscovererSubtask0.discoverPartitions();
- List<KafkaTopicPartition> secondDiscoverySubtask1 =
- partitionDiscovererSubtask1.discoverPartitions();
- List<KafkaTopicPartition> secondDiscoverySubtask2 =
- partitionDiscovererSubtask2.discoverPartitions();
-
- // new discovered partitions must not have been discovered before
- assertThat(Collections.disjoint(secondDiscoverySubtask0, initialDiscoverySubtask0))
- .isTrue();
- assertThat(Collections.disjoint(secondDiscoverySubtask1, initialDiscoverySubtask1))
- .isTrue();
- assertThat(Collections.disjoint(secondDiscoverySubtask2, initialDiscoverySubtask2))
- .isTrue();
-
- assertThat(secondDiscoverySubtask0.size() + initialDiscoverySubtask0.size())
- .isGreaterThanOrEqualTo(minNewPartitionsPerConsumer);
- assertThat(secondDiscoverySubtask0.size() + initialDiscoverySubtask0.size())
- .isLessThanOrEqualTo(maxNewPartitionsPerConsumer);
- assertThat(secondDiscoverySubtask1.size() + initialDiscoverySubtask1.size())
- .isGreaterThanOrEqualTo(minNewPartitionsPerConsumer);
- assertThat(secondDiscoverySubtask1.size() + initialDiscoverySubtask1.size())
- .isLessThanOrEqualTo(maxNewPartitionsPerConsumer);
- assertThat(secondDiscoverySubtask2.size() + initialDiscoverySubtask2.size())
- .isGreaterThanOrEqualTo(minNewPartitionsPerConsumer);
- assertThat(secondDiscoverySubtask2.size() + initialDiscoverySubtask2.size())
- .isLessThanOrEqualTo(maxNewPartitionsPerConsumer);
-
- // check that the two discoveries combined form all partitions
-
- for (KafkaTopicPartition p : initialDiscoverySubtask0) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(0).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : initialDiscoverySubtask1) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(1).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : initialDiscoverySubtask2) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(2).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : secondDiscoverySubtask0) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(0).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : secondDiscoverySubtask1) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(1).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- for (KafkaTopicPartition p : secondDiscoverySubtask2) {
- assertThat(allNewPartitions.remove(p)).isTrue();
- assertThat(2).isEqualTo(getExpectedSubtaskIndex(p, startIndex, numConsumers));
- }
-
- // all partitions must have been assigned
- assertThat(allNewPartitions).isEmpty();
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering()
- throws Exception {
- int numSubtasks = 4;
-
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn =
- Arrays.asList(
- new KafkaTopicPartition("test-topic", 0),
- new KafkaTopicPartition("test-topic", 1),
- new KafkaTopicPartition("test-topic", 2),
- new KafkaTopicPartition("test-topic", 3),
- new KafkaTopicPartition("test-topic2", 0),
- new KafkaTopicPartition("test-topic2", 1));
-
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturnOutOfOrder =
- Arrays.asList(
- new KafkaTopicPartition("test-topic", 3),
- new KafkaTopicPartition("test-topic", 1),
- new KafkaTopicPartition("test-topic2", 1),
- new KafkaTopicPartition("test-topic", 0),
- new KafkaTopicPartition("test-topic2", 0),
- new KafkaTopicPartition("test-topic", 2));
-
- for (int subtaskIndex = 0; subtaskIndex < numSubtasks; subtaskIndex++) {
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- subtaskIndex,
- numSubtasks,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Arrays.asList("test-topic", "test-topic2")),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockGetAllPartitionsForTopicsReturn));
- partitionDiscoverer.open();
-
- TestPartitionDiscoverer partitionDiscovererOutOfOrder =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- subtaskIndex,
- numSubtasks,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Arrays.asList("test-topic", "test-topic2")),
- TestPartitionDiscoverer
- .createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- mockGetAllPartitionsForTopicsReturnOutOfOrder));
- partitionDiscovererOutOfOrder.open();
-
- List<KafkaTopicPartition> discoveredPartitions =
- partitionDiscoverer.discoverPartitions();
- List<KafkaTopicPartition> discoveredPartitionsOutOfOrder =
- partitionDiscovererOutOfOrder.discoverPartitions();
-
- // the subscribed partitions should be identical, regardless of the input partition
- // ordering
- Collections.sort(discoveredPartitions, new KafkaTopicPartition.Comparator());
- Collections.sort(discoveredPartitionsOutOfOrder, new KafkaTopicPartition.Comparator());
- assertThat(discoveredPartitionsOutOfOrder).isEqualTo(discoveredPartitions);
- }
- }
-
- @Test
- public void testNonContiguousPartitionIdDiscovery() throws Exception {
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn1 =
- Arrays.asList(
- new KafkaTopicPartition("test-topic", 1),
- new KafkaTopicPartition("test-topic", 4));
-
- List<KafkaTopicPartition> mockGetAllPartitionsForTopicsReturn2 =
- Arrays.asList(
- new KafkaTopicPartition("test-topic", 0),
- new KafkaTopicPartition("test-topic", 1),
- new KafkaTopicPartition("test-topic", 2),
- new KafkaTopicPartition("test-topic", 3),
- new KafkaTopicPartition("test-topic", 4));
-
- TestPartitionDiscoverer partitionDiscoverer =
- new TestPartitionDiscoverer(
- topicsDescriptor,
- 0,
- 1,
- TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(
- Collections.singletonList("test-topic")),
- // first metadata fetch has missing partitions that appears only in the
- // second fetch;
- // need to create new modifiable lists for each fetch, since internally
- // Iterable.remove() is used.
- Arrays.asList(
- new ArrayList<>(mockGetAllPartitionsForTopicsReturn1),
- new ArrayList<>(mockGetAllPartitionsForTopicsReturn2)));
- partitionDiscoverer.open();
-
- List<KafkaTopicPartition> discoveredPartitions1 = partitionDiscoverer.discoverPartitions();
- assertThat(discoveredPartitions1)
- .hasSize(2)
- .contains(
- new KafkaTopicPartition("test-topic", 1),
- new KafkaTopicPartition("test-topic", 4));
-
- List<KafkaTopicPartition> discoveredPartitions2 = partitionDiscoverer.discoverPartitions();
- assertThat(discoveredPartitions2)
- .hasSize(3)
- .contains(
- new KafkaTopicPartition("test-topic", 0),
- new KafkaTopicPartition("test-topic", 2),
- new KafkaTopicPartition("test-topic", 3));
- }
-
- private boolean contains(List<KafkaTopicPartition> partitions, int partition) {
- for (KafkaTopicPartition ktp : partitions) {
- if (ktp.getPartition() == partition) {
- return true;
- }
- }
-
- return false;
- }
-
- private List<List<KafkaTopicPartition>> deepClone(List<List<KafkaTopicPartition>> toClone) {
- List<List<KafkaTopicPartition>> clone = new ArrayList<>(toClone.size());
- for (List<KafkaTopicPartition> partitionsToClone : toClone) {
- List<KafkaTopicPartition> clonePartitions = new ArrayList<>(partitionsToClone.size());
- clonePartitions.addAll(partitionsToClone);
-
- clone.add(clonePartitions);
- }
-
- return clone;
- }
-
- /**
- * Utility method that determines the expected subtask index a partition should be assigned to,
- * depending on the start index and using the partition id as the offset from that start index
- * in clockwise direction.
- *
- * <p>The expectation is based on the distribution contract of {@link
- * KafkaTopicPartitionAssigner#assign(KafkaTopicPartition, int)}.
- */
- private static int getExpectedSubtaskIndex(
- KafkaTopicPartition partition, int startIndex, int numSubtasks) {
- return (startIndex + partition.getPartition()) % numSubtasks;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
deleted file mode 100644
index 8697b14..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java
+++ /dev/null
@@ -1,616 +0,0 @@
-/*
- * 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.internals;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Arrays.asList;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Tests for the {@link ClosableBlockingQueue}. */
-public class ClosableBlockingQueueTest {
-
- // ------------------------------------------------------------------------
- // single-threaded unit tests
- // ------------------------------------------------------------------------
-
- @Test
- public void testCreateQueueHashCodeEquals() {
- try {
- ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
- ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>(22);
-
- assertThat(queue1.isOpen()).isTrue();
- assertThat(queue2.isOpen()).isTrue();
- assertThat(queue1.isEmpty()).isTrue();
- assertThat(queue2.isEmpty()).isTrue();
- assertThat(queue1.size()).isEqualTo(0);
- assertThat(queue2.size()).isEqualTo(0);
-
- assertThat(queue1.hashCode()).isEqualTo(queue2.hashCode());
- //noinspection EqualsWithItself
- assertThat(queue1.equals(queue1)).isTrue();
- //noinspection EqualsWithItself
- assertThat(queue2.equals(queue2)).isTrue();
- assertThat(queue1.equals(queue2)).isTrue();
-
- assertThat(queue1.toString()).isNotNull();
- assertThat(queue2.toString()).isNotNull();
-
- List<String> elements = new ArrayList<>();
- elements.add("a");
- elements.add("b");
- elements.add("c");
-
- ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>(elements);
- ClosableBlockingQueue<String> queue4 =
- new ClosableBlockingQueue<>(asList("a", "b", "c"));
-
- assertThat(queue3.isOpen()).isTrue();
- assertThat(queue4.isOpen()).isTrue();
- assertThat(queue3.isEmpty()).isFalse();
- assertThat(queue4.isEmpty()).isFalse();
- assertThat(queue3.size()).isEqualTo(3);
- assertThat(queue4.size()).isEqualTo(3);
-
- assertThat(queue3.hashCode()).isEqualTo(queue4.hashCode());
- //noinspection EqualsWithItself
- assertThat(queue3.equals(queue3)).isTrue();
- //noinspection EqualsWithItself
- assertThat(queue4.equals(queue4)).isTrue();
- assertThat(queue3.equals(queue4)).isTrue();
-
- assertThat(queue3.toString()).isNotNull();
- assertThat(queue4.toString()).isNotNull();
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCloseEmptyQueue() {
- try {
- ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
- assertThat(queue.isOpen()).isTrue();
- assertThat(queue.close()).isTrue();
- assertThat(queue.isOpen()).isFalse();
-
- assertThat(queue.addIfOpen("element")).isFalse();
- assertThat(queue.isEmpty()).isTrue();
-
- try {
- queue.add("some element");
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testCloseNonEmptyQueue() {
- try {
- ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>(asList(1, 2, 3));
- assertThat(queue.isOpen()).isTrue();
-
- assertThat(queue.close()).isFalse();
- assertThat(queue.close()).isFalse();
-
- queue.poll();
-
- assertThat(queue.close()).isFalse();
- assertThat(queue.close()).isFalse();
-
- queue.pollBatch();
-
- assertThat(queue.close()).isTrue();
- assertThat(queue.isOpen()).isFalse();
-
- assertThat(queue.addIfOpen(42)).isFalse();
- assertThat(queue.isEmpty()).isTrue();
-
- try {
- queue.add(99);
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testPeekAndPoll() {
- try {
- ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
- assertThat(queue.peek()).isNull();
- assertThat(queue.peek()).isNull();
- assertThat(queue.poll()).isNull();
- assertThat(queue.poll()).isNull();
-
- assertThat(queue.size()).isEqualTo(0);
-
- queue.add("a");
- queue.add("b");
- queue.add("c");
-
- assertThat(queue.size()).isEqualTo(3);
-
- assertThat(queue.peek()).isEqualTo("a");
- assertThat(queue.peek()).isEqualTo("a");
- assertThat(queue.peek()).isEqualTo("a");
-
- assertThat(queue.size()).isEqualTo(3);
-
- assertThat(queue.poll()).isEqualTo("a");
- assertThat(queue.poll()).isEqualTo("b");
-
- assertThat(queue.size()).isEqualTo(1);
-
- assertThat(queue.peek()).isEqualTo("c");
- assertThat(queue.peek()).isEqualTo("c");
-
- assertThat(queue.poll()).isEqualTo("c");
-
- assertThat(queue.size()).isEqualTo(0);
- assertThat(queue.poll()).isNull();
- assertThat(queue.peek()).isNull();
- assertThat(queue.peek()).isNull();
-
- assertThat(queue.close()).isTrue();
-
- try {
- queue.peek();
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
-
- try {
- queue.poll();
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testPollBatch() {
- try {
- ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
- assertThat(queue.pollBatch()).isNull();
-
- queue.add("a");
- queue.add("b");
-
- assertThat(queue.pollBatch()).isEqualTo(asList("a", "b"));
- assertThat(queue.pollBatch()).isNull();
-
- queue.add("c");
-
- assertThat(queue.pollBatch()).containsExactly("c");
- assertThat(queue.pollBatch()).isNull();
-
- assertThat(queue.close()).isTrue();
-
- try {
- queue.pollBatch();
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testGetElementBlocking() {
- try {
- ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
- assertThat(queue.getElementBlocking(1)).isNull();
- assertThat(queue.getElementBlocking(3)).isNull();
- assertThat(queue.getElementBlocking(2)).isNull();
-
- assertThat(queue.size()).isEqualTo(0);
-
- queue.add("a");
- queue.add("b");
- queue.add("c");
- queue.add("d");
- queue.add("e");
- queue.add("f");
-
- assertThat(queue.size()).isEqualTo(6);
-
- assertThat(queue.getElementBlocking(99)).isEqualTo("a");
- assertThat(queue.getElementBlocking()).isEqualTo("b");
-
- assertThat(queue.size()).isEqualTo(4);
-
- assertThat(queue.getElementBlocking(0)).isEqualTo("c");
- assertThat(queue.getElementBlocking(1000000)).isEqualTo("d");
- assertThat(queue.getElementBlocking()).isEqualTo("e");
- assertThat(queue.getElementBlocking(1786598)).isEqualTo("f");
-
- assertThat(queue.size()).isEqualTo(0);
-
- assertThat(queue.getElementBlocking(1)).isNull();
- assertThat(queue.getElementBlocking(3)).isNull();
- assertThat(queue.getElementBlocking(2)).isNull();
-
- assertThat(queue.close()).isTrue();
-
- try {
- queue.getElementBlocking();
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
-
- try {
- queue.getElementBlocking(1000000000L);
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @Test
- public void testGetBatchBlocking() {
- try {
- ClosableBlockingQueue<String> queue = new ClosableBlockingQueue<>();
-
- assertThat(queue.getBatchBlocking(1)).isEmpty();
- assertThat(queue.getBatchBlocking(3)).isEmpty();
- assertThat(queue.getBatchBlocking(2)).isEmpty();
-
- queue.add("a");
- queue.add("b");
-
- assertThat(queue.getBatchBlocking(900000009)).isEqualTo(asList("a", "b"));
-
- queue.add("c");
- queue.add("d");
-
- assertThat(queue.getBatchBlocking()).isEqualTo(asList("c", "d"));
-
- assertThat(queue.getBatchBlocking(2)).isEmpty();
-
- queue.add("e");
-
- assertThat(queue.getBatchBlocking(0)).containsExactly("e");
-
- queue.add("f");
-
- assertThat(queue.getBatchBlocking(1000000000)).containsExactly("f");
-
- assertThat(queue.size()).isEqualTo(0);
-
- assertThat(queue.getBatchBlocking(1)).isEmpty();
- assertThat(queue.getBatchBlocking(3)).isEmpty();
- assertThat(queue.getBatchBlocking(2)).isEmpty();
-
- assertThat(queue.close()).isTrue();
-
- try {
- queue.getBatchBlocking();
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
-
- try {
- queue.getBatchBlocking(1000000000L);
- fail("should cause an exception");
- } catch (IllegalStateException ignored) {
- // expected
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- // ------------------------------------------------------------------------
- // multi-threaded tests
- // ------------------------------------------------------------------------
-
- @Test
- public void notifyOnClose() {
- try {
- final long oneYear = 365L * 24 * 60 * 60 * 1000;
-
- // test "getBatchBlocking()"
- final ClosableBlockingQueue<String> queue1 = new ClosableBlockingQueue<>();
- QueueCall call1 =
- new QueueCall() {
- @Override
- public void call() throws Exception {
- queue1.getBatchBlocking();
- }
- };
- testCallExitsOnClose(call1, queue1);
-
- // test "getBatchBlocking()"
- final ClosableBlockingQueue<String> queue2 = new ClosableBlockingQueue<>();
- QueueCall call2 =
- new QueueCall() {
- @Override
- public void call() throws Exception {
- queue2.getBatchBlocking(oneYear);
- }
- };
- testCallExitsOnClose(call2, queue2);
-
- // test "getBatchBlocking()"
- final ClosableBlockingQueue<String> queue3 = new ClosableBlockingQueue<>();
- QueueCall call3 =
- new QueueCall() {
- @Override
- public void call() throws Exception {
- queue3.getElementBlocking();
- }
- };
- testCallExitsOnClose(call3, queue3);
-
- // test "getBatchBlocking()"
- final ClosableBlockingQueue<String> queue4 = new ClosableBlockingQueue<>();
- QueueCall call4 =
- new QueueCall() {
- @Override
- public void call() throws Exception {
- queue4.getElementBlocking(oneYear);
- }
- };
- testCallExitsOnClose(call4, queue4);
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
- @Test
- public void testMultiThreadedAddGet() {
- try {
- final ClosableBlockingQueue<Integer> queue = new ClosableBlockingQueue<>();
- final AtomicReference<Throwable> pushErrorRef = new AtomicReference<>();
- final AtomicReference<Throwable> pollErrorRef = new AtomicReference<>();
-
- final int numElements = 2000;
-
- Thread pusher =
- new Thread("pusher") {
-
- @Override
- public void run() {
- try {
- final Random rnd = new Random();
- for (int i = 0; i < numElements; i++) {
- queue.add(i);
-
- // sleep a bit, sometimes
- int sleepTime = rnd.nextInt(3);
- if (sleepTime > 1) {
- Thread.sleep(sleepTime);
- }
- }
-
- while (true) {
- if (queue.close()) {
- break;
- } else {
- Thread.sleep(5);
- }
- }
- } catch (Throwable t) {
- pushErrorRef.set(t);
- }
- }
- };
- pusher.start();
-
- Thread poller =
- new Thread("poller") {
-
- @SuppressWarnings("InfiniteLoopStatement")
- @Override
- public void run() {
- try {
- int count = 0;
-
- try {
- final Random rnd = new Random();
- int nextExpected = 0;
-
- while (true) {
- int getMethod = count % 7;
- switch (getMethod) {
- case 0:
- {
- Integer next = queue.getElementBlocking(1);
- if (next != null) {
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- break;
- }
- case 1:
- {
- List<Integer> nextList =
- queue.getBatchBlocking();
- for (Integer next : nextList) {
- assertThat(next).isNotNull();
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- break;
- }
- case 2:
- {
- List<Integer> nextList =
- queue.getBatchBlocking(1);
- if (nextList != null) {
- for (Integer next : nextList) {
- assertThat(next).isNotNull();
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- }
- break;
- }
- case 3:
- {
- Integer next = queue.poll();
- if (next != null) {
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- break;
- }
- case 4:
- {
- List<Integer> nextList = queue.pollBatch();
- if (nextList != null) {
- for (Integer next : nextList) {
- assertThat(next).isNotNull();
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- }
- break;
- }
- default:
- {
- Integer next = queue.getElementBlocking();
- assertThat(next).isNotNull();
- assertThat(next.intValue())
- .isEqualTo(nextExpected);
- nextExpected++;
- count++;
- }
- }
-
- // sleep a bit, sometimes
- int sleepTime = rnd.nextInt(3);
- if (sleepTime > 1) {
- Thread.sleep(sleepTime);
- }
- }
- } catch (IllegalStateException e) {
- // we get this once the queue is closed
- assertThat(count).isEqualTo(numElements);
- }
- } catch (Throwable t) {
- pollErrorRef.set(t);
- }
- }
- };
- poller.start();
-
- pusher.join();
- poller.join();
-
- if (pushErrorRef.get() != null) {
- Throwable t = pushErrorRef.get();
- t.printStackTrace();
- fail("Error in pusher: " + t.getMessage());
- }
- if (pollErrorRef.get() != null) {
- Throwable t = pollErrorRef.get();
- t.printStackTrace();
- fail("Error in poller: " + t.getMessage());
- }
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- // ------------------------------------------------------------------------
- // Utils
- // ------------------------------------------------------------------------
-
- private static void testCallExitsOnClose(
- final QueueCall call, ClosableBlockingQueue<String> queue) throws Exception {
-
- final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-
- Runnable runnable =
- new Runnable() {
- @Override
- public void run() {
- try {
- call.call();
- } catch (Throwable t) {
- errorRef.set(t);
- }
- }
- };
-
- Thread thread = new Thread(runnable);
- thread.start();
- Thread.sleep(100);
- queue.close();
- thread.join();
-
- @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
- Throwable cause = errorRef.get();
- assertThat(cause).isInstanceOf(IllegalStateException.class);
- }
-
- private interface QueueCall {
- void call() throws Exception;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
deleted file mode 100644
index 30e6516..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.internals;
-
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Tests for the {@link KafkaTopicPartition}. */
-public class KafkaTopicPartitionTest {
-
- @Test
- public void validateUid() {
- Field uidField;
- try {
- uidField = KafkaTopicPartition.class.getDeclaredField("serialVersionUID");
- uidField.setAccessible(true);
- } catch (NoSuchFieldException e) {
- fail("serialVersionUID is not defined");
- return;
- }
-
- assertThat(Modifier.isStatic(uidField.getModifiers())).isTrue();
- assertThat(Modifier.isFinal(uidField.getModifiers())).isTrue();
- assertThat(Modifier.isPrivate(uidField.getModifiers())).isTrue();
-
- assertThat(uidField.getType()).isEqualTo(long.class);
-
- // the UID has to be constant to make sure old checkpoints/savepoints can be read
- try {
- assertThat(uidField.getLong(null)).isEqualTo(722083576322742325L);
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java
deleted file mode 100644
index bb029d8..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.internals;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.regex.Pattern;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for the {@link KafkaTopicsDescriptor}. */
-@RunWith(Parameterized.class)
-public class KafkaTopicsDescriptorTest {
-
- @Parameterized.Parameters
- public static Collection<Object[]> data() {
- return Arrays.asList(
- new Object[][] {
- {"topic1", null, Arrays.asList("topic1", "topic2", "topic3"), true},
- {"topic1", null, Arrays.asList("topic2", "topic3"), false},
- {"topic1", Pattern.compile("topic[0-9]"), null, true},
- {"topicx", Pattern.compile("topic[0-9]"), null, false}
- });
- }
-
- private String topic;
- private Pattern topicPattern;
- private List<String> fixedTopics;
- boolean expected;
-
- public KafkaTopicsDescriptorTest(
- String topic, Pattern topicPattern, List<String> fixedTopics, boolean expected) {
- this.topic = topic;
- this.topicPattern = topicPattern;
- this.fixedTopics = fixedTopics;
- this.expected = expected;
- }
-
- @Test
- public void testIsMatchingTopic() {
- KafkaTopicsDescriptor topicsDescriptor =
- new KafkaTopicsDescriptor(fixedTopics, topicPattern);
-
- assertThat(topicsDescriptor.isMatchingTopic(topic)).isEqualTo(expected);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java
deleted file mode 100644
index 952f0fd..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.internals.metrics;
-
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.function.Function;
-import java.util.stream.Stream;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-
-@Testcontainers
-@ExtendWith(TestLoggerExtension.class)
-class KafkaMetricMutableWrapperTest {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricMutableWrapperTest.class);
- private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- private static final Network NETWORK = Network.newNetwork();
-
- @Container
- public static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG)
- .withEmbeddedZookeeper()
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS);
-
- @Test
- void testOnlyMeasurableMetricsAreRegisteredWithMutableWrapper() {
- testOnlyMeasurableMetricsAreRegistered(KafkaMetricMutableWrapper::new);
- }
-
- @Test
- void testOnlyMeasurableMetricsAreRegistered() {
- testOnlyMeasurableMetricsAreRegistered(KafkaMetricWrapper::new);
- }
-
- private static void testOnlyMeasurableMetricsAreRegistered(
- Function<Metric, Gauge<Double>> wrapperFactory) {
- final Collection<Gauge<Double>> metricWrappers = new ArrayList<>();
- final KafkaConsumer<?, ?> consumer = new KafkaConsumer<>(getKafkaClientConfiguration());
- final KafkaProducer<?, ?> producer = new KafkaProducer<>(getKafkaClientConfiguration());
- Stream.concat(consumer.metrics().values().stream(), producer.metrics().values().stream())
- .map(wrapperFactory::apply)
- .forEach(metricWrappers::add);
-
- // Ensure that all values are accessible and return valid double values
- metricWrappers.forEach(Gauge::getValue);
- }
-
- private static Properties getKafkaClientConfiguration() {
- final Properties standardProps = new Properties();
- standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers());
- standardProps.put("group.id", UUID.randomUUID().toString());
- standardProps.put("enable.auto.commit", false);
- standardProps.put("key.deserializer", ByteArrayDeserializer.class.getName());
- standardProps.put("value.deserializer", ByteArrayDeserializer.class.getName());
- standardProps.put("key.serializer", ByteArraySerializer.class.getName());
- standardProps.put("value.serializer", ByteArraySerializer.class.getName());
- standardProps.put("auto.offset.reset", "earliest");
- standardProps.put("max.partition.fetch.bytes", 256);
- return standardProps;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java
deleted file mode 100644
index 7d37f6c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.util.UUID;
-
-import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime;
-import static org.apache.flink.streaming.api.TimeCharacteristic.IngestionTime;
-import static org.apache.flink.streaming.api.TimeCharacteristic.ProcessingTime;
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-
-/** Failure Recovery IT Test for KafkaShuffle. */
-public class KafkaShuffleExactlyOnceITCase extends KafkaShuffleTestBase {
-
- @Rule public final Timeout timeout = Timeout.millis(600000L);
-
- /**
- * Failure Recovery after processing 2/3 data with time characteristic: ProcessingTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testFailureRecoveryProcessingTime() throws Exception {
- testKafkaShuffleFailureRecovery(1000, ProcessingTime);
- }
-
- /**
- * Failure Recovery after processing 2/3 data with time characteristic: IngestionTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testFailureRecoveryIngestionTime() throws Exception {
- testKafkaShuffleFailureRecovery(1000, IngestionTime);
- }
-
- /**
- * Failure Recovery after processing 2/3 data with time characteristic: EventTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testFailureRecoveryEventTime() throws Exception {
- testKafkaShuffleFailureRecovery(1000, EventTime);
- }
-
- /**
- * Failure Recovery after data is repartitioned with time characteristic: ProcessingTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionFailureRecoveryProcessingTime() throws Exception {
- testAssignedToPartitionFailureRecovery(500, ProcessingTime);
- }
-
- /**
- * Failure Recovery after data is repartitioned with time characteristic: IngestionTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionFailureRecoveryIngestionTime() throws Exception {
- testAssignedToPartitionFailureRecovery(500, IngestionTime);
- }
-
- /**
- * Failure Recovery after data is repartitioned with time characteristic: EventTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionFailureRecoveryEventTime() throws Exception {
- testAssignedToPartitionFailureRecovery(500, EventTime);
- }
-
- /**
- * To test failure recovery after processing 2/3 data.
- *
- * <p>Schema: (key, timestamp, source instance Id). Producer Parallelism = 1; Kafka Partition #
- * = 1; Consumer Parallelism = 1
- */
- private void testKafkaShuffleFailureRecovery(
- int numElementsPerProducer, TimeCharacteristic timeCharacteristic) throws Exception {
-
- String topic = topic("failure_recovery-" + UUID.randomUUID(), timeCharacteristic);
- final int numberOfPartitions = 1;
- final int producerParallelism = 1;
- final int failAfterElements = numElementsPerProducer * numberOfPartitions * 2 / 3;
-
- createTestTopic(topic, numberOfPartitions, 1);
-
- final StreamExecutionEnvironment env =
- createEnvironment(producerParallelism, timeCharacteristic).enableCheckpointing(500);
-
- createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions)
- .map(new FailingIdentityMapper<>(failAfterElements))
- .setParallelism(1)
- .map(new ToInteger(producerParallelism))
- .setParallelism(1)
- .addSink(
- new ValidatingExactlyOnceSink(numElementsPerProducer * producerParallelism))
- .setParallelism(1);
-
- FailingIdentityMapper.failedBefore = false;
-
- tryExecute(env, topic);
-
- deleteTestTopic(topic);
- }
-
- /**
- * To test failure recovery with partition assignment after processing 2/3 data.
- *
- * <p>Schema: (key, timestamp, source instance Id). Producer Parallelism = 2; Kafka Partition #
- * = 3; Consumer Parallelism = 3
- */
- private void testAssignedToPartitionFailureRecovery(
- int numElementsPerProducer, TimeCharacteristic timeCharacteristic) throws Exception {
- String topic = topic("partition_failure_recovery-" + UUID.randomUUID(), timeCharacteristic);
- final int numberOfPartitions = 3;
- final int producerParallelism = 2;
- final int failAfterElements = numElementsPerProducer * producerParallelism * 2 / 3;
-
- createTestTopic(topic, numberOfPartitions, 1);
-
- final StreamExecutionEnvironment env =
- createEnvironment(producerParallelism, timeCharacteristic);
-
- KeyedStream<Tuple3<Integer, Long, Integer>, Tuple> keyedStream =
- createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions);
- keyedStream
- .process(
- new PartitionValidator(
- keyedStream.getKeySelector(), numberOfPartitions, topic))
- .setParallelism(numberOfPartitions)
- .map(new ToInteger(producerParallelism))
- .setParallelism(numberOfPartitions)
- .map(new FailingIdentityMapper<>(failAfterElements))
- .setParallelism(1)
- .addSink(
- new ValidatingExactlyOnceSink(numElementsPerProducer * producerParallelism))
- .setParallelism(1);
-
- FailingIdentityMapper.failedBefore = false;
-
- tryExecute(env, topic);
-
- deleteTestTopic(topic);
- }
-
- private StreamExecutionEnvironment createEnvironment(
- int producerParallelism, TimeCharacteristic timeCharacteristic) {
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(producerParallelism);
- env.setStreamTimeCharacteristic(timeCharacteristic);
- env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
- env.setBufferTimeout(0);
- env.enableCheckpointing(500);
-
- return env;
- }
-
- private static class ToInteger implements MapFunction<Tuple3<Integer, Long, Integer>, Integer> {
- private final int producerParallelism;
-
- ToInteger(int producerParallelism) {
- this.producerParallelism = producerParallelism;
- }
-
- @Override
- public Integer map(Tuple3<Integer, Long, Integer> element) throws Exception {
-
- return element.f0 * producerParallelism + element.f2;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java
deleted file mode 100644
index 9a0a14d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleElement;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleElementDeserializer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleRecord;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleWatermark;
-import org.apache.flink.util.PropertiesUtil;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
-import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
-
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime;
-import static org.apache.flink.streaming.api.TimeCharacteristic.IngestionTime;
-import static org.apache.flink.streaming.api.TimeCharacteristic.ProcessingTime;
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle.PARTITION_NUMBER;
-import static org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle.PRODUCER_PARALLELISM;
-import static org.apache.flink.test.util.TestUtils.tryExecute;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Simple End to End Test for Kafka. */
-public class KafkaShuffleITCase extends KafkaShuffleTestBase {
-
- @Rule public final Timeout timeout = Timeout.millis(600000L);
-
- /**
- * To test no data is lost or duplicated end-2-end with the default time characteristic:
- * ProcessingTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSimpleProcessingTime() throws Exception {
- testKafkaShuffle(200000, ProcessingTime);
- }
-
- /**
- * To test no data is lost or duplicated end-2-end with time characteristic: IngestionTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSimpleIngestionTime() throws Exception {
- testKafkaShuffle(200000, IngestionTime);
- }
-
- /**
- * To test no data is lost or duplicated end-2-end with time characteristic: EventTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSimpleEventTime() throws Exception {
- testKafkaShuffle(100000, EventTime);
- }
-
- /**
- * To test data is partitioned to the right partition with time characteristic: ProcessingTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionProcessingTime() throws Exception {
- testAssignedToPartition(300000, ProcessingTime);
- }
-
- /**
- * To test data is partitioned to the right partition with time characteristic: IngestionTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionIngestionTime() throws Exception {
- testAssignedToPartition(300000, IngestionTime);
- }
-
- /**
- * To test data is partitioned to the right partition with time characteristic: EventTime.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testAssignedToPartitionEventTime() throws Exception {
- testAssignedToPartition(100000, EventTime);
- }
-
- /**
- * To test watermark is monotonically incremental with randomized watermark.
- *
- * <p>Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3.
- */
- @Test
- public void testWatermarkIncremental() throws Exception {
- testWatermarkIncremental(100000);
- }
-
- /**
- * To test value serialization and deserialization with time characteristic: ProcessingTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSerDeProcessingTime() throws Exception {
- testRecordSerDe(ProcessingTime);
- }
-
- /**
- * To test value and watermark serialization and deserialization with time characteristic:
- * IngestionTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSerDeIngestionTime() throws Exception {
- testRecordSerDe(IngestionTime);
- }
-
- /**
- * To test value and watermark serialization and deserialization with time characteristic:
- * EventTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testSerDeEventTime() throws Exception {
- testRecordSerDe(EventTime);
- }
-
- /**
- * To test value and watermark serialization and deserialization with time characteristic:
- * EventTime.
- *
- * <p>Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1.
- */
- @Test
- public void testWatermarkBroadcasting() throws Exception {
- final int numberOfPartitions = 3;
- final int producerParallelism = 2;
- final int numElementsPerProducer = 1000;
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- Map<Integer, Collection<ConsumerRecord<byte[], byte[]>>> results =
- testKafkaShuffleProducer(
- topic("test_watermark_broadcast-" + UUID.randomUUID(), EventTime),
- env,
- numberOfPartitions,
- producerParallelism,
- numElementsPerProducer,
- EventTime);
- TypeSerializer<Tuple3<Integer, Long, Integer>> typeSerializer = createTypeSerializer(env);
- KafkaShuffleElementDeserializer deserializer =
- new KafkaShuffleElementDeserializer<>(typeSerializer);
-
- // Records in a single partition are kept in order
- for (int p = 0; p < numberOfPartitions; p++) {
- Collection<ConsumerRecord<byte[], byte[]>> records = results.get(p);
- Map<Integer, List<KafkaShuffleWatermark>> watermarks = new HashMap<>();
-
- for (ConsumerRecord<byte[], byte[]> consumerRecord : records) {
- assertThat(consumerRecord.key()).isNull();
- KafkaShuffleElement element = deserializer.deserialize(consumerRecord);
- if (element.isRecord()) {
- KafkaShuffleRecord<Tuple3<Integer, Long, Integer>> record = element.asRecord();
- assertThat(INIT_TIMESTAMP + record.getValue().f0)
- .isEqualTo(record.getValue().f1.longValue());
- assertThat(record.getValue().f1.longValue())
- .isEqualTo(record.getTimestamp().longValue());
- } else if (element.isWatermark()) {
- KafkaShuffleWatermark watermark = element.asWatermark();
- watermarks.computeIfAbsent(watermark.getSubtask(), k -> new ArrayList<>());
- watermarks.get(watermark.getSubtask()).add(watermark);
- } else {
- fail("KafkaShuffleElement is either record or watermark");
- }
- }
-
- // According to the setting how watermarks are generated in this ITTest,
- // every producer task emits a watermark corresponding to each record + the
- // end-of-event-time watermark.
- // Hence each producer sub task generates `numElementsPerProducer + 1` watermarks.
- // Each producer sub task broadcasts these `numElementsPerProducer + 1` watermarks to
- // all partitions.
- // Thus in total, each producer sub task emits `(numElementsPerProducer + 1) *
- // numberOfPartitions` watermarks.
- // From the consumer side, each partition receives `(numElementsPerProducer + 1) *
- // producerParallelism` watermarks,
- // with each producer sub task produces `numElementsPerProducer + 1` watermarks.
- // Besides, watermarks from the same producer sub task should keep in order.
- for (List<KafkaShuffleWatermark> subTaskWatermarks : watermarks.values()) {
- int index = 0;
- assertThat(subTaskWatermarks).hasSize(numElementsPerProducer + 1);
- for (KafkaShuffleWatermark watermark : subTaskWatermarks) {
- if (index == numElementsPerProducer) {
- // the last element is the watermark that signifies end-of-event-time
- assertThat(Watermark.MAX_WATERMARK.getTimestamp())
- .isEqualTo(watermark.getWatermark());
- } else {
- assertThat(INIT_TIMESTAMP + index++).isEqualTo(watermark.getWatermark());
- }
- }
- }
- }
- }
-
- /**
- * To test no data is lost or duplicated end-2-end.
- *
- * <p>Schema: (key, timestamp, source instance Id). Producer Parallelism = 1; Kafka Partition #
- * = 1; Consumer Parallelism = 1
- */
- private void testKafkaShuffle(int numElementsPerProducer, TimeCharacteristic timeCharacteristic)
- throws Exception {
- String topic = topic("test_simple-" + UUID.randomUUID(), timeCharacteristic);
- final int numberOfPartitions = 1;
- final int producerParallelism = 1;
-
- createTestTopic(topic, numberOfPartitions, 1);
-
- final StreamExecutionEnvironment env =
- createEnvironment(producerParallelism, timeCharacteristic);
- createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions)
- .map(
- new ElementCountNoMoreThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1)
- .map(
- new ElementCountNoLessThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1);
-
- tryExecute(env, topic);
-
- deleteTestTopic(topic);
- }
-
- /**
- * To test data is partitioned to the right partition.
- *
- * <p>Schema: (key, timestamp, source instance Id). Producer Parallelism = 2; Kafka Partition #
- * = 3; Consumer Parallelism = 3
- */
- private void testAssignedToPartition(
- int numElementsPerProducer, TimeCharacteristic timeCharacteristic) throws Exception {
- String topic = topic("test_assigned_to_partition-" + UUID.randomUUID(), timeCharacteristic);
- final int numberOfPartitions = 3;
- final int producerParallelism = 2;
-
- createTestTopic(topic, numberOfPartitions, 1);
-
- final StreamExecutionEnvironment env =
- createEnvironment(producerParallelism, timeCharacteristic);
-
- KeyedStream<Tuple3<Integer, Long, Integer>, Tuple> keyedStream =
- createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions);
- keyedStream
- .process(
- new PartitionValidator(
- keyedStream.getKeySelector(), numberOfPartitions, topic))
- .setParallelism(numberOfPartitions)
- .map(
- new ElementCountNoMoreThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1)
- .map(
- new ElementCountNoLessThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1);
-
- tryExecute(env, topic);
-
- deleteTestTopic(topic);
- }
-
- /**
- * To watermark from the consumer side always increase.
- *
- * <p>Schema: (key, timestamp, source instance Id). Producer Parallelism = 2; Kafka Partition #
- * = 3; Consumer Parallelism = 3
- */
- private void testWatermarkIncremental(int numElementsPerProducer) throws Exception {
- TimeCharacteristic timeCharacteristic = EventTime;
- String topic = topic("test_watermark_incremental-" + UUID.randomUUID(), timeCharacteristic);
- final int numberOfPartitions = 3;
- final int producerParallelism = 2;
-
- createTestTopic(topic, numberOfPartitions, 1);
-
- final StreamExecutionEnvironment env =
- createEnvironment(producerParallelism, timeCharacteristic);
-
- KeyedStream<Tuple3<Integer, Long, Integer>, Tuple> keyedStream =
- createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions,
- true);
- keyedStream
- .process(new WatermarkValidator())
- .setParallelism(numberOfPartitions)
- .map(
- new ElementCountNoMoreThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1)
- .map(
- new ElementCountNoLessThanValidator(
- numElementsPerProducer * producerParallelism))
- .setParallelism(1);
-
- tryExecute(env, topic);
-
- deleteTestTopic(topic);
- }
-
- private void testRecordSerDe(TimeCharacteristic timeCharacteristic) throws Exception {
- final int numElementsPerProducer = 2000;
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- // Records in a single partition are kept in order
- Collection<ConsumerRecord<byte[], byte[]>> records =
- Iterables.getOnlyElement(
- testKafkaShuffleProducer(
- topic(
- "test_serde-" + UUID.randomUUID(),
- timeCharacteristic),
- env,
- 1,
- 1,
- numElementsPerProducer,
- timeCharacteristic)
- .values());
-
- switch (timeCharacteristic) {
- case ProcessingTime:
- // NonTimestampContext, no intermediate watermarks, and one end-of-event-time
- // watermark
- assertThat(records.size()).isEqualTo(numElementsPerProducer + 1);
- break;
- case IngestionTime:
- // IngestionTime uses AutomaticWatermarkContext and it emits a watermark after every
- // `watermarkInterval`
- // with default interval 200, hence difficult to control the number of watermarks
- break;
- case EventTime:
- // ManualWatermarkContext
- // `numElementsPerProducer` records, `numElementsPerProducer` watermarks, and one
- // end-of-event-time watermark
- assertThat(records.size()).isEqualTo(numElementsPerProducer * 2 + 1);
- break;
- default:
- fail("unknown TimeCharacteristic type");
- }
-
- TypeSerializer<Tuple3<Integer, Long, Integer>> typeSerializer = createTypeSerializer(env);
-
- KafkaShuffleElementDeserializer deserializer =
- new KafkaShuffleElementDeserializer<>(typeSerializer);
-
- int recordIndex = 0;
- int watermarkIndex = 0;
- for (ConsumerRecord<byte[], byte[]> consumerRecord : records) {
- assertThat(consumerRecord.key()).isNull();
- KafkaShuffleElement element = deserializer.deserialize(consumerRecord);
- if (element.isRecord()) {
- KafkaShuffleRecord<Tuple3<Integer, Long, Integer>> record = element.asRecord();
- switch (timeCharacteristic) {
- case ProcessingTime:
- assertThat(record.getTimestamp()).isNull();
- break;
- case IngestionTime:
- assertThat(record.getTimestamp()).isNotNull();
- break;
- case EventTime:
- assertThat(record.getValue().f1.longValue())
- .isEqualTo(record.getTimestamp().longValue());
- break;
- default:
- fail("unknown TimeCharacteristic type");
- }
- assertThat(recordIndex).isEqualTo(record.getValue().f0.intValue());
- assertThat(INIT_TIMESTAMP + recordIndex)
- .isEqualTo(record.getValue().f1.longValue());
- assertThat(0).isEqualTo(record.getValue().f2.intValue());
- recordIndex++;
- } else if (element.isWatermark()) {
- KafkaShuffleWatermark watermark = element.asWatermark();
- switch (timeCharacteristic) {
- case ProcessingTime:
- assertThat(watermark.getSubtask()).isEqualTo(0);
- // the last element is the watermark that signifies end-of-event-time
- assertThat(recordIndex).isEqualTo(numElementsPerProducer);
- assertThat(Watermark.MAX_WATERMARK.getTimestamp())
- .isEqualTo(watermark.getWatermark());
- break;
- case IngestionTime:
- break;
- case EventTime:
- assertThat(watermark.getSubtask()).isEqualTo(0);
- if (watermarkIndex == recordIndex) {
- // the last element is the watermark that signifies end-of-event-time
- assertThat(Watermark.MAX_WATERMARK.getTimestamp())
- .isEqualTo(watermark.getWatermark());
- } else {
- assertThat(INIT_TIMESTAMP + watermarkIndex)
- .isEqualTo(watermark.getWatermark());
- }
- break;
- default:
- fail("unknown TimeCharacteristic type");
- }
- watermarkIndex++;
- } else {
- fail("KafkaShuffleElement is either record or watermark");
- }
- }
- }
-
- private Map<Integer, Collection<ConsumerRecord<byte[], byte[]>>> testKafkaShuffleProducer(
- String topic,
- StreamExecutionEnvironment env,
- int numberOfPartitions,
- int producerParallelism,
- int numElementsPerProducer,
- TimeCharacteristic timeCharacteristic)
- throws Exception {
- createTestTopic(topic, numberOfPartitions, 1);
-
- env.setParallelism(producerParallelism);
- env.setRestartStrategy(RestartStrategies.noRestart());
- env.setStreamTimeCharacteristic(timeCharacteristic);
-
- DataStream<Tuple3<Integer, Long, Integer>> source =
- env.addSource(new KafkaSourceFunction(numElementsPerProducer, false))
- .setParallelism(producerParallelism);
- DataStream<Tuple3<Integer, Long, Integer>> input =
- (timeCharacteristic == EventTime)
- ? source.assignTimestampsAndWatermarks(new PunctuatedExtractor())
- .setParallelism(producerParallelism)
- : source;
-
- Properties properties = kafkaServer.getStandardProperties();
- Properties kafkaProperties = PropertiesUtil.flatten(properties);
-
- kafkaProperties.setProperty(PRODUCER_PARALLELISM, String.valueOf(producerParallelism));
- kafkaProperties.setProperty(PARTITION_NUMBER, String.valueOf(numberOfPartitions));
- kafkaProperties.setProperty(
- "key.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer");
- kafkaProperties.setProperty(
- "value.deserializer",
- "org.apache.kafka.common.serialization.ByteArrayDeserializer");
- FlinkKafkaShuffle.writeKeyBy(input, topic, kafkaProperties, 0);
-
- env.execute("Write to " + topic);
-
- Map<Integer, Collection<ConsumerRecord<byte[], byte[]>>> results = new HashMap<>();
-
- kafkaServer
- .<byte[], byte[]>getAllRecordsFromTopic(kafkaProperties, topic)
- .forEach(
- r -> {
- final int partition = r.partition();
- if (!results.containsKey(partition)) {
- results.put(partition, Lists.newArrayList());
- }
- results.get(partition).add(r);
- });
-
- deleteTestTopic(topic);
-
- return results;
- }
-
- private StreamExecutionEnvironment createEnvironment(
- int producerParallelism, TimeCharacteristic timeCharacteristic) {
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(producerParallelism);
- env.setStreamTimeCharacteristic(timeCharacteristic);
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- return env;
- }
-
- private TypeSerializer<Tuple3<Integer, Long, Integer>> createTypeSerializer(
- StreamExecutionEnvironment env) {
- return new TupleTypeInfo<Tuple3<Integer, Long, Integer>>(
- BasicTypeInfo.INT_TYPE_INFO,
- BasicTypeInfo.LONG_TYPE_INFO,
- BasicTypeInfo.INT_TYPE_INFO)
- .createSerializer(env.getConfig());
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java
deleted file mode 100644
index 064aebd..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * 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.shuffle;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.connectors.kafka.KafkaConsumerTestBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaProducerTestBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionAssigner;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.util.Collector;
-
-import org.junit.BeforeClass;
-
-import java.util.Random;
-
-import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime;
-
-/** Base Test Class for KafkaShuffle. */
-public class KafkaShuffleTestBase extends KafkaConsumerTestBase {
- static final long INIT_TIMESTAMP = System.currentTimeMillis();
-
- @BeforeClass
- public static void prepare() throws Exception {
- KafkaProducerTestBase.prepare();
- ((KafkaTestEnvironmentImpl) kafkaServer)
- .setProducerSemantic(FlinkKafkaProducer.Semantic.EXACTLY_ONCE);
- }
-
- static class KafkaSourceFunction
- extends RichParallelSourceFunction<Tuple3<Integer, Long, Integer>> {
- private volatile boolean running = true;
- private final int numElementsPerProducer;
- private final boolean unBounded;
-
- KafkaSourceFunction(int numElementsPerProducer) {
- this.numElementsPerProducer = numElementsPerProducer;
- this.unBounded = true;
- }
-
- KafkaSourceFunction(int numElementsPerProducer, boolean unBounded) {
- this.numElementsPerProducer = numElementsPerProducer;
- this.unBounded = unBounded;
- }
-
- @Override
- public void run(SourceContext<Tuple3<Integer, Long, Integer>> ctx) throws Exception {
- long timestamp = INIT_TIMESTAMP;
- int sourceInstanceId = getRuntimeContext().getIndexOfThisSubtask();
- for (int i = 0; i < numElementsPerProducer && running; i++) {
- ctx.collect(new Tuple3<>(i, timestamp++, sourceInstanceId));
- }
-
- while (running && unBounded) {
- Thread.sleep(100);
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- }
-
- static KeyedStream<Tuple3<Integer, Long, Integer>, Tuple> createKafkaShuffle(
- StreamExecutionEnvironment env,
- String topic,
- int numElementsPerProducer,
- int producerParallelism,
- TimeCharacteristic timeCharacteristic,
- int numberOfPartitions) {
- return createKafkaShuffle(
- env,
- topic,
- numElementsPerProducer,
- producerParallelism,
- timeCharacteristic,
- numberOfPartitions,
- false);
- }
-
- static KeyedStream<Tuple3<Integer, Long, Integer>, Tuple> createKafkaShuffle(
- StreamExecutionEnvironment env,
- String topic,
- int numElementsPerProducer,
- int producerParallelism,
- TimeCharacteristic timeCharacteristic,
- int numberOfPartitions,
- boolean randomness) {
- DataStream<Tuple3<Integer, Long, Integer>> source =
- env.addSource(new KafkaSourceFunction(numElementsPerProducer))
- .setParallelism(producerParallelism);
- DataStream<Tuple3<Integer, Long, Integer>> input =
- (timeCharacteristic == EventTime)
- ? source.assignTimestampsAndWatermarks(new PunctuatedExtractor(randomness))
- .setParallelism(producerParallelism)
- : source;
-
- return FlinkKafkaShuffle.persistentKeyBy(
- input,
- topic,
- producerParallelism,
- numberOfPartitions,
- kafkaServer.getStandardProperties(),
- 0);
- }
-
- static class PunctuatedExtractor
- implements AssignerWithPunctuatedWatermarks<Tuple3<Integer, Long, Integer>> {
- private static final long serialVersionUID = 1L;
- boolean randomness;
- Random rnd = new Random(123);
-
- PunctuatedExtractor() {
- randomness = false;
- }
-
- PunctuatedExtractor(boolean randomness) {
- this.randomness = randomness;
- }
-
- @Override
- public long extractTimestamp(
- Tuple3<Integer, Long, Integer> element, long previousTimestamp) {
- return element.f1;
- }
-
- @Override
- public Watermark checkAndGetNextWatermark(
- Tuple3<Integer, Long, Integer> lastElement, long extractedTimestamp) {
- long randomValue = randomness ? rnd.nextInt(10) : 0;
- return new Watermark(extractedTimestamp + randomValue);
- }
- }
-
- static class PartitionValidator
- extends KeyedProcessFunction<
- Tuple, Tuple3<Integer, Long, Integer>, Tuple3<Integer, Long, Integer>> {
- private final KeySelector<Tuple3<Integer, Long, Integer>, Tuple> keySelector;
- private final int numberOfPartitions;
- private final String topic;
- private KeyGroupRange keyGroupRange;
-
- private int previousPartition;
-
- PartitionValidator(
- KeySelector<Tuple3<Integer, Long, Integer>, Tuple> keySelector,
- int numberOfPartitions,
- String topic) {
- this.keySelector = keySelector;
- this.numberOfPartitions = numberOfPartitions;
- this.topic = topic;
- this.previousPartition = -1;
- }
-
- @Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- this.keyGroupRange =
- KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex(
- getRuntimeContext().getMaxNumberOfParallelSubtasks(),
- numberOfPartitions,
- getRuntimeContext().getIndexOfThisSubtask());
- }
-
- @Override
- public void processElement(
- Tuple3<Integer, Long, Integer> in,
- Context ctx,
- Collector<Tuple3<Integer, Long, Integer>> out)
- throws Exception {
- int expectedSubtask =
- KeyGroupRangeAssignment.assignKeyToParallelOperator(
- keySelector.getKey(in), numberOfPartitions, numberOfPartitions);
- int expectedPartition = -1;
- // This is how Kafka assign partition to subTask;
- for (int i = 0; i < numberOfPartitions; i++) {
- if (KafkaTopicPartitionAssigner.assign(topic, i, numberOfPartitions)
- == expectedSubtask) {
- expectedPartition = i;
- }
- }
- int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
-
- boolean rightAssignment =
- (expectedSubtask == indexOfThisSubtask)
- && keyGroupRange.contains(
- KeyGroupRangeAssignment.assignToKeyGroup(
- keySelector.getKey(in),
- getRuntimeContext().getMaxNumberOfParallelSubtasks()));
- boolean samePartition =
- (expectedPartition != -1)
- && ((previousPartition == expectedPartition)
- || (previousPartition == -1));
- previousPartition = expectedPartition;
-
- if (!(rightAssignment && samePartition)) {
- throw new Exception("Error: Kafka partition assignment error ");
- }
- out.collect(in);
- }
- }
-
- static class WatermarkValidator
- extends KeyedProcessFunction<
- Tuple, Tuple3<Integer, Long, Integer>, Tuple3<Integer, Long, Integer>> {
- private long previousWatermark = Long.MIN_VALUE; // initial watermark get from timeService
-
- @Override
- public void processElement(
- Tuple3<Integer, Long, Integer> in,
- Context ctx,
- Collector<Tuple3<Integer, Long, Integer>> out)
- throws Exception {
-
- long watermark = ctx.timerService().currentWatermark();
-
- // Notice that the timerService might not be updated if no new watermark has been
- // emitted, hence equivalent
- // watermark is allowed, strictly incremental check is done when fetching watermark from
- // KafkaShuffleFetcher.
- if (watermark < previousWatermark) {
- throw new Exception(
- "Error: watermark should always increase. current watermark : previous watermark ["
- + watermark
- + " : "
- + previousWatermark
- + "]");
- }
- previousWatermark = watermark;
-
- out.collect(in);
- }
- }
-
- static class ElementCountNoLessThanValidator
- implements MapFunction<Tuple3<Integer, Long, Integer>, Tuple3<Integer, Long, Integer>> {
- private final int totalCount;
- private int counter = 0;
-
- ElementCountNoLessThanValidator(int totalCount) {
- this.totalCount = totalCount;
- }
-
- @Override
- public Tuple3<Integer, Long, Integer> map(Tuple3<Integer, Long, Integer> element)
- throws Exception {
- counter++;
-
- if (counter == totalCount) {
- throw new SuccessException();
- }
-
- return element;
- }
- }
-
- static class ElementCountNoMoreThanValidator
- implements MapFunction<Tuple3<Integer, Long, Integer>, Tuple3<Integer, Long, Integer>> {
- private final int totalCount;
- private int counter = 0;
-
- ElementCountNoMoreThanValidator(int totalCount) {
- this.totalCount = totalCount;
- }
-
- @Override
- public Tuple3<Integer, Long, Integer> map(Tuple3<Integer, Long, Integer> element)
- throws Exception {
- counter++;
-
- if (counter > totalCount) {
- throw new Exception("Error: number of elements more than expected");
- }
-
- return element;
- }
- }
-
- String topic(String prefix, TimeCharacteristic timeCharacteristic) {
- return prefix + "_" + timeCharacteristic;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
deleted file mode 100644
index e8bc9e3..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.table.api.TableConfig;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.config.ExecutionConfigOptions;
-import org.apache.flink.table.api.config.OptimizerConfigOptions;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.time.Duration;
-import java.time.ZoneId;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.waitingExpectedResults;
-
-/** IT cases for Kafka with changelog format for Table API & SQL. */
-public class KafkaChangelogTableITCase extends KafkaTableTestBase {
-
- @Before
- public void before() {
- // we have to use single parallelism,
- // because we will count the messages in sink to terminate the job
- env.setParallelism(1);
- }
-
- @Test
- public void testKafkaDebeziumChangelogSource() throws Exception {
- final String topic = "changelog_topic";
- createTestTopic(topic, 1, 1);
-
- // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
- TableConfig tableConf = tEnv.getConfig();
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
- tableConf.set(
- ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
- tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
-
- // ---------- Write the Debezium json into Kafka -------------------
- List<String> lines = readLines("debezium-data-schema-exclude.txt");
- try {
- writeRecordsToKafka(topic, lines);
- } catch (Exception e) {
- throw new Exception("Failed to write debezium data to Kafka.", e);
- }
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
- String sourceDDL =
- String.format(
- "CREATE TABLE debezium_source ("
- // test format metadata
- + " origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL," // unused
- + " origin_table STRING METADATA FROM 'value.source.table' VIRTUAL,"
- + " id INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(10,3),"
- // test connector metadata
- + " origin_topic STRING METADATA FROM 'topic' VIRTUAL,"
- + " origin_partition STRING METADATA FROM 'partition' VIRTUAL" // unused
- + ") WITH ("
- + " 'connector' = 'kafka',"
- + " 'topic' = '%s',"
- + " 'properties.bootstrap.servers' = '%s',"
- + " 'scan.startup.mode' = 'earliest-offset',"
- + " 'value.format' = 'debezium-json'"
- + ")",
- topic, bootstraps);
- String sinkDDL =
- "CREATE TABLE sink ("
- + " origin_topic STRING,"
- + " origin_table STRING,"
- + " name STRING,"
- + " weightSum DECIMAL(10,3),"
- + " PRIMARY KEY (name) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false'"
- + ")";
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO sink "
- + "SELECT FIRST_VALUE(origin_topic), FIRST_VALUE(origin_table), name, SUM(weight) "
- + "FROM debezium_source GROUP BY name");
- /*
- * Debezium captures change data on the `products` table:
- *
- * <pre>
- * CREATE TABLE products (
- * id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
- * name VARCHAR(255),
- * description VARCHAR(512),
- * weight FLOAT
- * );
- * ALTER TABLE products AUTO_INCREMENT = 101;
- *
- * INSERT INTO products
- * VALUES (default,"scooter","Small 2-wheel scooter",3.14),
- * (default,"car battery","12V car battery",8.1),
- * (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8),
- * (default,"hammer","12oz carpenter's hammer",0.75),
- * (default,"hammer","14oz carpenter's hammer",0.875),
- * (default,"hammer","16oz carpenter's hammer",1.0),
- * (default,"rocks","box of assorted rocks",5.3),
- * (default,"jacket","water resistent black wind breaker",0.1),
- * (default,"spare tire","24 inch spare tire",22.2);
- * UPDATE products SET description='18oz carpenter hammer' WHERE id=106;
- * UPDATE products SET weight='5.1' WHERE id=107;
- * INSERT INTO products VALUES (default,"jacket","water resistent white wind breaker",0.2);
- * INSERT INTO products VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
- * UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;
- * UPDATE products SET weight='5.17' WHERE id=111;
- * DELETE FROM products WHERE id=111;
- *
- * > SELECT * FROM products;
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | id | name | description | weight |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | 101 | scooter | Small 2-wheel scooter | 3.14 |
- * | 102 | car battery | 12V car battery | 8.1 |
- * | 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 |
- * | 104 | hammer | 12oz carpenter's hammer | 0.75 |
- * | 105 | hammer | 14oz carpenter's hammer | 0.875 |
- * | 106 | hammer | 18oz carpenter hammer | 1 |
- * | 107 | rocks | box of assorted rocks | 5.1 |
- * | 108 | jacket | water resistent black wind breaker | 0.1 |
- * | 109 | spare tire | 24 inch spare tire | 22.2 |
- * | 110 | jacket | new water resistent white wind breaker | 0.5 |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * </pre>
- */
- List<String> expected =
- Arrays.asList(
- "+I[changelog_topic, products, scooter, 3.140]",
- "+I[changelog_topic, products, car battery, 8.100]",
- "+I[changelog_topic, products, 12-pack drill bits, 0.800]",
- "+I[changelog_topic, products, hammer, 2.625]",
- "+I[changelog_topic, products, rocks, 5.100]",
- "+I[changelog_topic, products, jacket, 0.600]",
- "+I[changelog_topic, products, spare tire, 22.200]");
-
- waitingExpectedResults("sink", expected, Duration.ofSeconds(10));
-
- // ------------- cleanup -------------------
-
- tableResult.getJobClient().get().cancel().get(); // stop the job
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaCanalChangelogSource() throws Exception {
- final String topic = "changelog_canal";
- createTestTopic(topic, 1, 1);
-
- // configure time zone of the Canal Json metadata "ingestion-timestamp"
- tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC"));
- // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
- TableConfig tableConf = tEnv.getConfig();
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
- tableConf.set(
- ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
- tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
-
- // ---------- Write the Canal json into Kafka -------------------
- List<String> lines = readLines("canal-data.txt");
- try {
- writeRecordsToKafka(topic, lines);
- } catch (Exception e) {
- throw new Exception("Failed to write canal data to Kafka.", e);
- }
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
- String sourceDDL =
- String.format(
- "CREATE TABLE canal_source ("
- // test format metadata
- + " origin_database STRING METADATA FROM 'value.database' VIRTUAL,"
- + " origin_table STRING METADATA FROM 'value.table' VIRTUAL,"
- + " origin_sql_type MAP<STRING, INT> METADATA FROM 'value.sql-type' VIRTUAL,"
- + " origin_pk_names ARRAY<STRING> METADATA FROM 'value.pk-names' VIRTUAL,"
- + " origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL,"
- + " origin_es TIMESTAMP(3) METADATA FROM 'value.event-timestamp' VIRTUAL,"
- + " id INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(10,3),"
- // test connector metadata
- + " origin_topic STRING METADATA FROM 'topic' VIRTUAL,"
- + " origin_partition STRING METADATA FROM 'partition' VIRTUAL," // unused
- + " WATERMARK FOR origin_es AS origin_es - INTERVAL '5' SECOND"
- + ") WITH ("
- + " 'connector' = 'kafka',"
- + " 'topic' = '%s',"
- + " 'properties.bootstrap.servers' = '%s',"
- + " 'scan.startup.mode' = 'earliest-offset',"
- + " 'value.format' = 'canal-json'"
- + ")",
- topic, bootstraps);
- String sinkDDL =
- "CREATE TABLE sink ("
- + " origin_topic STRING,"
- + " origin_database STRING,"
- + " origin_table STRING,"
- + " origin_sql_type MAP<STRING, INT>,"
- + " origin_pk_names ARRAY<STRING>,"
- + " origin_ts TIMESTAMP(3),"
- + " origin_es TIMESTAMP(3),"
- + " name STRING,"
- + " PRIMARY KEY (name) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false'"
- + ")";
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO sink "
- + "SELECT origin_topic, origin_database, origin_table, origin_sql_type, "
- + "origin_pk_names, origin_ts, origin_es, name "
- + "FROM canal_source");
-
- /*
- * Canal captures change data on the `products` table:
- *
- * <pre>
- * CREATE TABLE products (
- * id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
- * name VARCHAR(255),
- * description VARCHAR(512),
- * weight FLOAT
- * );
- * ALTER TABLE products AUTO_INCREMENT = 101;
- *
- * INSERT INTO products
- * VALUES (default,"scooter","Small 2-wheel scooter",3.14),
- * (default,"car battery","12V car battery",8.1),
- * (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8),
- * (default,"hammer","12oz carpenter's hammer",0.75),
- * (default,"hammer","14oz carpenter's hammer",0.875),
- * (default,"hammer","16oz carpenter's hammer",1.0),
- * (default,"rocks","box of assorted rocks",5.3),
- * (default,"jacket","water resistent black wind breaker",0.1),
- * (default,"spare tire","24 inch spare tire",22.2);
- * UPDATE products SET description='18oz carpenter hammer' WHERE id=106;
- * UPDATE products SET weight='5.1' WHERE id=107;
- * INSERT INTO products VALUES (default,"jacket","water resistent white wind breaker",0.2);
- * INSERT INTO products VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
- * UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;
- * UPDATE products SET weight='5.17' WHERE id=111;
- * DELETE FROM products WHERE id=111;
- *
- * > SELECT * FROM products;
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | id | name | description | weight |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | 101 | scooter | Small 2-wheel scooter | 3.14 |
- * | 102 | car battery | 12V car battery | 8.1 |
- * | 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 |
- * | 104 | hammer | 12oz carpenter's hammer | 0.75 |
- * | 105 | hammer | 14oz carpenter's hammer | 0.875 |
- * | 106 | hammer | 18oz carpenter hammer | 1 |
- * | 107 | rocks | box of assorted rocks | 5.1 |
- * | 108 | jacket | water resistent black wind breaker | 0.1 |
- * | 109 | spare tire | 24 inch spare tire | 22.2 |
- * | 110 | jacket | new water resistent white wind breaker | 0.5 |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * </pre>
- */
-
- List<String> expected =
- Arrays.asList(
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:38:35.477, 2020-05-13T12:38:35, 12-pack drill bits]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:38:35.477, 2020-05-13T12:38:35, spare tire]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:39:06.301, 2020-05-13T12:39:06, hammer]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:39:09.489, 2020-05-13T12:39:09, rocks]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:39:18.230, 2020-05-13T12:39:18, jacket]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:42:33.939, 2020-05-13T12:42:33, car battery]",
- "+I[changelog_canal, inventory, products2, {name=12, weight=7, description=12, id=4}, [id], 2020-05-13T12:42:33.939, 2020-05-13T12:42:33, scooter]");
-
- waitingExpectedResults("sink", expected, Duration.ofSeconds(10));
-
- // ------------- cleanup -------------------
-
- tableResult.getJobClient().get().cancel().get(); // stop the job
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaMaxwellChangelogSource() throws Exception {
- final String topic = "changelog_maxwell";
- createTestTopic(topic, 1, 1);
-
- // configure time zone of the Maxwell Json metadata "ingestion-timestamp"
- tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC"));
- // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
- TableConfig tableConf = tEnv.getConfig();
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
- tableConf.set(
- ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
- tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
- tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
-
- // ---------- Write the Maxwell json into Kafka -------------------
- List<String> lines = readLines("maxwell-data.txt");
- try {
- writeRecordsToKafka(topic, lines);
- } catch (Exception e) {
- throw new Exception("Failed to write maxwell data to Kafka.", e);
- }
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
- String sourceDDL =
- String.format(
- "CREATE TABLE maxwell_source ("
- // test format metadata
- + " origin_database STRING METADATA FROM 'value.database' VIRTUAL,"
- + " origin_table STRING METADATA FROM 'value.table' VIRTUAL,"
- + " origin_primary_key_columns ARRAY<STRING> METADATA FROM 'value.primary-key-columns' VIRTUAL,"
- + " origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL,"
- + " id INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(10,3),"
- // test connector metadata
- + " origin_topic STRING METADATA FROM 'topic' VIRTUAL,"
- + " origin_partition STRING METADATA FROM 'partition' VIRTUAL" // unused
- + ") WITH ("
- + " 'connector' = 'kafka',"
- + " 'topic' = '%s',"
- + " 'properties.bootstrap.servers' = '%s',"
- + " 'scan.startup.mode' = 'earliest-offset',"
- + " 'value.format' = 'maxwell-json'"
- + ")",
- topic, bootstraps);
- String sinkDDL =
- "CREATE TABLE sink ("
- + " origin_topic STRING,"
- + " origin_database STRING,"
- + " origin_table STRING,"
- + " origin_primary_key_columns ARRAY<STRING>,"
- + " origin_ts TIMESTAMP(3),"
- + " name STRING,"
- + " PRIMARY KEY (name) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false'"
- + ")";
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO sink "
- + "SELECT origin_topic, origin_database, origin_table, origin_primary_key_columns, "
- + "origin_ts, name "
- + "FROM maxwell_source");
-
- /*
- * Maxwell captures change data on the `products` table:
- *
- * <pre>
- * CREATE TABLE products (
- * id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
- * name VARCHAR(255),
- * description VARCHAR(512),
- * weight FLOAT
- * );
- * ALTER TABLE products AUTO_INCREMENT = 101;
- *
- * INSERT INTO products
- * VALUES (default,"scooter","Small 2-wheel scooter",3.14),
- * (default,"car battery","12V car battery",8.1),
- * (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8),
- * (default,"hammer","12oz carpenter's hammer",0.75),
- * (default,"hammer","14oz carpenter's hammer",0.875),
- * (default,"hammer","16oz carpenter's hammer",1.0),
- * (default,"rocks","box of assorted rocks",5.3),
- * (default,"jacket","water resistent black wind breaker",0.1),
- * (default,"spare tire","24 inch spare tire",22.2);
- * UPDATE products SET description='18oz carpenter hammer' WHERE id=106;
- * UPDATE products SET weight='5.1' WHERE id=107;
- * INSERT INTO products VALUES (default,"jacket","water resistent white wind breaker",0.2);
- * INSERT INTO products VALUES (default,"scooter","Big 2-wheel scooter ",5.18);
- * UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;
- * UPDATE products SET weight='5.17' WHERE id=111;
- * DELETE FROM products WHERE id=111;
- *
- * > SELECT * FROM products;
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | id | name | description | weight |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | 101 | scooter | Small 2-wheel scooter | 3.14 |
- * | 102 | car battery | 12V car battery | 8.1 |
- * | 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 |
- * | 104 | hammer | 12oz carpenter's hammer | 0.75 |
- * | 105 | hammer | 14oz carpenter's hammer | 0.875 |
- * | 106 | hammer | 18oz carpenter hammer | 1 |
- * | 107 | rocks | box of assorted rocks | 5.1 |
- * | 108 | jacket | water resistent black wind breaker | 0.1 |
- * | 109 | spare tire | 24 inch spare tire | 22.2 |
- * | 110 | jacket | new water resistent white wind breaker | 0.5 |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * </pre>
- */
-
- List<String> expected =
- Arrays.asList(
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:34:43, 12-pack drill bits]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:34:43, spare tire]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:34:53, hammer]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:34:57, rocks]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:35:06, jacket]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:35:28, car battery]",
- "+I[changelog_maxwell, test, product, null, 2020-08-06T03:35:28, scooter]");
-
- waitingExpectedResults("sink", expected, Duration.ofSeconds(10));
-
- // ------------- cleanup -------------------
-
- tableResult.getJobClient().get().cancel().get(); // stop the job
- deleteTestTopic(topic);
- }
-
- private void writeRecordsToKafka(String topic, List<String> lines) throws Exception {
- DataStreamSource<String> stream = env.fromCollection(lines);
- SerializationSchema<String> serSchema = new SimpleStringSchema();
- FlinkKafkaPartitioner<String> partitioner = new FlinkFixedPartitioner<>();
-
- // the producer must not produce duplicates
- Properties producerProperties = getStandardProps();
- producerProperties.setProperty("retries", "0");
- stream.sinkTo(
- KafkaSink.<String>builder()
- .setBootstrapServers(
- producerProperties.getProperty(
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(topic)
- .setValueSerializationSchema(serSchema)
- .setPartitioner(partitioner)
- .build())
- .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)
- .build());
- env.execute("Write sequence");
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java
deleted file mode 100644
index e995908..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.types.DataType;
-
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection;
-import static org.apache.flink.table.api.DataTypes.FIELD;
-import static org.apache.flink.table.api.DataTypes.INT;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Test for {@link KafkaConnectorOptionsUtil}. */
-public class KafkaConnectorOptionsUtilTest {
-
- @Test
- public void testFormatProjection() {
- final DataType dataType =
- DataTypes.ROW(
- FIELD("id", INT()),
- FIELD("name", STRING()),
- FIELD("age", INT()),
- FIELD("address", STRING()));
-
- final Map<String, String> options = createTestOptions();
- options.put("key.fields", "address; name");
- options.put("value.fields-include", "EXCEPT_KEY");
-
- final Configuration config = Configuration.fromMap(options);
-
- assertThat(createKeyFormatProjection(config, dataType)).isEqualTo(new int[] {3, 1});
- assertThat(createValueFormatProjection(config, dataType)).isEqualTo(new int[] {0, 2});
- }
-
- @Test
- public void testMissingKeyFormatProjection() {
- final DataType dataType = ROW(FIELD("id", INT()));
- final Map<String, String> options = createTestOptions();
-
- final Configuration config = Configuration.fromMap(options);
-
- assertThatThrownBy(() -> createKeyFormatProjection(config, dataType))
- .isInstanceOf(ValidationException.class)
- .hasMessage(
- "A key format 'key.format' requires the declaration of one or more "
- + "of key fields using 'key.fields'.");
- }
-
- @Test
- public void testInvalidKeyFormatFieldProjection() {
- final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING()));
- final Map<String, String> options = createTestOptions();
- options.put("key.fields", "non_existing");
-
- final Configuration config = Configuration.fromMap(options);
-
- assertThatThrownBy(() -> createKeyFormatProjection(config, dataType))
- .isInstanceOf(ValidationException.class)
- .hasMessage(
- "Could not find the field 'non_existing' in the table schema for "
- + "usage in the key format. A key field must be a regular, "
- + "physical column. The following columns can be selected "
- + "in the 'key.fields' option:\n"
- + "[id, name]");
- }
-
- @Test
- public void testInvalidKeyFormatPrefixProjection() {
- final DataType dataType =
- ROW(FIELD("k_part_1", INT()), FIELD("part_2", STRING()), FIELD("name", STRING()));
- final Map<String, String> options = createTestOptions();
- options.put("key.fields", "k_part_1;part_2");
- options.put("key.fields-prefix", "k_");
-
- final Configuration config = Configuration.fromMap(options);
-
- assertThatThrownBy(() -> createKeyFormatProjection(config, dataType))
- .isInstanceOf(ValidationException.class)
- .hasMessage(
- "All fields in 'key.fields' must be prefixed with 'k_' when option "
- + "'key.fields-prefix' is set but field 'part_2' is not prefixed.");
- }
-
- @Test
- public void testInvalidValueFormatProjection() {
- final DataType dataType = ROW(FIELD("k_id", INT()), FIELD("id", STRING()));
- final Map<String, String> options = createTestOptions();
- options.put("key.fields", "k_id");
- options.put("key.fields-prefix", "k_");
-
- final Configuration config = Configuration.fromMap(options);
-
- assertThatThrownBy(() -> createValueFormatProjection(config, dataType))
- .isInstanceOf(ValidationException.class)
- .hasMessage(
- "A key prefix is not allowed when option 'value.fields-include' "
- + "is set to 'ALL'. Set it to 'EXCEPT_KEY' instead to avoid field overlaps.");
- }
-
- // --------------------------------------------------------------------------------------------
-
- private static Map<String, String> createTestOptions() {
- final Map<String, String> options = new HashMap<>();
- options.put("key.format", "test-format");
- options.put("key.test-format.delimiter", ",");
- options.put("value.format", "test-format");
- options.put("value.test-format.delimiter", "|");
- options.put("value.test-format.fail-on-missing", "true");
- return options;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
deleted file mode 100644
index d7b5384..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java
+++ /dev/null
@@ -1,1340 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.connector.sink2.Sink;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.KafkaSourceOptions;
-import org.apache.flink.connector.kafka.source.KafkaSourceTestUtils;
-import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState;
-import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.formats.avro.AvroRowDataSerializationSchema;
-import org.apache.flink.formats.avro.RowDataToAvroConverters;
-import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema;
-import org.apache.flink.formats.avro.registry.confluent.debezium.DebeziumAvroSerializationSchema;
-import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.catalog.UniqueConstraint;
-import org.apache.flink.table.catalog.WatermarkSpec;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkV2Provider;
-import org.apache.flink.table.connector.source.DataStreamScanProvider;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.ScanTableSource;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.expressions.utils.ResolvedExpressionMock;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.TestFormatFactory;
-import org.apache.flink.table.factories.TestFormatFactory.DecodingFormatMock;
-import org.apache.flink.table.factories.TestFormatFactory.EncodingFormatMock;
-import org.apache.flink.table.factories.utils.FactoryMocks;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
-import org.apache.kafka.common.TopicPartition;
-import org.junit.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.NullSource;
-import org.junit.jupiter.params.provider.ValueSource;
-
-import javax.annotation.Nullable;
-
-import java.util.Arrays;
-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.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.regex.Pattern;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.DEBEZIUM_AVRO_CONFLUENT;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Tests for {@link KafkaDynamicTableFactory}. */
-@ExtendWith(TestLoggerExtension.class)
-public class KafkaDynamicTableFactoryTest {
-
- private static final String TOPIC = "myTopic";
- private static final String TOPICS = "myTopic-1;myTopic-2;myTopic-3";
- private static final String TOPIC_REGEX = "myTopic-\\d+";
- private static final List<String> TOPIC_LIST =
- Arrays.asList("myTopic-1", "myTopic-2", "myTopic-3");
- private static final String TEST_REGISTRY_URL = "http://localhost:8081";
- private static final String DEFAULT_VALUE_SUBJECT = TOPIC + "-value";
- private static final String DEFAULT_KEY_SUBJECT = TOPIC + "-key";
- private static final int PARTITION_0 = 0;
- private static final long OFFSET_0 = 100L;
- private static final int PARTITION_1 = 1;
- private static final long OFFSET_1 = 123L;
- private static final String NAME = "name";
- private static final String COUNT = "count";
- private static final String TIME = "time";
- private static final String METADATA = "metadata";
- private static final String WATERMARK_EXPRESSION = TIME + " - INTERVAL '5' SECOND";
- private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3);
- private static final String COMPUTED_COLUMN_NAME = "computed-column";
- private static final String COMPUTED_COLUMN_EXPRESSION = COUNT + " + 1.0";
- private static final DataType COMPUTED_COLUMN_DATATYPE = DataTypes.DECIMAL(10, 3);
- private static final String DISCOVERY_INTERVAL = "1000 ms";
-
- private static final Properties KAFKA_SOURCE_PROPERTIES = new Properties();
- private static final Properties KAFKA_FINAL_SOURCE_PROPERTIES = new Properties();
- private static final Properties KAFKA_SINK_PROPERTIES = new Properties();
- private static final Properties KAFKA_FINAL_SINK_PROPERTIES = new Properties();
-
- static {
- KAFKA_SOURCE_PROPERTIES.setProperty("group.id", "dummy");
- KAFKA_SOURCE_PROPERTIES.setProperty("bootstrap.servers", "dummy");
- KAFKA_SOURCE_PROPERTIES.setProperty("partition.discovery.interval.ms", "1000");
-
- KAFKA_SINK_PROPERTIES.setProperty("group.id", "dummy");
- KAFKA_SINK_PROPERTIES.setProperty("bootstrap.servers", "dummy");
-
- KAFKA_FINAL_SINK_PROPERTIES.putAll(KAFKA_SINK_PROPERTIES);
- KAFKA_FINAL_SOURCE_PROPERTIES.putAll(KAFKA_SOURCE_PROPERTIES);
- }
-
- private static final String PROPS_SCAN_OFFSETS =
- String.format(
- "partition:%d,offset:%d;partition:%d,offset:%d",
- PARTITION_0, OFFSET_0, PARTITION_1, OFFSET_1);
-
- private static final ResolvedSchema SCHEMA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical(NAME, DataTypes.STRING().notNull()),
- Column.physical(COUNT, DataTypes.DECIMAL(38, 18)),
- Column.physical(TIME, DataTypes.TIMESTAMP(3)),
- Column.computed(
- COMPUTED_COLUMN_NAME,
- ResolvedExpressionMock.of(
- COMPUTED_COLUMN_DATATYPE, COMPUTED_COLUMN_EXPRESSION))),
- Collections.singletonList(
- WatermarkSpec.of(
- TIME,
- ResolvedExpressionMock.of(
- WATERMARK_DATATYPE, WATERMARK_EXPRESSION))),
- null);
-
- private static final ResolvedSchema SCHEMA_WITH_METADATA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical(NAME, DataTypes.STRING()),
- Column.physical(COUNT, DataTypes.DECIMAL(38, 18)),
- Column.metadata(TIME, DataTypes.TIMESTAMP(3), "timestamp", false),
- Column.metadata(
- METADATA, DataTypes.STRING(), "value.metadata_2", false)),
- Collections.emptyList(),
- null);
-
- private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType();
-
- @Test
- public void testTableSource() {
- final DynamicTableSource actualSource = createTableSource(SCHEMA, getBasicSourceOptions());
- final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource;
-
- final Map<KafkaTopicPartition, Long> specificOffsets = new HashMap<>();
- specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0);
- specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1);
-
- final DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat =
- new DecodingFormatMock(",", true);
-
- // Test scan source equals
- final KafkaDynamicSource expectedKafkaSource =
- createExpectedScanSource(
- SCHEMA_DATA_TYPE,
- null,
- valueDecodingFormat,
- new int[0],
- new int[] {0, 1, 2},
- null,
- Collections.singletonList(TOPIC),
- null,
- KAFKA_SOURCE_PROPERTIES,
- StartupMode.SPECIFIC_OFFSETS,
- specificOffsets,
- 0);
- assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource);
-
- ScanTableSource.ScanRuntimeProvider provider =
- actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
- assertKafkaSource(provider);
- }
-
- @Test
- public void testTableSourceWithPattern() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options -> {
- options.remove("topic");
- options.put("topic-pattern", TOPIC_REGEX);
- options.put(
- "scan.startup.mode",
- ScanStartupMode.EARLIEST_OFFSET.toString());
- options.remove("scan.startup.specific-offsets");
- });
- final DynamicTableSource actualSource = createTableSource(SCHEMA, modifiedOptions);
-
- final Map<KafkaTopicPartition, Long> specificOffsets = new HashMap<>();
-
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat =
- new DecodingFormatMock(",", true);
-
- // Test scan source equals
- final KafkaDynamicSource expectedKafkaSource =
- createExpectedScanSource(
- SCHEMA_DATA_TYPE,
- null,
- valueDecodingFormat,
- new int[0],
- new int[] {0, 1, 2},
- null,
- null,
- Pattern.compile(TOPIC_REGEX),
- KAFKA_SOURCE_PROPERTIES,
- StartupMode.EARLIEST,
- specificOffsets,
- 0);
- final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource;
- assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource);
-
- ScanTableSource.ScanRuntimeProvider provider =
- actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
-
- assertKafkaSource(provider);
- }
-
- @Test
- public void testTableSourceWithKeyValue() {
- final DynamicTableSource actualSource = createTableSource(SCHEMA, getKeyValueOptions());
- final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource;
- // initialize stateful testing formats
- actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
-
- final DecodingFormatMock keyDecodingFormat = new DecodingFormatMock("#", false);
- keyDecodingFormat.producedDataType =
- DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull();
-
- final DecodingFormatMock valueDecodingFormat = new DecodingFormatMock("|", false);
- valueDecodingFormat.producedDataType =
- DataTypes.ROW(
- DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)),
- DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3)))
- .notNull();
-
- final KafkaDynamicSource expectedKafkaSource =
- createExpectedScanSource(
- SCHEMA_DATA_TYPE,
- keyDecodingFormat,
- valueDecodingFormat,
- new int[] {0},
- new int[] {1, 2},
- null,
- Collections.singletonList(TOPIC),
- null,
- KAFKA_FINAL_SOURCE_PROPERTIES,
- StartupMode.GROUP_OFFSETS,
- Collections.emptyMap(),
- 0);
-
- assertThat(actualSource).isEqualTo(expectedKafkaSource);
- }
-
- @Test
- public void testTableSourceWithKeyValueAndMetadata() {
- final Map<String, String> options = getKeyValueOptions();
- options.put("value.test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING");
-
- final DynamicTableSource actualSource = createTableSource(SCHEMA_WITH_METADATA, options);
- final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource;
- // initialize stateful testing formats
- actualKafkaSource.applyReadableMetadata(
- Arrays.asList("timestamp", "value.metadata_2"),
- SCHEMA_WITH_METADATA.toSourceRowDataType());
- actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
-
- final DecodingFormatMock expectedKeyFormat =
- new DecodingFormatMock(
- "#", false, ChangelogMode.insertOnly(), Collections.emptyMap());
- expectedKeyFormat.producedDataType =
- DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING())).notNull();
-
- final Map<String, DataType> expectedReadableMetadata = new HashMap<>();
- expectedReadableMetadata.put("metadata_1", DataTypes.INT());
- expectedReadableMetadata.put("metadata_2", DataTypes.STRING());
-
- final DecodingFormatMock expectedValueFormat =
- new DecodingFormatMock(
- "|", false, ChangelogMode.insertOnly(), expectedReadableMetadata);
- expectedValueFormat.producedDataType =
- DataTypes.ROW(
- DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)),
- DataTypes.FIELD("metadata_2", DataTypes.STRING()))
- .notNull();
- expectedValueFormat.metadataKeys = Collections.singletonList("metadata_2");
-
- final KafkaDynamicSource expectedKafkaSource =
- createExpectedScanSource(
- SCHEMA_WITH_METADATA.toPhysicalRowDataType(),
- expectedKeyFormat,
- expectedValueFormat,
- new int[] {0},
- new int[] {1},
- null,
- Collections.singletonList(TOPIC),
- null,
- KAFKA_FINAL_SOURCE_PROPERTIES,
- StartupMode.GROUP_OFFSETS,
- Collections.emptyMap(),
- 0);
- expectedKafkaSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType();
- expectedKafkaSource.metadataKeys = Collections.singletonList("timestamp");
-
- assertThat(actualSource).isEqualTo(expectedKafkaSource);
- }
-
- @Test
- public void testTableSourceCommitOnCheckpointDisabled() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(), options -> options.remove("properties.group.id"));
- final DynamicTableSource tableSource = createTableSource(SCHEMA, modifiedOptions);
-
- assertThat(tableSource).isInstanceOf(KafkaDynamicSource.class);
- ScanTableSource.ScanRuntimeProvider providerWithoutGroupId =
- ((KafkaDynamicSource) tableSource)
- .getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
- assertThat(providerWithoutGroupId).isInstanceOf(DataStreamScanProvider.class);
- final KafkaSource<?> kafkaSource = assertKafkaSource(providerWithoutGroupId);
- final Configuration configuration =
- KafkaSourceTestUtils.getKafkaSourceConfiguration(kafkaSource);
-
- // Test offset commit on checkpoint should be disabled when do not set consumer group.
- assertThat(configuration.get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT)).isFalse();
- assertThat(
- configuration.get(
- ConfigOptions.key(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)
- .booleanType()
- .noDefaultValue()))
- .isFalse();
- }
-
- @ParameterizedTest
- @ValueSource(strings = {"none", "earliest", "latest"})
- @NullSource
- public void testTableSourceSetOffsetReset(final String strategyName) {
- testSetOffsetResetForStartFromGroupOffsets(strategyName);
- }
-
- @Test
- public void testTableSourceSetOffsetResetWithException() {
- String errorStrategy = "errorStrategy";
- assertThatThrownBy(() -> testTableSourceSetOffsetReset(errorStrategy))
- .isInstanceOf(IllegalArgumentException.class)
- .hasMessage(
- String.format(
- "%s can not be set to %s. Valid values: [latest,earliest,none]",
- ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, errorStrategy));
- }
-
- private void testSetOffsetResetForStartFromGroupOffsets(String value) {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options -> {
- options.remove("scan.startup.mode");
- if (value == null) {
- return;
- }
- options.put(
- PROPERTIES_PREFIX + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
- value);
- });
- final DynamicTableSource tableSource = createTableSource(SCHEMA, modifiedOptions);
- assertThat(tableSource).isInstanceOf(KafkaDynamicSource.class);
- ScanTableSource.ScanRuntimeProvider provider =
- ((KafkaDynamicSource) tableSource)
- .getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
- assertThat(provider).isInstanceOf(DataStreamScanProvider.class);
- final KafkaSource<?> kafkaSource = assertKafkaSource(provider);
- final Configuration configuration =
- KafkaSourceTestUtils.getKafkaSourceConfiguration(kafkaSource);
-
- if (value == null) {
- assertThat(configuration.toMap().get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG))
- .isEqualTo("none");
- } else {
- assertThat(configuration.toMap().get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG))
- .isEqualTo(value);
- }
- }
-
- @Test
- public void testBoundedSpecificOffsetsValidate() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options -> {
- options.put(
- KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(),
- "specific-offsets");
- });
- assertThatThrownBy(() -> createTableSource(SCHEMA, modifiedOptions))
- .cause()
- .hasMessageContaining(
- "'scan.bounded.specific-offsets' is required in 'specific-offsets' bounded mode but missing.");
- }
-
- @Test
- public void testBoundedSpecificOffsets() {
- testBoundedOffsets(
- "specific-offsets",
- options -> {
- options.put("scan.bounded.specific-offsets", "partition:0,offset:2");
- },
- source -> {
- assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED);
- OffsetsInitializer offsetsInitializer =
- KafkaSourceTestUtils.getStoppingOffsetsInitializer(source);
- TopicPartition partition = new TopicPartition(TOPIC, 0);
- Map<TopicPartition, Long> partitionOffsets =
- offsetsInitializer.getPartitionOffsets(
- Collections.singletonList(partition),
- MockPartitionOffsetsRetriever.noInteractions());
- assertThat(partitionOffsets)
- .containsOnlyKeys(partition)
- .containsEntry(partition, 2L);
- });
- }
-
- @Test
- public void testBoundedLatestOffset() {
- testBoundedOffsets(
- "latest-offset",
- options -> {},
- source -> {
- assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED);
- OffsetsInitializer offsetsInitializer =
- KafkaSourceTestUtils.getStoppingOffsetsInitializer(source);
- TopicPartition partition = new TopicPartition(TOPIC, 0);
- Map<TopicPartition, Long> partitionOffsets =
- offsetsInitializer.getPartitionOffsets(
- Collections.singletonList(partition),
- MockPartitionOffsetsRetriever.noInteractions());
- assertThat(partitionOffsets)
- .containsOnlyKeys(partition)
- .containsEntry(partition, KafkaPartitionSplit.LATEST_OFFSET);
- });
- }
-
- @Test
- public void testBoundedGroupOffsets() {
- testBoundedOffsets(
- "group-offsets",
- options -> {},
- source -> {
- assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED);
- OffsetsInitializer offsetsInitializer =
- KafkaSourceTestUtils.getStoppingOffsetsInitializer(source);
- TopicPartition partition = new TopicPartition(TOPIC, 0);
- Map<TopicPartition, Long> partitionOffsets =
- offsetsInitializer.getPartitionOffsets(
- Collections.singletonList(partition),
- MockPartitionOffsetsRetriever.noInteractions());
- assertThat(partitionOffsets)
- .containsOnlyKeys(partition)
- .containsEntry(partition, KafkaPartitionSplit.COMMITTED_OFFSET);
- });
- }
-
- @Test
- public void testBoundedTimestamp() {
- testBoundedOffsets(
- "timestamp",
- options -> {
- options.put("scan.bounded.timestamp-millis", "1");
- },
- source -> {
- assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED);
- OffsetsInitializer offsetsInitializer =
- KafkaSourceTestUtils.getStoppingOffsetsInitializer(source);
- TopicPartition partition = new TopicPartition(TOPIC, 0);
- long offsetForTimestamp = 123L;
- Map<TopicPartition, Long> partitionOffsets =
- offsetsInitializer.getPartitionOffsets(
- Collections.singletonList(partition),
- MockPartitionOffsetsRetriever.timestampAndEnd(
- partitions -> {
- assertThat(partitions)
- .containsOnlyKeys(partition)
- .containsEntry(partition, 1L);
- Map<TopicPartition, OffsetAndTimestamp> result =
- new HashMap<>();
- result.put(
- partition,
- new OffsetAndTimestamp(
- offsetForTimestamp, 1L));
- return result;
- },
- partitions -> {
- Map<TopicPartition, Long> result = new HashMap<>();
- result.put(
- partition,
- // the end offset is bigger than given by
- // timestamp
- // to make sure the one for timestamp is
- // used
- offsetForTimestamp + 1000L);
- return result;
- }));
- assertThat(partitionOffsets)
- .containsOnlyKeys(partition)
- .containsEntry(partition, offsetForTimestamp);
- });
- }
-
- private void testBoundedOffsets(
- String boundedMode,
- Consumer<Map<String, String>> optionsConfig,
- Consumer<KafkaSource<?>> validator) {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options -> {
- options.put(KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), boundedMode);
- optionsConfig.accept(options);
- });
- final DynamicTableSource tableSource = createTableSource(SCHEMA, modifiedOptions);
- assertThat(tableSource).isInstanceOf(KafkaDynamicSource.class);
- ScanTableSource.ScanRuntimeProvider provider =
- ((KafkaDynamicSource) tableSource)
- .getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
- assertThat(provider).isInstanceOf(DataStreamScanProvider.class);
- final KafkaSource<?> kafkaSource = assertKafkaSource(provider);
- validator.accept(kafkaSource);
- }
-
- private interface OffsetsRetriever
- extends Function<Collection<TopicPartition>, Map<TopicPartition, Long>> {}
-
- private interface TimestampOffsetsRetriever
- extends Function<Map<TopicPartition, Long>, Map<TopicPartition, OffsetAndTimestamp>> {}
-
- private static final class MockPartitionOffsetsRetriever
- implements OffsetsInitializer.PartitionOffsetsRetriever {
-
- public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL =
- partitions -> {
- throw new UnsupportedOperationException(
- "The method was not supposed to be called");
- };
- private final OffsetsRetriever committedOffsets;
- private final OffsetsRetriever endOffsets;
- private final OffsetsRetriever beginningOffsets;
- private final TimestampOffsetsRetriever offsetsForTimes;
-
- static MockPartitionOffsetsRetriever noInteractions() {
- return new MockPartitionOffsetsRetriever(
- UNSUPPORTED_RETRIEVAL,
- UNSUPPORTED_RETRIEVAL,
- UNSUPPORTED_RETRIEVAL,
- partitions -> {
- throw new UnsupportedOperationException(
- "The method was not supposed to be called");
- });
- }
-
- static MockPartitionOffsetsRetriever timestampAndEnd(
- TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) {
- return new MockPartitionOffsetsRetriever(
- UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever);
- }
-
- private MockPartitionOffsetsRetriever(
- OffsetsRetriever committedOffsets,
- OffsetsRetriever endOffsets,
- OffsetsRetriever beginningOffsets,
- TimestampOffsetsRetriever offsetsForTimes) {
- this.committedOffsets = committedOffsets;
- this.endOffsets = endOffsets;
- this.beginningOffsets = beginningOffsets;
- this.offsetsForTimes = offsetsForTimes;
- }
-
- @Override
- public Map<TopicPartition, Long> committedOffsets(Collection<TopicPartition> partitions) {
- return committedOffsets.apply(partitions);
- }
-
- @Override
- public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions) {
- return endOffsets.apply(partitions);
- }
-
- @Override
- public Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions) {
- return beginningOffsets.apply(partitions);
- }
-
- @Override
- public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(
- Map<TopicPartition, Long> timestampsToSearch) {
- return offsetsForTimes.apply(timestampsToSearch);
- }
- }
-
- @Test
- public void testTableSink() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options -> {
- options.put("sink.delivery-guarantee", "exactly-once");
- options.put("sink.transactional-id-prefix", "kafka-sink");
- });
- final DynamicTableSink actualSink = createTableSink(SCHEMA, modifiedOptions);
-
- final EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- new EncodingFormatMock(",");
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SCHEMA_DATA_TYPE,
- null,
- valueEncodingFormat,
- new int[0],
- new int[] {0, 1, 2},
- null,
- TOPIC,
- KAFKA_SINK_PROPERTIES,
- new FlinkFixedPartitioner<>(),
- DeliveryGuarantee.EXACTLY_ONCE,
- null,
- "kafka-sink");
- assertThat(actualSink).isEqualTo(expectedSink);
-
- // Test kafka producer.
- final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink;
- DynamicTableSink.SinkRuntimeProvider provider =
- actualKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
- assertThat(provider).isInstanceOf(SinkV2Provider.class);
- final SinkV2Provider sinkProvider = (SinkV2Provider) provider;
- final Sink<RowData> sinkFunction = sinkProvider.createSink();
- assertThat(sinkFunction).isInstanceOf(KafkaSink.class);
- }
-
- @Test
- public void testTableSinkSemanticTranslation() {
- final List<String> semantics = ImmutableList.of("exactly-once", "at-least-once", "none");
- final EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- new EncodingFormatMock(",");
- for (final String semantic : semantics) {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options -> {
- options.put("sink.semantic", semantic);
- options.put("sink.transactional-id-prefix", "kafka-sink");
- });
- final DynamicTableSink actualSink = createTableSink(SCHEMA, modifiedOptions);
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SCHEMA_DATA_TYPE,
- null,
- valueEncodingFormat,
- new int[0],
- new int[] {0, 1, 2},
- null,
- TOPIC,
- KAFKA_SINK_PROPERTIES,
- new FlinkFixedPartitioner<>(),
- DeliveryGuarantee.valueOf(semantic.toUpperCase().replace("-", "_")),
- null,
- "kafka-sink");
- assertThat(actualSink).isEqualTo(expectedSink);
- }
- }
-
- @Test
- public void testTableSinkWithKeyValue() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getKeyValueOptions(),
- options -> {
- options.put("sink.delivery-guarantee", "exactly-once");
- options.put("sink.transactional-id-prefix", "kafka-sink");
- });
- final DynamicTableSink actualSink = createTableSink(SCHEMA, modifiedOptions);
- final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink;
- // initialize stateful testing formats
- actualKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
-
- final EncodingFormatMock keyEncodingFormat = new EncodingFormatMock("#");
- keyEncodingFormat.consumedDataType =
- DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull();
-
- final EncodingFormatMock valueEncodingFormat = new EncodingFormatMock("|");
- valueEncodingFormat.consumedDataType =
- DataTypes.ROW(
- DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)),
- DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3)))
- .notNull();
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SCHEMA_DATA_TYPE,
- keyEncodingFormat,
- valueEncodingFormat,
- new int[] {0},
- new int[] {1, 2},
- null,
- TOPIC,
- KAFKA_FINAL_SINK_PROPERTIES,
- new FlinkFixedPartitioner<>(),
- DeliveryGuarantee.EXACTLY_ONCE,
- null,
- "kafka-sink");
-
- assertThat(actualSink).isEqualTo(expectedSink);
- }
-
- @Test
- public void testTableSinkWithParallelism() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(), options -> options.put("sink.parallelism", "100"));
- KafkaDynamicSink actualSink = (KafkaDynamicSink) createTableSink(SCHEMA, modifiedOptions);
-
- final EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- new EncodingFormatMock(",");
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SCHEMA_DATA_TYPE,
- null,
- valueEncodingFormat,
- new int[0],
- new int[] {0, 1, 2},
- null,
- TOPIC,
- KAFKA_SINK_PROPERTIES,
- new FlinkFixedPartitioner<>(),
- DeliveryGuarantee.EXACTLY_ONCE,
- 100,
- "kafka-sink");
- assertThat(actualSink).isEqualTo(expectedSink);
-
- final DynamicTableSink.SinkRuntimeProvider provider =
- actualSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
- assertThat(provider).isInstanceOf(SinkV2Provider.class);
- final SinkV2Provider sinkProvider = (SinkV2Provider) provider;
- assertThat(sinkProvider.getParallelism().isPresent()).isTrue();
- assertThat((long) sinkProvider.getParallelism().get()).isEqualTo(100);
- }
-
- @Test
- public void testTableSinkAutoCompleteSchemaRegistrySubject() {
- // only format
- verifyEncoderSubject(
- options -> {
- options.put("format", "debezium-avro-confluent");
- options.put("debezium-avro-confluent.url", TEST_REGISTRY_URL);
- },
- DEFAULT_VALUE_SUBJECT,
- "N/A");
-
- // only value.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- },
- DEFAULT_VALUE_SUBJECT,
- "N/A");
-
- // value.format + key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.fields", NAME);
- },
- DEFAULT_VALUE_SUBJECT,
- DEFAULT_KEY_SUBJECT);
-
- // value.format + non-avro key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "csv");
- options.put("key.fields", NAME);
- },
- DEFAULT_VALUE_SUBJECT,
- "N/A");
-
- // non-avro value.format + key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "json");
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.fields", NAME);
- },
- "N/A",
- DEFAULT_KEY_SUBJECT);
-
- // not override for 'format'
- verifyEncoderSubject(
- options -> {
- options.put("format", "debezium-avro-confluent");
- options.put("debezium-avro-confluent.url", TEST_REGISTRY_URL);
- options.put("debezium-avro-confluent.subject", "sub1");
- },
- "sub1",
- "N/A");
-
- // not override for 'key.format'
- verifyEncoderSubject(
- options -> {
- options.put("format", "avro-confluent");
- options.put("avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.avro-confluent.subject", "sub2");
- options.put("key.fields", NAME);
- },
- DEFAULT_VALUE_SUBJECT,
- "sub2");
- }
-
- private void verifyEncoderSubject(
- Consumer<Map<String, String>> optionModifier,
- String expectedValueSubject,
- String expectedKeySubject) {
- Map<String, String> options = new HashMap<>();
- // Kafka specific options.
- options.put("connector", KafkaDynamicTableFactory.IDENTIFIER);
- options.put("topic", TOPIC);
- options.put("properties.group.id", "dummy");
- options.put("properties.bootstrap.servers", "dummy");
- optionModifier.accept(options);
-
- final RowType rowType = (RowType) SCHEMA_DATA_TYPE.getLogicalType();
- final String valueFormat =
- options.getOrDefault(
- FactoryUtil.FORMAT.key(),
- options.get(KafkaConnectorOptions.VALUE_FORMAT.key()));
- final String keyFormat = options.get(KafkaConnectorOptions.KEY_FORMAT.key());
-
- KafkaDynamicSink sink = (KafkaDynamicSink) createTableSink(SCHEMA, options);
- final Set<String> avroFormats = new HashSet<>();
- avroFormats.add(AVRO_CONFLUENT);
- avroFormats.add(DEBEZIUM_AVRO_CONFLUENT);
-
- if (avroFormats.contains(valueFormat)) {
- SerializationSchema<RowData> actualValueEncoder =
- sink.valueEncodingFormat.createRuntimeEncoder(
- new SinkRuntimeProviderContext(false), SCHEMA_DATA_TYPE);
- final SerializationSchema<RowData> expectedValueEncoder;
- if (AVRO_CONFLUENT.equals(valueFormat)) {
- expectedValueEncoder = createConfluentAvroSerSchema(rowType, expectedValueSubject);
- } else {
- expectedValueEncoder = createDebeziumAvroSerSchema(rowType, expectedValueSubject);
- }
- assertThat(actualValueEncoder).isEqualTo(expectedValueEncoder);
- }
-
- if (avroFormats.contains(keyFormat)) {
- assertThat(sink.keyEncodingFormat).isNotNull();
- SerializationSchema<RowData> actualKeyEncoder =
- sink.keyEncodingFormat.createRuntimeEncoder(
- new SinkRuntimeProviderContext(false), SCHEMA_DATA_TYPE);
- final SerializationSchema<RowData> expectedKeyEncoder;
- if (AVRO_CONFLUENT.equals(keyFormat)) {
- expectedKeyEncoder = createConfluentAvroSerSchema(rowType, expectedKeySubject);
- } else {
- expectedKeyEncoder = createDebeziumAvroSerSchema(rowType, expectedKeySubject);
- }
- assertThat(actualKeyEncoder).isEqualTo(expectedKeyEncoder);
- }
- }
-
- private SerializationSchema<RowData> createConfluentAvroSerSchema(
- RowType rowType, String subject) {
- return new AvroRowDataSerializationSchema(
- rowType,
- ConfluentRegistryAvroSerializationSchema.forGeneric(
- subject, AvroSchemaConverter.convertToSchema(rowType), TEST_REGISTRY_URL),
- RowDataToAvroConverters.createConverter(rowType));
- }
-
- private SerializationSchema<RowData> createDebeziumAvroSerSchema(
- RowType rowType, String subject) {
- return new DebeziumAvroSerializationSchema(rowType, TEST_REGISTRY_URL, subject, null, null);
- }
-
- // --------------------------------------------------------------------------------------------
- // Negative tests
- // --------------------------------------------------------------------------------------------
-
- @Test
- public void testSourceTableWithTopicAndTopicPattern() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options -> {
- options.put("topic", TOPICS);
- options.put("topic-pattern", TOPIC_REGEX);
- });
-
- createTableSource(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "Option 'topic' and 'topic-pattern' shouldn't be set together."));
- }
-
- @Test
- public void testMissingStartupTimestamp() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options ->
- options.put("scan.startup.mode", "timestamp"));
-
- createTableSource(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "'scan.startup.timestamp-millis' "
- + "is required in 'timestamp' startup mode but missing."));
- }
-
- @Test
- public void testMissingSpecificOffsets() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options ->
- options.remove(
- "scan.startup.specific-offsets"));
-
- createTableSource(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "'scan.startup.specific-offsets' "
- + "is required in 'specific-offsets' startup mode but missing."));
- }
-
- @Test
- public void testInvalidSinkPartitioner() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options -> options.put("sink.partitioner", "abc"));
-
- createTableSink(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "Could not find and instantiate partitioner " + "class 'abc'"));
- }
-
- @Test
- public void testInvalidRoundRobinPartitionerWithKeyFields() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getKeyValueOptions(),
- options ->
- options.put("sink.partitioner", "round-robin"));
-
- createTableSink(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "Currently 'round-robin' partitioner only works "
- + "when option 'key.fields' is not specified."));
- }
-
- @Test
- public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() {
- assertThatThrownBy(
- () -> {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getKeyValueOptions(),
- options -> {
- options.remove(
- KafkaConnectorOptions
- .TRANSACTIONAL_ID_PREFIX
- .key());
- options.put(
- KafkaConnectorOptions.DELIVERY_GUARANTEE
- .key(),
- DeliveryGuarantee.EXACTLY_ONCE.toString());
- });
- createTableSink(SCHEMA, modifiedOptions);
- })
- .isInstanceOf(ValidationException.class)
- .satisfies(
- anyCauseMatches(
- ValidationException.class,
- "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE."));
- }
-
- @Test
- public void testSinkWithTopicListOrTopicPattern() {
- Map<String, String> modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options -> {
- options.put("topic", TOPICS);
- options.put("scan.startup.mode", "earliest-offset");
- options.remove("specific-offsets");
- });
- final String errorMessageTemp =
- "Flink Kafka sink currently only supports single topic, but got %s: %s.";
-
- try {
- createTableSink(SCHEMA, modifiedOptions);
- } catch (Throwable t) {
- assertThat(t.getCause().getMessage())
- .isEqualTo(
- String.format(
- errorMessageTemp,
- "'topic'",
- String.format("[%s]", String.join(", ", TOPIC_LIST))));
- }
-
- modifiedOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options -> options.put("topic-pattern", TOPIC_REGEX));
-
- try {
- createTableSink(SCHEMA, modifiedOptions);
- } catch (Throwable t) {
- assertThat(t.getCause().getMessage())
- .isEqualTo(String.format(errorMessageTemp, "'topic-pattern'", TOPIC_REGEX));
- }
- }
-
- @Test
- public void testPrimaryKeyValidation() {
- final ResolvedSchema pkSchema =
- new ResolvedSchema(
- SCHEMA.getColumns(),
- SCHEMA.getWatermarkSpecs(),
- UniqueConstraint.primaryKey(NAME, Collections.singletonList(NAME)));
-
- Map<String, String> sinkOptions =
- getModifiedOptions(
- getBasicSinkOptions(),
- options ->
- options.put(
- String.format(
- "%s.%s",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.CHANGELOG_MODE.key()),
- "I;UA;UB;D"));
- // pk can be defined on cdc table, should pass
- createTableSink(pkSchema, sinkOptions);
-
- assertThatExceptionOfType(ValidationException.class)
- .isThrownBy(() -> createTableSink(pkSchema, getBasicSinkOptions()))
- .havingRootCause()
- .withMessage(
- "The Kafka table 'default.default.t1' with 'test-format' format"
- + " doesn't support defining PRIMARY KEY constraint on the table, because it can't"
- + " guarantee the semantic of primary key.");
-
- assertThatExceptionOfType(ValidationException.class)
- .isThrownBy(() -> createTableSink(pkSchema, getKeyValueOptions()))
- .havingRootCause()
- .withMessage(
- "The Kafka table 'default.default.t1' with 'test-format' format"
- + " doesn't support defining PRIMARY KEY constraint on the table, because it can't"
- + " guarantee the semantic of primary key.");
-
- Map<String, String> sourceOptions =
- getModifiedOptions(
- getBasicSourceOptions(),
- options ->
- options.put(
- String.format(
- "%s.%s",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.CHANGELOG_MODE.key()),
- "I;UA;UB;D"));
- // pk can be defined on cdc table, should pass
- createTableSource(pkSchema, sourceOptions);
-
- assertThatExceptionOfType(ValidationException.class)
- .isThrownBy(() -> createTableSource(pkSchema, getBasicSourceOptions()))
- .havingRootCause()
- .withMessage(
- "The Kafka table 'default.default.t1' with 'test-format' format"
- + " doesn't support defining PRIMARY KEY constraint on the table, because it can't"
- + " guarantee the semantic of primary key.");
- }
-
- // --------------------------------------------------------------------------------------------
- // Utilities
- // --------------------------------------------------------------------------------------------
-
- private static KafkaDynamicSource createExpectedScanSource(
- DataType physicalDataType,
- @Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- @Nullable List<String> topics,
- @Nullable Pattern topicPattern,
- Properties properties,
- StartupMode startupMode,
- Map<KafkaTopicPartition, Long> specificStartupOffsets,
- long startupTimestampMillis) {
- return new KafkaDynamicSource(
- physicalDataType,
- keyDecodingFormat,
- valueDecodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topics,
- topicPattern,
- properties,
- startupMode,
- specificStartupOffsets,
- startupTimestampMillis,
- BoundedMode.UNBOUNDED,
- Collections.emptyMap(),
- 0,
- false,
- FactoryMocks.IDENTIFIER.asSummaryString());
- }
-
- private static KafkaDynamicSink createExpectedSink(
- DataType physicalDataType,
- @Nullable EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat,
- EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- @Nullable String keyPrefix,
- String topic,
- Properties properties,
- @Nullable FlinkKafkaPartitioner<RowData> partitioner,
- DeliveryGuarantee deliveryGuarantee,
- @Nullable Integer parallelism,
- String transactionalIdPrefix) {
- return new KafkaDynamicSink(
- physicalDataType,
- physicalDataType,
- keyEncodingFormat,
- valueEncodingFormat,
- keyProjection,
- valueProjection,
- keyPrefix,
- topic,
- properties,
- partitioner,
- deliveryGuarantee,
- false,
- SinkBufferFlushMode.DISABLED,
- parallelism,
- transactionalIdPrefix);
- }
-
- /**
- * Returns the full options modified by the given consumer {@code optionModifier}.
- *
- * @param optionModifier Consumer to modify the options
- */
- private static Map<String, String> getModifiedOptions(
- Map<String, String> options, Consumer<Map<String, String>> optionModifier) {
- optionModifier.accept(options);
- return options;
- }
-
- private static Map<String, String> getBasicSourceOptions() {
- Map<String, String> tableOptions = new HashMap<>();
- // Kafka specific options.
- tableOptions.put("connector", KafkaDynamicTableFactory.IDENTIFIER);
- tableOptions.put("topic", TOPIC);
- tableOptions.put("properties.group.id", "dummy");
- tableOptions.put("properties.bootstrap.servers", "dummy");
- tableOptions.put("scan.startup.mode", "specific-offsets");
- tableOptions.put("scan.startup.specific-offsets", PROPS_SCAN_OFFSETS);
- tableOptions.put("scan.topic-partition-discovery.interval", DISCOVERY_INTERVAL);
- // Format options.
- tableOptions.put("format", TestFormatFactory.IDENTIFIER);
- final String formatDelimiterKey =
- String.format(
- "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key());
- final String failOnMissingKey =
- String.format(
- "%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key());
- tableOptions.put(formatDelimiterKey, ",");
- tableOptions.put(failOnMissingKey, "true");
- return tableOptions;
- }
-
- private static Map<String, String> getBasicSinkOptions() {
- Map<String, String> tableOptions = new HashMap<>();
- // Kafka specific options.
- tableOptions.put("connector", KafkaDynamicTableFactory.IDENTIFIER);
- tableOptions.put("topic", TOPIC);
- tableOptions.put("properties.group.id", "dummy");
- tableOptions.put("properties.bootstrap.servers", "dummy");
- tableOptions.put(
- "sink.partitioner", KafkaConnectorOptionsUtil.SINK_PARTITIONER_VALUE_FIXED);
- tableOptions.put("sink.delivery-guarantee", DeliveryGuarantee.EXACTLY_ONCE.toString());
- tableOptions.put("sink.transactional-id-prefix", "kafka-sink");
- // Format options.
- tableOptions.put("format", TestFormatFactory.IDENTIFIER);
- final String formatDelimiterKey =
- String.format(
- "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key());
- tableOptions.put(formatDelimiterKey, ",");
- return tableOptions;
- }
-
- private static Map<String, String> getKeyValueOptions() {
- Map<String, String> tableOptions = new HashMap<>();
- // Kafka specific options.
- tableOptions.put("connector", KafkaDynamicTableFactory.IDENTIFIER);
- tableOptions.put("topic", TOPIC);
- tableOptions.put("properties.group.id", "dummy");
- tableOptions.put("properties.bootstrap.servers", "dummy");
- tableOptions.put("scan.topic-partition-discovery.interval", DISCOVERY_INTERVAL);
- tableOptions.put(
- "sink.partitioner", KafkaConnectorOptionsUtil.SINK_PARTITIONER_VALUE_FIXED);
- tableOptions.put("sink.delivery-guarantee", DeliveryGuarantee.EXACTLY_ONCE.toString());
- tableOptions.put("sink.transactional-id-prefix", "kafka-sink");
- // Format options.
- tableOptions.put("key.format", TestFormatFactory.IDENTIFIER);
- tableOptions.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- "#");
- tableOptions.put("key.fields", NAME);
- tableOptions.put("value.format", TestFormatFactory.IDENTIFIER);
- tableOptions.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- "|");
- tableOptions.put(
- "value.fields-include",
- KafkaConnectorOptions.ValueFieldsStrategy.EXCEPT_KEY.toString());
- return tableOptions;
- }
-
- private KafkaSource<?> assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) {
- assertThat(provider).isInstanceOf(DataStreamScanProvider.class);
- final DataStreamScanProvider dataStreamScanProvider = (DataStreamScanProvider) provider;
- final Transformation<RowData> transformation =
- dataStreamScanProvider
- .produceDataStream(
- n -> Optional.empty(),
- StreamExecutionEnvironment.createLocalEnvironment())
- .getTransformation();
- assertThat(transformation).isInstanceOf(SourceTransformation.class);
- SourceTransformation<RowData, KafkaPartitionSplit, KafkaSourceEnumState>
- sourceTransformation =
- (SourceTransformation<RowData, KafkaPartitionSplit, KafkaSourceEnumState>)
- transformation;
- assertThat(sourceTransformation.getSource()).isInstanceOf(KafkaSource.class);
- return (KafkaSource<?>) sourceTransformation.getSource();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
deleted file mode 100644
index 515526f..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
+++ /dev/null
@@ -1,1169 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.core.execution.JobClient;
-import org.apache.flink.core.testutils.FlinkAssertions;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.config.TableConfigOptions;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.utils.EncodingUtils;
-import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.CloseableIterator;
-
-import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.assertj.core.api.Assertions;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.time.Duration;
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines;
-import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT;
-import static org.apache.flink.table.utils.TableTestMatchers.deepEqualTo;
-import static org.apache.flink.util.CollectionUtil.entry;
-import static org.apache.flink.util.CollectionUtil.map;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-import static org.assertj.core.api.HamcrestCondition.matching;
-
-/** Basic IT cases for the Kafka table source and sink. */
-@RunWith(Parameterized.class)
-public class KafkaTableITCase extends KafkaTableTestBase {
-
- private static final String JSON_FORMAT = "json";
- private static final String AVRO_FORMAT = "avro";
- private static final String CSV_FORMAT = "csv";
-
- @Parameterized.Parameter public String format;
-
- @Parameterized.Parameters(name = "format = {0}")
- public static Collection<String> parameters() {
- return Arrays.asList(JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT);
- }
-
- @Before
- public void before() {
- // we have to use single parallelism,
- // because we will count the messages in sink to terminate the job
- env.setParallelism(1);
- }
-
- @Test
- public void testKafkaSourceSink() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "tstopic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- final String createTable =
- String.format(
- "create table kafka (\n"
- + " `computed-price` as price + 1.0,\n"
- + " price decimal(38, 18),\n"
- + " currency string,\n"
- + " log_date date,\n"
- + " log_time time(3),\n"
- + " log_ts timestamp(3),\n"
- + " ts as log_ts + INTERVAL '1' SECOND,\n"
- + " watermark for ts as ts\n"
- + ") with (\n"
- + " 'connector' = '%s',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " %s\n"
- + ")",
- KafkaDynamicTableFactory.IDENTIFIER,
- topic,
- bootstraps,
- groupId,
- formatOptions());
-
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO kafka\n"
- + "SELECT CAST(price AS DECIMAL(10, 2)), currency, "
- + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n"
- + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n"
- + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n"
- + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n"
- + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n"
- + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n"
- + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n"
- + " AS orders (price, currency, d, t, ts)";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- String query =
- "SELECT\n"
- + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n"
- + " CAST(MAX(log_date) AS VARCHAR),\n"
- + " CAST(MAX(log_time) AS VARCHAR),\n"
- + " CAST(MAX(ts) AS VARCHAR),\n"
- + " COUNT(*),\n"
- + " CAST(MAX(price) AS DECIMAL(10, 2))\n"
- + "FROM kafka\n"
- + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)";
-
- DataStream<RowData> result = tEnv.toAppendStream(tEnv.sqlQuery(query), RowData.class);
- TestingSinkFunction sink = new TestingSinkFunction(2);
- result.addSink(sink).setParallelism(1);
-
- try {
- env.execute("Job_2");
- } catch (Throwable e) {
- // we have to use a specific exception to indicate the job is finished,
- // because the registered Kafka source is infinite.
- if (!isCausedByJobFinished(e)) {
- // re-throw
- throw e;
- }
- }
-
- List<String> expected =
- Arrays.asList(
- "+I(2019-12-12 00:00:05.000,2019-12-12,00:00:03,2019-12-12 00:00:04.004,3,50.00)",
- "+I(2019-12-12 00:00:10.000,2019-12-12,00:00:05,2019-12-12 00:00:06.006,2,5.33)");
-
- assertThat(TestingSinkFunction.rows).isEqualTo(expected);
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "bounded_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `user_id` INT,\n"
- + " `item_id` INT,\n"
- + " `behavior` STRING\n"
- + ") WITH (\n"
- + " 'connector' = '%s',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'scan.bounded.mode' = 'specific-offsets',\n"
- + " 'scan.bounded.specific-offsets' = 'partition:0,offset:2',\n"
- + " %s\n"
- + ")\n",
- KafkaDynamicTableFactory.IDENTIFIER,
- topic,
- bootstraps,
- groupId,
- formatOptions());
-
- tEnv.executeSql(createTable);
-
- List<Row> values =
- Arrays.asList(
- Row.of(1, 1102, "behavior 1"),
- Row.of(2, 1103, "behavior 2"),
- Row.of(3, 1104, "behavior 3"));
- tEnv.fromValues(values).insertInto("kafka").execute().await();
-
- // ---------- Consume stream from Kafka -------------------
-
- List<Row> results = new ArrayList<>();
- try (CloseableIterator<Row> resultsItr =
- tEnv.sqlQuery("SELECT * from kafka").execute().collect()) {
- while (resultsItr.hasNext()) {
- results.add(resultsItr.next());
- }
- }
-
- assertThat(results)
- .containsExactly(Row.of(1, 1102, "behavior 1"), Row.of(2, 1103, "behavior 2"));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "bounded_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `user_id` INT,\n"
- + " `item_id` INT,\n"
- + " `behavior` STRING,\n"
- + " `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp'"
- + ") WITH (\n"
- + " 'connector' = '%s',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'scan.bounded.mode' = 'timestamp',\n"
- + " 'scan.bounded.timestamp-millis' = '5',\n"
- + " %s\n"
- + ")\n",
- KafkaDynamicTableFactory.IDENTIFIER,
- topic,
- bootstraps,
- groupId,
- formatOptions());
-
- tEnv.executeSql(createTable);
-
- List<Row> values =
- Arrays.asList(
- Row.of(1, 1102, "behavior 1", Instant.ofEpochMilli(0L)),
- Row.of(2, 1103, "behavior 2", Instant.ofEpochMilli(3L)),
- Row.of(3, 1104, "behavior 3", Instant.ofEpochMilli(7L)));
- tEnv.fromValues(values).insertInto("kafka").execute().await();
-
- // ---------- Consume stream from Kafka -------------------
-
- List<Row> results = new ArrayList<>();
- try (CloseableIterator<Row> resultsItr =
- tEnv.sqlQuery("SELECT * from kafka").execute().collect()) {
- while (resultsItr.hasNext()) {
- results.add(resultsItr.next());
- }
- }
-
- assertThat(results)
- .containsExactly(
- Row.of(1, 1102, "behavior 1", Instant.ofEpochMilli(0L)),
- Row.of(2, 1103, "behavior 2", Instant.ofEpochMilli(3L)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaTableWithMultipleTopics() throws Exception {
- // ---------- create source and sink tables -------------------
- String tableTemp =
- "create table %s (\n"
- + " currency string\n"
- + ") with (\n"
- + " 'connector' = '%s',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " %s\n"
- + ")";
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
- List<String> currencies = Arrays.asList("Euro", "Dollar", "Yen", "Dummy");
- List<String> topics =
- currencies.stream()
- .map(
- currency ->
- String.format(
- "%s_%s_%s", currency, format, UUID.randomUUID()))
- .collect(Collectors.toList());
- // Because kafka connector currently doesn't support write data into multiple topic
- // together,
- // we have to create multiple sink tables.
- IntStream.range(0, 4)
- .forEach(
- index -> {
- createTestTopic(topics.get(index), 1, 1);
- tEnv.executeSql(
- String.format(
- tableTemp,
- currencies.get(index).toLowerCase(),
- KafkaDynamicTableFactory.IDENTIFIER,
- topics.get(index),
- bootstraps,
- groupId,
- formatOptions()));
- });
- // create source table
- tEnv.executeSql(
- String.format(
- tableTemp,
- "currencies_topic_list",
- KafkaDynamicTableFactory.IDENTIFIER,
- String.join(";", topics),
- bootstraps,
- groupId,
- formatOptions()));
-
- // ---------- Prepare data in Kafka topics -------------------
- String insertTemp =
- "INSERT INTO %s\n"
- + "SELECT currency\n"
- + " FROM (VALUES ('%s'))\n"
- + " AS orders (currency)";
- currencies.forEach(
- currency -> {
- try {
- tEnv.executeSql(String.format(insertTemp, currency.toLowerCase(), currency))
- .await();
- } catch (Exception e) {
- fail(e.getMessage());
- }
- });
-
- // ------------- test the topic-list kafka source -------------------
- DataStream<RowData> result =
- tEnv.toAppendStream(
- tEnv.sqlQuery("SELECT currency FROM currencies_topic_list"), RowData.class);
- TestingSinkFunction sink = new TestingSinkFunction(4); // expect to receive 4 records
- result.addSink(sink);
-
- try {
- env.execute("Job_3");
- } catch (Throwable e) {
- // we have to use a specific exception to indicate the job is finished,
- // because the registered Kafka source is infinite.
- if (!isCausedByJobFinished(e)) {
- // re-throw
- throw e;
- }
- }
- List<String> expected = Arrays.asList("+I(Dollar)", "+I(Dummy)", "+I(Euro)", "+I(Yen)");
- TestingSinkFunction.rows.sort(Comparator.naturalOrder());
- assertThat(TestingSinkFunction.rows).isEqualTo(expected);
-
- // ------------- cleanup -------------------
- topics.forEach(super::deleteTestTopic);
- }
-
- @Test
- public void testKafkaSourceSinkWithMetadata() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "metadata_topic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `physical_1` STRING,\n"
- + " `physical_2` INT,\n"
- // metadata fields are out of order on purpose
- // offset is ignored because it might not be deterministic
- + " `timestamp-type` STRING METADATA VIRTUAL,\n"
- + " `timestamp` TIMESTAMP(3) METADATA,\n"
- + " `leader-epoch` INT METADATA VIRTUAL,\n"
- + " `headers` MAP<STRING, BYTES> METADATA,\n"
- + " `partition` INT METADATA VIRTUAL,\n"
- + " `topic` STRING METADATA VIRTUAL,\n"
- + " `physical_3` BOOLEAN\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " %s\n"
- + ")",
- topic, bootstraps, groupId, formatOptions());
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO kafka\n"
- + "VALUES\n"
- + " ('data 1', 1, TIMESTAMP '2020-03-08 13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE01'], TRUE),\n"
- + " ('data 2', 2, TIMESTAMP '2020-03-09 13:12:11.123', CAST(NULL AS MAP<STRING, BYTES>), FALSE),\n"
- + " ('data 3', 3, TIMESTAMP '2020-03-10 13:12:11.123', MAP['k1', X'102030', 'k2', X'203040'], TRUE)";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM kafka"), 3);
-
- final List<Row> expected =
- Arrays.asList(
- Row.of(
- "data 1",
- 1,
- "CreateTime",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 0,
- map(
- entry("k1", EncodingUtils.decodeHex("C0FFEE")),
- entry("k2", EncodingUtils.decodeHex("BABE01"))),
- 0,
- topic,
- true),
- Row.of(
- "data 2",
- 2,
- "CreateTime",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 0,
- Collections.emptyMap(),
- 0,
- topic,
- false),
- Row.of(
- "data 3",
- 3,
- "CreateTime",
- LocalDateTime.parse("2020-03-10T13:12:11.123"),
- 0,
- map(
- entry("k1", EncodingUtils.decodeHex("102030")),
- entry("k2", EncodingUtils.decodeHex("203040"))),
- 0,
- topic,
- true));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "key_partial_value_topic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- // k_user_id and user_id have different data types to verify the correct mapping,
- // fields are reordered on purpose
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `k_user_id` BIGINT,\n"
- + " `name` STRING,\n"
- + " `timestamp` TIMESTAMP(3) METADATA,\n"
- + " `k_event_id` BIGINT,\n"
- + " `user_id` INT,\n"
- + " `payload` STRING\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'key.format' = '%s',\n"
- + " 'key.fields' = 'k_event_id; k_user_id',\n"
- + " 'key.fields-prefix' = 'k_',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'EXCEPT_KEY'\n"
- + ")",
- topic, bootstraps, groupId, format, format);
-
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO kafka\n"
- + "VALUES\n"
- + " (1, 'name 1', TIMESTAMP '2020-03-08 13:12:11.123', 100, 41, 'payload 1'),\n"
- + " (2, 'name 2', TIMESTAMP '2020-03-09 13:12:11.123', 101, 42, 'payload 2'),\n"
- + " (3, 'name 3', TIMESTAMP '2020-03-10 13:12:11.123', 102, 43, 'payload 3')";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM kafka"), 3);
-
- final List<Row> expected =
- Arrays.asList(
- Row.of(
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 100L,
- 41,
- "payload 1"),
- Row.of(
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 101L,
- 42,
- "payload 2"),
- Row.of(
- 3L,
- "name 3",
- LocalDateTime.parse("2020-03-10T13:12:11.123"),
- 102L,
- 43,
- "payload 3"));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "key_full_value_topic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- // compared to the partial value test we cannot support both k_user_id and user_id in a full
- // value due to duplicate names after key prefix stripping,
- // fields are reordered on purpose,
- // fields for keys and values are overlapping
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `user_id` BIGINT,\n"
- + " `name` STRING,\n"
- + " `timestamp` TIMESTAMP(3) METADATA,\n"
- + " `event_id` BIGINT,\n"
- + " `payload` STRING\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'key.format' = '%s',\n"
- + " 'key.fields' = 'event_id; user_id',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'ALL'\n"
- + ")",
- topic, bootstraps, groupId, format, format);
-
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO kafka\n"
- + "VALUES\n"
- + " (1, 'name 1', TIMESTAMP '2020-03-08 13:12:11.123', 100, 'payload 1'),\n"
- + " (2, 'name 2', TIMESTAMP '2020-03-09 13:12:11.123', 101, 'payload 2'),\n"
- + " (3, 'name 3', TIMESTAMP '2020-03-10 13:12:11.123', 102, 'payload 3')";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM kafka"), 3);
-
- final List<Row> expected =
- Arrays.asList(
- Row.of(
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 100L,
- "payload 1"),
- Row.of(
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 101L,
- "payload 2"),
- Row.of(
- 3L,
- "name 3",
- LocalDateTime.parse("2020-03-10T13:12:11.123"),
- 102L,
- "payload 3"));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaTemporalJoinChangelog() throws Exception {
- // Set the session time zone to UTC, because the next `METADATA FROM
- // 'value.source.timestamp'` DDL
- // will use the session time zone when convert the changelog time from milliseconds to
- // timestamp
- tEnv.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "UTC");
-
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String orderTopic = "temporal_join_topic_order_" + format + "_" + UUID.randomUUID();
- createTestTopic(orderTopic, 1, 1);
-
- final String productTopic =
- "temporal_join_topic_product_" + format + "_" + UUID.randomUUID();
- createTestTopic(productTopic, 1, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- // create order table and set initial values
- final String orderTableDDL =
- String.format(
- "CREATE TABLE ordersTable (\n"
- + " order_id STRING,\n"
- + " product_id STRING,\n"
- + " order_time TIMESTAMP(3),\n"
- + " quantity INT,\n"
- + " purchaser STRING,\n"
- + " WATERMARK FOR order_time AS order_time - INTERVAL '1' SECOND\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'format' = '%s'\n"
- + ")",
- orderTopic, bootstraps, groupId, format);
- tEnv.executeSql(orderTableDDL);
- String orderInitialValues =
- "INSERT INTO ordersTable\n"
- + "VALUES\n"
- + "('o_001', 'p_001', TIMESTAMP '2020-10-01 00:01:00', 1, 'Alice'),"
- + "('o_002', 'p_002', TIMESTAMP '2020-10-01 00:02:00', 1, 'Bob'),"
- + "('o_003', 'p_001', TIMESTAMP '2020-10-01 12:00:00', 2, 'Tom'),"
- + "('o_004', 'p_002', TIMESTAMP '2020-10-01 12:00:00', 2, 'King'),"
- + "('o_005', 'p_001', TIMESTAMP '2020-10-01 18:00:00', 10, 'Leonard'),"
- + "('o_006', 'p_002', TIMESTAMP '2020-10-01 18:00:00', 10, 'Leonard'),"
- + "('o_007', 'p_002', TIMESTAMP '2020-10-01 18:00:01', 10, 'Robinson')"; // used to advance watermark
- tEnv.executeSql(orderInitialValues).await();
-
- // create product table and set initial values
- final String productTableDDL =
- String.format(
- "CREATE TABLE productChangelogTable (\n"
- + " product_id STRING,\n"
- + " product_name STRING,\n"
- + " product_price DECIMAL(10, 4),\n"
- + " update_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL,\n"
- + " PRIMARY KEY(product_id) NOT ENFORCED,\n"
- + " WATERMARK FOR update_time AS update_time - INTERVAL '1' SECOND\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'value.format' = 'debezium-json'\n"
- + ")",
- productTopic, bootstraps, groupId);
- tEnv.executeSql(productTableDDL);
-
- // use raw format to initial the changelog data
- initialProductChangelog(productTopic, bootstraps);
-
- // ---------- query temporal join result from Kafka -------------------
- final List<String> result =
- collectRows(
- tEnv.sqlQuery(
- "SELECT"
- + " order_id,"
- + " order_time,"
- + " P.product_id,"
- + " P.update_time as product_update_time,"
- + " product_price,"
- + " purchaser,"
- + " product_name,"
- + " quantity,"
- + " quantity * product_price AS order_amount "
- + "FROM ordersTable AS O "
- + "LEFT JOIN productChangelogTable FOR SYSTEM_TIME AS OF O.order_time AS P "
- + "ON O.product_id = P.product_id"),
- 6)
- .stream()
- .map(row -> row.toString())
- .sorted()
- .collect(Collectors.toList());
-
- final List<String> expected =
- Arrays.asList(
- "+I[o_001, 2020-10-01T00:01, p_001, 1970-01-01T00:00, 11.1100, Alice, scooter, 1, 11.1100]",
- "+I[o_002, 2020-10-01T00:02, p_002, 1970-01-01T00:00, 23.1100, Bob, basketball, 1, 23.1100]",
- "+I[o_003, 2020-10-01T12:00, p_001, 2020-10-01T12:00, 12.9900, Tom, scooter, 2, 25.9800]",
- "+I[o_004, 2020-10-01T12:00, p_002, 2020-10-01T12:00, 19.9900, King, basketball, 2, 39.9800]",
- "+I[o_005, 2020-10-01T18:00, p_001, 2020-10-01T18:00, 11.9900, Leonard, scooter, 10, 119.9000]",
- "+I[o_006, 2020-10-01T18:00, null, null, null, Leonard, null, 10, null]");
-
- assertThat(result).isEqualTo(expected);
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(orderTopic);
- deleteTestTopic(productTopic);
- }
-
- private void initialProductChangelog(String topic, String bootstraps) throws Exception {
- String productChangelogDDL =
- String.format(
- "CREATE TABLE productChangelog (\n"
- + " changelog STRING"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'format' = 'raw'\n"
- + ")",
- topic, bootstraps);
- tEnv.executeSql(productChangelogDDL);
- String[] allChangelog = readLines("product_changelog.txt").toArray(new String[0]);
-
- StringBuilder insertSqlSb = new StringBuilder();
- insertSqlSb.append("INSERT INTO productChangelog VALUES ");
- for (String log : allChangelog) {
- insertSqlSb.append("('" + log + "'),");
- }
- // trim the last comma
- String insertSql = insertSqlSb.substring(0, insertSqlSb.toString().length() - 1);
- tEnv.executeSql(insertSql).await();
- }
-
- @Test
- public void testPerPartitionWatermarkKafka() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "per_partition_watermark_topic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 4, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
-
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `partition_id` INT,\n"
- + " `name` STRING,\n"
- + " `timestamp` TIMESTAMP(3),\n"
- + " WATERMARK FOR `timestamp` AS `timestamp`\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'sink.partitioner' = '%s',\n"
- + " 'format' = '%s'\n"
- + ")",
- topic, bootstraps, groupId, TestPartitioner.class.getName(), format);
-
- tEnv.executeSql(createTable);
-
- // make every partition have more than one record
- String initialValues =
- "INSERT INTO kafka\n"
- + "VALUES\n"
- + " (0, 'partition-0-name-0', TIMESTAMP '2020-03-08 13:12:11.123'),\n"
- + " (0, 'partition-0-name-1', TIMESTAMP '2020-03-08 14:12:12.223'),\n"
- + " (0, 'partition-0-name-2', TIMESTAMP '2020-03-08 15:12:13.323'),\n"
- + " (1, 'partition-1-name-0', TIMESTAMP '2020-03-09 13:13:11.123'),\n"
- + " (1, 'partition-1-name-1', TIMESTAMP '2020-03-09 15:13:11.133'),\n"
- + " (1, 'partition-1-name-2', TIMESTAMP '2020-03-09 16:13:11.143'),\n"
- + " (2, 'partition-2-name-0', TIMESTAMP '2020-03-10 13:12:14.123'),\n"
- + " (2, 'partition-2-name-1', TIMESTAMP '2020-03-10 14:12:14.123'),\n"
- + " (2, 'partition-2-name-2', TIMESTAMP '2020-03-10 14:13:14.123'),\n"
- + " (2, 'partition-2-name-3', TIMESTAMP '2020-03-10 14:14:14.123'),\n"
- + " (2, 'partition-2-name-4', TIMESTAMP '2020-03-10 14:15:14.123'),\n"
- + " (2, 'partition-2-name-5', TIMESTAMP '2020-03-10 14:16:14.123'),\n"
- + " (3, 'partition-3-name-0', TIMESTAMP '2020-03-11 17:12:11.123'),\n"
- + " (3, 'partition-3-name-1', TIMESTAMP '2020-03-11 18:12:11.123')";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- env.setParallelism(1);
- String createSink =
- "CREATE TABLE MySink(\n"
- + " id INT,\n"
- + " name STRING,\n"
- + " ts TIMESTAMP(3),\n"
- + " WATERMARK FOR ts as ts\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'sink.drop-late-event' = 'true'\n"
- + ")";
- tEnv.executeSql(createSink);
- TableResult tableResult = tEnv.executeSql("INSERT INTO MySink SELECT * FROM kafka");
- final List<String> expected =
- Arrays.asList(
- "+I[0, partition-0-name-0, 2020-03-08T13:12:11.123]",
- "+I[0, partition-0-name-1, 2020-03-08T14:12:12.223]",
- "+I[0, partition-0-name-2, 2020-03-08T15:12:13.323]",
- "+I[1, partition-1-name-0, 2020-03-09T13:13:11.123]",
- "+I[1, partition-1-name-1, 2020-03-09T15:13:11.133]",
- "+I[1, partition-1-name-2, 2020-03-09T16:13:11.143]",
- "+I[2, partition-2-name-0, 2020-03-10T13:12:14.123]",
- "+I[2, partition-2-name-1, 2020-03-10T14:12:14.123]",
- "+I[2, partition-2-name-2, 2020-03-10T14:13:14.123]",
- "+I[2, partition-2-name-3, 2020-03-10T14:14:14.123]",
- "+I[2, partition-2-name-4, 2020-03-10T14:15:14.123]",
- "+I[2, partition-2-name-5, 2020-03-10T14:16:14.123]",
- "+I[3, partition-3-name-0, 2020-03-11T17:12:11.123]",
- "+I[3, partition-3-name-1, 2020-03-11T18:12:11.123]");
- KafkaTableTestUtils.waitingExpectedResults("MySink", expected, Duration.ofSeconds(5));
-
- // ------------- cleanup -------------------
-
- tableResult.getJobClient().ifPresent(JobClient::cancel);
- deleteTestTopic(topic);
- }
-
- @Test
- public void testPerPartitionWatermarkWithIdleSource() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "idle_partition_watermark_topic_" + format + "_" + UUID.randomUUID();
- createTestTopic(topic, 4, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String groupId = getStandardProps().getProperty("group.id");
- String bootstraps = getBootstrapServers();
- tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
-
- final String createTable =
- String.format(
- "CREATE TABLE kafka (\n"
- + " `partition_id` INT,\n"
- + " `value` INT,\n"
- + " `timestamp` TIMESTAMP(3),\n"
- + " WATERMARK FOR `timestamp` AS `timestamp`\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'sink.partitioner' = '%s',\n"
- + " 'format' = '%s'\n"
- + ")",
- topic, bootstraps, groupId, TestPartitioner.class.getName(), format);
-
- tEnv.executeSql(createTable);
-
- // Only two partitions have elements and others are idle.
- // When idle timer triggers, the WatermarkOutputMultiplexer will use the minimum watermark
- // among active partitions as the output watermark.
- // Therefore, we need to make sure the watermark in the each partition is large enough to
- // trigger the window.
- String initialValues =
- "INSERT INTO kafka\n"
- + "VALUES\n"
- + " (0, 0, TIMESTAMP '2020-03-08 13:12:11.123'),\n"
- + " (0, 1, TIMESTAMP '2020-03-08 13:15:12.223'),\n"
- + " (0, 2, TIMESTAMP '2020-03-08 16:12:13.323'),\n"
- + " (1, 3, TIMESTAMP '2020-03-08 13:13:11.123'),\n"
- + " (1, 4, TIMESTAMP '2020-03-08 13:19:11.133'),\n"
- + " (1, 5, TIMESTAMP '2020-03-08 16:13:11.143')\n";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- env.setParallelism(1);
- String createSink =
- "CREATE TABLE MySink(\n"
- + " `id` INT,\n"
- + " `cnt` BIGINT\n"
- + ") WITH (\n"
- + " 'connector' = 'values'\n"
- + ")";
- tEnv.executeSql(createSink);
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO MySink\n"
- + "SELECT `partition_id` as `id`, COUNT(`value`) as `cnt`\n"
- + "FROM kafka\n"
- + "GROUP BY `partition_id`, TUMBLE(`timestamp`, INTERVAL '1' HOUR) ");
-
- final List<String> expected = Arrays.asList("+I[0, 2]", "+I[1, 2]");
- KafkaTableTestUtils.waitingExpectedResults("MySink", expected, Duration.ofSeconds(5));
-
- // ------------- cleanup -------------------
-
- tableResult.getJobClient().ifPresent(JobClient::cancel);
- deleteTestTopic(topic);
- }
-
- @Test
- public void testStartFromGroupOffsetsLatest() throws Exception {
- testStartFromGroupOffsets("latest");
- }
-
- @Test
- public void testStartFromGroupOffsetsEarliest() throws Exception {
- testStartFromGroupOffsets("earliest");
- }
-
- @Test
- public void testStartFromGroupOffsetsNone() {
- Assertions.assertThatThrownBy(() -> testStartFromGroupOffsetsWithNoneResetStrategy())
- .satisfies(FlinkAssertions.anyCauseMatches(NoOffsetForPartitionException.class));
- }
-
- private List<String> appendNewData(
- String topic, String tableName, String groupId, int targetNum) throws Exception {
- waitUtil(
- () -> {
- Map<TopicPartition, OffsetAndMetadata> offsets = getConsumerOffset(groupId);
- long sum =
- offsets.entrySet().stream()
- .filter(e -> e.getKey().topic().contains(topic))
- .mapToLong(e -> e.getValue().offset())
- .sum();
- return sum == targetNum;
- },
- Duration.ofMillis(20000),
- "Can not reach the expected offset before adding new data.");
- String appendValues =
- "INSERT INTO "
- + tableName
- + "\n"
- + "VALUES\n"
- + " (2, 6),\n"
- + " (2, 7),\n"
- + " (2, 8)\n";
- tEnv.executeSql(appendValues).await();
- return Arrays.asList("+I[2, 6]", "+I[2, 7]", "+I[2, 8]");
- }
-
- private TableResult startFromGroupOffset(
- String tableName, String topic, String groupId, String resetStrategy, String sinkName)
- throws ExecutionException, InterruptedException {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- createTestTopic(topic, 4, 1);
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
- tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
-
- final String createTableSql =
- "CREATE TABLE %s (\n"
- + " `partition_id` INT,\n"
- + " `value` INT\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'properties.group.id' = '%s',\n"
- + " 'scan.startup.mode' = 'group-offsets',\n"
- + " 'properties.auto.offset.reset' = '%s',\n"
- + " 'properties.enable.auto.commit' = 'true',\n"
- + " 'properties.auto.commit.interval.ms' = '1000',\n"
- + " 'format' = '%s'\n"
- + ")";
- tEnv.executeSql(
- String.format(
- createTableSql,
- tableName,
- topic,
- bootstraps,
- groupId,
- resetStrategy,
- format));
-
- String initialValues =
- "INSERT INTO "
- + tableName
- + "\n"
- + "VALUES\n"
- + " (0, 0),\n"
- + " (0, 1),\n"
- + " (0, 2),\n"
- + " (1, 3),\n"
- + " (1, 4),\n"
- + " (1, 5)\n";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- env.setParallelism(1);
- String createSink =
- "CREATE TABLE "
- + sinkName
- + "(\n"
- + " `partition_id` INT,\n"
- + " `value` INT\n"
- + ") WITH (\n"
- + " 'connector' = 'values'\n"
- + ")";
- tEnv.executeSql(createSink);
-
- return tEnv.executeSql("INSERT INTO " + sinkName + " SELECT * FROM " + tableName);
- }
-
- private void testStartFromGroupOffsets(String resetStrategy) throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String tableName = "Table" + format + resetStrategy;
- final String topic =
- "groupOffset_" + format + resetStrategy + ThreadLocalRandom.current().nextLong();
- String groupId = format + resetStrategy;
- String sinkName = "mySink" + format + resetStrategy;
- List<String> expected =
- Arrays.asList(
- "+I[0, 0]", "+I[0, 1]", "+I[0, 2]", "+I[1, 3]", "+I[1, 4]", "+I[1, 5]");
-
- TableResult tableResult = null;
- try {
- tableResult = startFromGroupOffset(tableName, topic, groupId, resetStrategy, sinkName);
- if ("latest".equals(resetStrategy)) {
- expected = appendNewData(topic, tableName, groupId, expected.size());
- }
- KafkaTableTestUtils.waitingExpectedResults(sinkName, expected, Duration.ofSeconds(15));
- } finally {
- // ------------- cleanup -------------------
- if (tableResult != null) {
- tableResult.getJobClient().ifPresent(JobClient::cancel);
- }
- deleteTestTopic(topic);
- }
- }
-
- private void testStartFromGroupOffsetsWithNoneResetStrategy()
- throws ExecutionException, InterruptedException {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String resetStrategy = "none";
- final String tableName = resetStrategy + "Table";
- final String topic = "groupOffset_" + format + "_" + UUID.randomUUID();
- String groupId = resetStrategy + (new Random()).nextInt();
-
- TableResult tableResult = null;
- try {
- tableResult = startFromGroupOffset(tableName, topic, groupId, resetStrategy, "MySink");
- tableResult.await();
- } finally {
- // ------------- cleanup -------------------
- if (tableResult != null) {
- tableResult.getJobClient().ifPresent(JobClient::cancel);
- }
- deleteTestTopic(topic);
- }
- }
-
- // --------------------------------------------------------------------------------------------
- // Utilities
- // --------------------------------------------------------------------------------------------
-
- /** Extract the partition id from the row and set it on the record. */
- public static class TestPartitioner extends FlinkKafkaPartitioner<RowData> {
-
- private static final long serialVersionUID = 1L;
- private static final int PARTITION_ID_FIELD_IN_SCHEMA = 0;
-
- @Override
- public int partition(
- RowData record, byte[] key, byte[] value, String targetTopic, int[] partitions) {
- return partitions[record.getInt(PARTITION_ID_FIELD_IN_SCHEMA) % partitions.length];
- }
- }
-
- private String formatOptions() {
- return String.format("'format' = '%s'", format);
- }
-
- private static final class TestingSinkFunction implements SinkFunction<RowData> {
-
- private static final long serialVersionUID = 455430015321124493L;
- private static List<String> rows = new ArrayList<>();
-
- private final int expectedSize;
-
- private TestingSinkFunction(int expectedSize) {
- this.expectedSize = expectedSize;
- rows.clear();
- }
-
- @Override
- public void invoke(RowData value, Context context) {
- rows.add(value.toString());
- if (rows.size() >= expectedSize) {
- // job finish
- throw new SuccessException();
- }
- }
- }
-
- private static boolean isCausedByJobFinished(Throwable e) {
- if (e instanceof SuccessException) {
- return true;
- } else if (e.getCause() != null) {
- return isCausedByJobFinished(e.getCause());
- } else {
- return false;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java
deleted file mode 100644
index f80a54f..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.util.DockerImageVersions;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.admin.TopicDescription;
-import org.apache.kafka.clients.admin.TopicListing;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.output.Slf4jLogConsumer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.stream.Collectors;
-
-/** Base class for Kafka Table IT Cases. */
-public abstract class KafkaTableTestBase extends AbstractTestBase {
-
- private static final Logger LOG = LoggerFactory.getLogger(KafkaTableTestBase.class);
-
- private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- private static final int zkTimeoutMills = 30000;
-
- @ClassRule
- public static final KafkaContainer KAFKA_CONTAINER =
- new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) {
- @Override
- protected void doStart() {
- super.doStart();
- if (LOG.isInfoEnabled()) {
- this.followOutput(new Slf4jLogConsumer(LOG));
- }
- }
- }.withEmbeddedZookeeper()
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS)
- .withEnv(
- "KAFKA_TRANSACTION_MAX_TIMEOUT_MS",
- String.valueOf(Duration.ofHours(2).toMillis()))
- // Disable log deletion to prevent records from being deleted during test run
- .withEnv("KAFKA_LOG_RETENTION_MS", "-1");
-
- protected StreamExecutionEnvironment env;
- protected StreamTableEnvironment tEnv;
-
- // Timer for scheduling logging task if the test hangs
- private final Timer loggingTimer = new Timer("Debug Logging Timer");
-
- @Before
- public void setup() {
- env = StreamExecutionEnvironment.getExecutionEnvironment();
- tEnv = StreamTableEnvironment.create(env);
- env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-
- // Probe Kafka broker status per 30 seconds
- scheduleTimeoutLogger(
- Duration.ofSeconds(30),
- () -> {
- // List all non-internal topics
- final Map<String, TopicDescription> topicDescriptions =
- describeExternalTopics();
- LOG.info("Current existing topics: {}", topicDescriptions.keySet());
-
- // Log status of topics
- logTopicPartitionStatus(topicDescriptions);
- });
- }
-
- @After
- public void after() {
- // Cancel timer for debug logging
- cancelTimeoutLogger();
- }
-
- public Properties getStandardProps() {
- Properties standardProps = new Properties();
- standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers());
- standardProps.put("group.id", "flink-tests");
- standardProps.put("enable.auto.commit", false);
- standardProps.put("auto.offset.reset", "earliest");
- standardProps.put("max.partition.fetch.bytes", 256);
- standardProps.put("zookeeper.session.timeout.ms", zkTimeoutMills);
- standardProps.put("zookeeper.connection.timeout.ms", zkTimeoutMills);
- return standardProps;
- }
-
- public String getBootstrapServers() {
- return KAFKA_CONTAINER.getBootstrapServers();
- }
-
- public void createTestTopic(String topic, int numPartitions, int replicationFactor) {
- Map<String, Object> properties = new HashMap<>();
- properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, getBootstrapServers());
- try (AdminClient admin = AdminClient.create(properties)) {
- admin.createTopics(
- Collections.singletonList(
- new NewTopic(topic, numPartitions, (short) replicationFactor)))
- .all()
- .get();
- } catch (Exception e) {
- throw new IllegalStateException(
- String.format(
- "Fail to create topic [%s partitions: %d replication factor: %d].",
- topic, numPartitions, replicationFactor),
- e);
- }
- }
-
- public Map<TopicPartition, OffsetAndMetadata> getConsumerOffset(String groupId) {
- Map<String, Object> properties = new HashMap<>();
- properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, getBootstrapServers());
- try (AdminClient admin = AdminClient.create(properties)) {
- ListConsumerGroupOffsetsResult result = admin.listConsumerGroupOffsets(groupId);
- return result.partitionsToOffsetAndMetadata().get();
- } catch (Exception e) {
- throw new IllegalStateException(
- String.format("Fail to get consumer offsets with the group id [%s].", groupId),
- e);
- }
- }
-
- public void deleteTestTopic(String topic) {
- Map<String, Object> properties = new HashMap<>();
- properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, getBootstrapServers());
- try (AdminClient admin = AdminClient.create(properties)) {
- admin.deleteTopics(Collections.singletonList(topic)).all().get();
- } catch (Exception e) {
- throw new IllegalStateException(String.format("Fail to delete topic [%s].", topic), e);
- }
- }
-
- // ------------------------ For Debug Logging Purpose ----------------------------------
-
- private void scheduleTimeoutLogger(Duration period, Runnable loggingAction) {
- TimerTask timeoutLoggerTask =
- new TimerTask() {
- @Override
- public void run() {
- try {
- loggingAction.run();
- } catch (Exception e) {
- throw new RuntimeException("Failed to execute logging action", e);
- }
- }
- };
- loggingTimer.schedule(timeoutLoggerTask, 0L, period.toMillis());
- }
-
- private void cancelTimeoutLogger() {
- loggingTimer.cancel();
- }
-
- private Map<String, TopicDescription> describeExternalTopics() {
- try (final AdminClient adminClient = AdminClient.create(getStandardProps())) {
- final List<String> topics =
- adminClient.listTopics().listings().get().stream()
- .filter(listing -> !listing.isInternal())
- .map(TopicListing::name)
- .collect(Collectors.toList());
-
- return adminClient.describeTopics(topics).allTopicNames().get();
- } catch (Exception e) {
- throw new RuntimeException("Failed to list Kafka topics", e);
- }
- }
-
- private void logTopicPartitionStatus(Map<String, TopicDescription> topicDescriptions) {
- final Properties properties = getStandardProps();
- properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "flink-tests-debugging");
- properties.setProperty(
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
- StringDeserializer.class.getCanonicalName());
- properties.setProperty(
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
- StringDeserializer.class.getCanonicalName());
- final KafkaConsumer<?, ?> consumer = new KafkaConsumer<String, String>(properties);
- List<TopicPartition> partitions = new ArrayList<>();
- topicDescriptions.forEach(
- (topic, description) ->
- description
- .partitions()
- .forEach(
- tpInfo ->
- partitions.add(
- new TopicPartition(
- topic, tpInfo.partition()))));
- final Map<TopicPartition, Long> beginningOffsets = consumer.beginningOffsets(partitions);
- final Map<TopicPartition, Long> endOffsets = consumer.endOffsets(partitions);
- partitions.forEach(
- partition ->
- LOG.info(
- "TopicPartition \"{}\": starting offset: {}, stopping offset: {}",
- partition,
- beginningOffsets.get(partition),
- endOffsets.get(partition)));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java
deleted file mode 100644
index c1aada4..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.table.utils.TableTestMatchers;
-import org.apache.flink.types.Row;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.CloseableIterator;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.HamcrestCondition.matching;
-
-/** Utils for kafka table tests. */
-public class KafkaTableTestUtils {
- public static List<Row> collectRows(Table table, int expectedSize) throws Exception {
- final TableResult result = table.execute();
- final List<Row> collectedRows = new ArrayList<>();
- try (CloseableIterator<Row> iterator = result.collect()) {
- while (collectedRows.size() < expectedSize && iterator.hasNext()) {
- collectedRows.add(iterator.next());
- }
- }
- result.getJobClient()
- .ifPresent(
- jc -> {
- try {
- jc.cancel().get(5, TimeUnit.SECONDS);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- });
-
- return collectedRows;
- }
-
- public static List<String> readLines(String resource) throws IOException {
- final URL url = KafkaChangelogTableITCase.class.getClassLoader().getResource(resource);
- assertThat(url).isNotNull();
- Path path = new File(url.getFile()).toPath();
- return Files.readAllLines(path);
- }
-
- public static void waitingExpectedResults(
- String sinkName, List<String> expected, Duration timeout)
- throws InterruptedException, TimeoutException {
- Collections.sort(expected);
- CommonTestUtils.waitUtil(
- () -> {
- List<String> actual = TestValuesTableFactory.getResults(sinkName);
- Collections.sort(actual);
- return expected.equals(actual);
- },
- timeout,
- "Can not get the expected result.");
- }
-
- public static void comparedWithKeyAndOrder(
- Map<Row, List<Row>> expectedData, List<Row> actual, int[] keyLoc) {
- Map<Row, LinkedList<Row>> actualData = new HashMap<>();
- for (Row row : actual) {
- Row key = Row.project(row, keyLoc);
- // ignore row kind
- key.setKind(RowKind.INSERT);
- actualData.computeIfAbsent(key, k -> new LinkedList<>()).add(row);
- }
- // compare key first
- assertThat(actualData).as("Actual result: " + actual).hasSameSizeAs(expectedData);
- // compare by value
- for (Row key : expectedData.keySet()) {
- assertThat(actualData.get(key))
- .satisfies(
- matching(TableTestMatchers.deepEqualTo(expectedData.get(key), false)));
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java
deleted file mode 100644
index b6f2788..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.operators.ProcessingTimeService;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.sink2.StatefulSink;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
-import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ScheduledFuture;
-
-import static org.apache.flink.types.RowKind.DELETE;
-import static org.apache.flink.types.RowKind.INSERT;
-import static org.apache.flink.types.RowKind.UPDATE_AFTER;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Tests for {@link ReducingUpsertWriter}. */
-@RunWith(Parameterized.class)
-public class ReducingUpsertWriterTest {
- @Parameterized.Parameters(name = "object reuse = {0}")
- public static Object[] enableObjectReuse() {
- return new Boolean[] {true, false};
- }
-
- private static final ResolvedSchema SCHEMA =
- ResolvedSchema.of(
- Column.physical("id", DataTypes.INT().notNull()),
- Column.physical("title", DataTypes.STRING().notNull()),
- Column.physical("author", DataTypes.STRING()),
- Column.physical("price", DataTypes.DOUBLE()),
- Column.physical("qty", DataTypes.INT()),
- Column.physical("ts", DataTypes.TIMESTAMP_LTZ(3)));
-
- private static final int keyIndices = 0;
- private static final int TIMESTAMP_INDICES = 5;
- private static final SinkBufferFlushMode BUFFER_FLUSH_MODE =
- new SinkBufferFlushMode(4, Long.MAX_VALUE);
-
- public static final RowData[] TEST_DATA = {
- GenericRowData.ofKind(
- INSERT,
- 1001,
- StringData.fromString("Java public for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 11.11,
- 11,
- TimestampData.fromInstant(Instant.parse("2021-03-30T15:00:00Z"))),
- GenericRowData.ofKind(
- INSERT,
- 1002,
- StringData.fromString("More Java for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 22.22,
- 22,
- TimestampData.fromInstant(Instant.parse("2021-03-30T16:00:00Z"))),
- GenericRowData.ofKind(
- INSERT,
- 1004,
- StringData.fromString("A Cup of Java"),
- StringData.fromString("Kumar"),
- 44.44,
- 44,
- TimestampData.fromInstant(Instant.parse("2021-03-30T17:00:00Z"))),
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1004,
- StringData.fromString("A Teaspoon of Java"),
- StringData.fromString("Kevin Jones"),
- 55.55,
- 55,
- TimestampData.fromInstant(Instant.parse("2021-03-30T18:00:00Z"))),
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1004,
- StringData.fromString("A Teaspoon of Java 1.4"),
- StringData.fromString("Kevin Jones"),
- 66.66,
- 66,
- TimestampData.fromInstant(Instant.parse("2021-03-30T19:00:00Z"))),
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1004,
- StringData.fromString("A Teaspoon of Java 1.5"),
- StringData.fromString("Kevin Jones"),
- 77.77,
- 77,
- TimestampData.fromInstant(Instant.parse("2021-03-30T20:00:00Z"))),
- GenericRowData.ofKind(
- DELETE,
- 1004,
- StringData.fromString("A Teaspoon of Java 1.8"),
- StringData.fromString("Kevin Jones"),
- null,
- 1010,
- TimestampData.fromInstant(Instant.parse("2021-03-30T21:00:00Z"))),
- GenericRowData.ofKind(
- DELETE,
- 1005,
- StringData.fromString("A Teaspoon of Java 1.8"),
- StringData.fromString("Kevin Jones"),
- null,
- 1010,
- TimestampData.fromInstant(Instant.parse("2021-03-30T21:00:00Z")))
- };
-
- private final boolean enableObjectReuse;
-
- public ReducingUpsertWriterTest(boolean enableObjectReuse) {
- this.enableObjectReuse = enableObjectReuse;
- }
-
- @Test
- public void testWriteData() throws Exception {
- final MockedSinkWriter writer = new MockedSinkWriter();
- final ReducingUpsertWriter<?> bufferedWriter = createBufferedWriter(writer);
-
- // write 4 records which doesn't trigger batch size
- writeData(bufferedWriter, new ReusableIterator(0, 4));
- assertThat(writer.rowDataCollectors).isEmpty();
-
- // write one more record, and should flush the buffer
- writeData(bufferedWriter, new ReusableIterator(7, 1));
-
- HashMap<Integer, List<RowData>> expected = new HashMap<>();
- expected.put(
- 1001,
- Collections.singletonList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1001,
- StringData.fromString("Java public for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 11.11,
- 11,
- TimestampData.fromInstant(Instant.parse("2021-03-30T15:00:00Z")))));
- expected.put(
- 1002,
- Collections.singletonList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1002,
- StringData.fromString("More Java for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 22.22,
- 22,
- TimestampData.fromInstant(Instant.parse("2021-03-30T16:00:00Z")))));
- expected.put(
- 1004,
- Collections.singletonList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1004,
- StringData.fromString("A Teaspoon of Java"),
- StringData.fromString("Kevin Jones"),
- 55.55,
- 55,
- TimestampData.fromInstant(Instant.parse("2021-03-30T18:00:00Z")))));
-
- expected.put(
- 1005,
- Collections.singletonList(
- GenericRowData.ofKind(
- DELETE,
- 1005,
- StringData.fromString("A Teaspoon of Java 1.8"),
- StringData.fromString("Kevin Jones"),
- null,
- 1010,
- TimestampData.fromInstant(Instant.parse("2021-03-30T21:00:00Z")))));
-
- compareCompactedResult(expected, writer.rowDataCollectors);
-
- writer.rowDataCollectors.clear();
- // write remaining data, and they are still buffered
- writeData(bufferedWriter, new ReusableIterator(4, 3));
- assertThat(writer.rowDataCollectors).isEmpty();
- }
-
- @Test
- public void testFlushDataWhenCheckpointing() throws Exception {
- final MockedSinkWriter writer = new MockedSinkWriter();
- final ReducingUpsertWriter<?> bufferedWriter = createBufferedWriter(writer);
- // write all data, there should be 3 records are still buffered
- writeData(bufferedWriter, new ReusableIterator(0, 4));
- // snapshot should flush the buffer
- bufferedWriter.flush(true);
-
- HashMap<Integer, List<RowData>> expected = new HashMap<>();
- expected.put(
- 1001,
- Collections.singletonList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1001,
- StringData.fromString("Java public for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 11.11,
- 11,
- TimestampData.fromInstant(Instant.parse("2021-03-30T15:00:00Z")))));
- expected.put(
- 1002,
- Collections.singletonList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1002,
- StringData.fromString("More Java for dummies"),
- StringData.fromString("Tan Ah Teck"),
- 22.22,
- 22,
- TimestampData.fromInstant(Instant.parse("2021-03-30T16:00:00Z")))));
- expected.put(
- 1004,
- Arrays.asList(
- GenericRowData.ofKind(
- UPDATE_AFTER,
- 1004,
- StringData.fromString("A Teaspoon of Java"),
- StringData.fromString("Kevin Jones"),
- 55.55,
- 55,
- TimestampData.fromInstant(Instant.parse("2021-03-30T18:00:00Z")))));
-
- compareCompactedResult(expected, writer.rowDataCollectors);
- }
-
- private void compareCompactedResult(
- Map<Integer, List<RowData>> expected, List<RowData> actual) {
- Map<Integer, List<RowData>> actualMap = new HashMap<>();
-
- for (RowData rowData : actual) {
- Integer id = rowData.getInt(keyIndices);
- actualMap.computeIfAbsent(id, key -> new ArrayList<>()).add(rowData);
- }
-
- assertThat(actualMap).hasSameSizeAs(expected);
- for (Integer id : expected.keySet()) {
- assertThat(actualMap.get(id)).isEqualTo(expected.get(id));
- }
- }
-
- private void writeData(ReducingUpsertWriter<?> writer, Iterator<RowData> iterator)
- throws Exception {
- while (iterator.hasNext()) {
- RowData next = iterator.next();
- long rowtime = next.getTimestamp(TIMESTAMP_INDICES, 3).getMillisecond();
- writer.write(
- next,
- new org.apache.flink.api.connector.sink2.SinkWriter.Context() {
- @Override
- public long currentWatermark() {
- throw new UnsupportedOperationException("Not implemented.");
- }
-
- @Override
- public Long timestamp() {
- return rowtime;
- }
- });
- }
- }
-
- @SuppressWarnings("unchecked")
- private ReducingUpsertWriter<?> createBufferedWriter(MockedSinkWriter sinkWriter) {
- TypeInformation<RowData> typeInformation =
- (TypeInformation<RowData>)
- new SinkRuntimeProviderContext(false)
- .createTypeInformation(SCHEMA.toPhysicalRowDataType());
- return new ReducingUpsertWriter<>(
- sinkWriter,
- SCHEMA.toPhysicalRowDataType(),
- new int[] {keyIndices},
- BUFFER_FLUSH_MODE,
- new ProcessingTimeService() {
- @Override
- public long getCurrentProcessingTime() {
- return 0;
- }
-
- @Override
- public ScheduledFuture<?> registerTimer(
- long timestamp, ProcessingTimeCallback target) {
- return null;
- }
- },
- enableObjectReuse
- ? typeInformation.createSerializer(new ExecutionConfig())::copy
- : r -> r);
- }
-
- private static class MockedSinkWriter
- implements StatefulSink.StatefulSinkWriter<RowData, Void> {
-
- transient List<RowData> rowDataCollectors;
-
- MockedSinkWriter() {
- rowDataCollectors = new ArrayList<>();
- }
-
- @Override
- public void write(RowData element, Context context)
- throws IOException, InterruptedException {
- assertThat(Instant.ofEpochMilli(context.timestamp()))
- .isEqualTo(element.getTimestamp(TIMESTAMP_INDICES, 3).toInstant());
- rowDataCollectors.add(element);
- }
-
- @Override
- public void flush(boolean endOfInput) throws IOException, InterruptedException {}
-
- @Override
- public void close() throws Exception {}
-
- @Override
- public List<Void> snapshotState(long checkpointId) throws IOException {
- return null;
- }
- }
-
- private class ReusableIterator implements Iterator<RowData> {
-
- private final RowDataSerializer serializer =
- InternalTypeInfo.of(SCHEMA.toSinkRowDataType().getLogicalType()).toRowSerializer();
- private final RowData reusedRow = new GenericRowData(SCHEMA.getColumnCount());
-
- private int begin;
- private final int end;
-
- ReusableIterator(int begin, int size) {
- this.begin = begin;
- this.end = begin + size;
- }
-
- @Override
- public boolean hasNext() {
- return begin < end;
- }
-
- @Override
- public RowData next() {
- if (enableObjectReuse) {
- return serializer.copy(TEST_DATA[begin++], reusedRow);
- } else {
- return TEST_DATA[begin++];
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java
deleted file mode 100644
index 5caaaa0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java
+++ /dev/null
@@ -1,666 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.connector.sink2.Sink;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.connector.base.DeliveryGuarantee;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-import org.apache.flink.connector.kafka.source.KafkaSource;
-import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState;
-import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
-import org.apache.flink.formats.avro.AvroRowDataSerializationSchema;
-import org.apache.flink.formats.avro.RowDataToAvroConverters;
-import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema;
-import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-import org.apache.flink.streaming.connectors.kafka.config.BoundedMode;
-import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
-import org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperatorFactory;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.catalog.UniqueConstraint;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.format.DecodingFormat;
-import org.apache.flink.table.connector.format.EncodingFormat;
-import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkV2Provider;
-import org.apache.flink.table.connector.source.DataStreamScanProvider;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.ScanTableSource;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.binary.BinaryRowData;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.factories.TestFormatFactory;
-import org.apache.flink.table.factories.utils.FactoryMocks;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
-import org.apache.flink.table.types.AtomicDataType;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.function.Consumer;
-
-import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link UpsertKafkaDynamicTableFactory}. */
-public class UpsertKafkaDynamicTableFactoryTest extends TestLogger {
-
- private static final String SOURCE_TOPIC = "sourceTopic_1";
-
- private static final String SINK_TOPIC = "sinkTopic";
-
- private static final String TEST_REGISTRY_URL = "http://localhost:8081";
- private static final String DEFAULT_VALUE_SUBJECT = SINK_TOPIC + "-value";
- private static final String DEFAULT_KEY_SUBJECT = SINK_TOPIC + "-key";
-
- private static final ResolvedSchema SOURCE_SCHEMA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical("window_start", DataTypes.STRING().notNull()),
- Column.physical("region", DataTypes.STRING().notNull()),
- Column.physical("view_count", DataTypes.BIGINT())),
- Collections.emptyList(),
- UniqueConstraint.primaryKey("name", Arrays.asList("window_start", "region")));
-
- private static final int[] SOURCE_KEY_FIELDS = new int[] {0, 1};
-
- private static final int[] SOURCE_VALUE_FIELDS = new int[] {0, 1, 2};
-
- private static final ResolvedSchema SINK_SCHEMA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical(
- "region", new AtomicDataType(new VarCharType(false, 100))),
- Column.physical("view_count", DataTypes.BIGINT())),
- Collections.emptyList(),
- UniqueConstraint.primaryKey("name", Collections.singletonList("region")));
-
- private static final int[] SINK_KEY_FIELDS = new int[] {0};
-
- private static final int[] SINK_VALUE_FIELDS = new int[] {0, 1};
-
- private static final Properties UPSERT_KAFKA_SOURCE_PROPERTIES = new Properties();
- private static final Properties UPSERT_KAFKA_SINK_PROPERTIES = new Properties();
-
- static {
- UPSERT_KAFKA_SOURCE_PROPERTIES.setProperty("bootstrap.servers", "dummy");
-
- UPSERT_KAFKA_SINK_PROPERTIES.setProperty("bootstrap.servers", "dummy");
- }
-
- static EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat =
- new TestFormatFactory.EncodingFormatMock(",", ChangelogMode.insertOnly());
- static EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat =
- new TestFormatFactory.EncodingFormatMock(",", ChangelogMode.insertOnly());
-
- static DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat =
- new TestFormatFactory.DecodingFormatMock(
- ",", true, ChangelogMode.insertOnly(), Collections.emptyMap());
- static DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat =
- new TestFormatFactory.DecodingFormatMock(
- ",", true, ChangelogMode.insertOnly(), Collections.emptyMap());
-
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- @Test
- public void testTableSource() {
- final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType();
- // Construct table source using options and table source factory
- final DynamicTableSource actualSource =
- createTableSource(SOURCE_SCHEMA, getFullSourceOptions());
-
- final KafkaDynamicSource expectedSource =
- createExpectedScanSource(
- producedDataType,
- keyDecodingFormat,
- valueDecodingFormat,
- SOURCE_KEY_FIELDS,
- SOURCE_VALUE_FIELDS,
- null,
- SOURCE_TOPIC,
- UPSERT_KAFKA_SOURCE_PROPERTIES);
- assertThat(actualSource).isEqualTo(expectedSource);
-
- final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource;
- ScanTableSource.ScanRuntimeProvider provider =
- actualUpsertKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
- assertKafkaSource(provider);
- }
-
- @Test
- public void testTableSink() {
- // Construct table sink using options and table sink factory.
- final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, getFullSinkOptions());
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SINK_SCHEMA.toPhysicalRowDataType(),
- keyEncodingFormat,
- valueEncodingFormat,
- SINK_KEY_FIELDS,
- SINK_VALUE_FIELDS,
- null,
- SINK_TOPIC,
- UPSERT_KAFKA_SINK_PROPERTIES,
- DeliveryGuarantee.AT_LEAST_ONCE,
- SinkBufferFlushMode.DISABLED,
- null);
-
- // Test sink format.
- final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink;
- assertThat(actualSink).isEqualTo(expectedSink);
-
- // Test kafka producer.
- DynamicTableSink.SinkRuntimeProvider provider =
- actualUpsertKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
- assertThat(provider).isInstanceOf(SinkV2Provider.class);
- final SinkV2Provider sinkFunctionProvider = (SinkV2Provider) provider;
- final Sink<RowData> sink = sinkFunctionProvider.createSink();
- assertThat(sink).isInstanceOf(KafkaSink.class);
- }
-
- @SuppressWarnings("rawtypes")
- @Test
- public void testBufferedTableSink() {
- // Construct table sink using options and table sink factory.
- final DynamicTableSink actualSink =
- createTableSink(
- SINK_SCHEMA,
- getModifiedOptions(
- getFullSinkOptions(),
- options -> {
- options.put("sink.buffer-flush.max-rows", "100");
- options.put("sink.buffer-flush.interval", "1s");
- }));
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SINK_SCHEMA.toPhysicalRowDataType(),
- keyEncodingFormat,
- valueEncodingFormat,
- SINK_KEY_FIELDS,
- SINK_VALUE_FIELDS,
- null,
- SINK_TOPIC,
- UPSERT_KAFKA_SINK_PROPERTIES,
- DeliveryGuarantee.AT_LEAST_ONCE,
- new SinkBufferFlushMode(100, 1000L),
- null);
-
- // Test sink format.
- final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink;
- assertThat(actualSink).isEqualTo(expectedSink);
-
- // Test kafka producer.
- DynamicTableSink.SinkRuntimeProvider provider =
- actualUpsertKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
- assertThat(provider).isInstanceOf(DataStreamSinkProvider.class);
- final DataStreamSinkProvider sinkProvider = (DataStreamSinkProvider) provider;
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- sinkProvider.consumeDataStream(
- n -> Optional.empty(), env.fromElements(new BinaryRowData(1)));
- final StreamOperatorFactory<?> sinkOperatorFactory =
- env.getStreamGraph().getStreamNodes().stream()
- .filter(n -> n.getOperatorName().contains("Writer"))
- .findFirst()
- .orElseThrow(
- () ->
- new RuntimeException(
- "Expected operator with name Sink in stream graph."))
- .getOperatorFactory();
- assertThat(sinkOperatorFactory).isInstanceOf(SinkWriterOperatorFactory.class);
- org.apache.flink.api.connector.sink2.Sink sink =
- ((SinkWriterOperatorFactory) sinkOperatorFactory).getSink();
- assertThat(sink).isInstanceOf(ReducingUpsertSink.class);
- }
-
- @Test
- public void testTableSinkWithParallelism() {
- final Map<String, String> modifiedOptions =
- getModifiedOptions(
- getFullSinkOptions(), options -> options.put("sink.parallelism", "100"));
- final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions);
-
- final DynamicTableSink expectedSink =
- createExpectedSink(
- SINK_SCHEMA.toPhysicalRowDataType(),
- keyEncodingFormat,
- valueEncodingFormat,
- SINK_KEY_FIELDS,
- SINK_VALUE_FIELDS,
- null,
- SINK_TOPIC,
- UPSERT_KAFKA_SINK_PROPERTIES,
- DeliveryGuarantee.AT_LEAST_ONCE,
- SinkBufferFlushMode.DISABLED,
- 100);
- assertThat(actualSink).isEqualTo(expectedSink);
-
- final DynamicTableSink.SinkRuntimeProvider provider =
- actualSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
- assertThat(provider).isInstanceOf(SinkV2Provider.class);
- final SinkV2Provider sinkProvider = (SinkV2Provider) provider;
- assertThat(sinkProvider.getParallelism()).isPresent();
- assertThat((long) sinkProvider.getParallelism().get()).isEqualTo(100);
- }
-
- @Test
- public void testTableSinkAutoCompleteSchemaRegistrySubject() {
- // value.format + key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- },
- DEFAULT_VALUE_SUBJECT,
- DEFAULT_KEY_SUBJECT);
-
- // value.format + non-avro key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "csv");
- },
- DEFAULT_VALUE_SUBJECT,
- "N/A");
-
- // non-avro value.format + key.format
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "json");
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- },
- "N/A",
- DEFAULT_KEY_SUBJECT);
-
- // not override for 'key.format'
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("key.avro-confluent.subject", "sub2");
- },
- DEFAULT_VALUE_SUBJECT,
- "sub2");
-
- // not override for 'value.format'
- verifyEncoderSubject(
- options -> {
- options.put("value.format", "avro-confluent");
- options.put("value.avro-confluent.url", TEST_REGISTRY_URL);
- options.put("value.avro-confluent.subject", "sub1");
- options.put("key.format", "avro-confluent");
- options.put("key.avro-confluent.url", TEST_REGISTRY_URL);
- },
- "sub1",
- DEFAULT_KEY_SUBJECT);
- }
-
- private void verifyEncoderSubject(
- Consumer<Map<String, String>> optionModifier,
- String expectedValueSubject,
- String expectedKeySubject) {
- Map<String, String> options = new HashMap<>();
- // Kafka specific options.
- options.put("connector", UpsertKafkaDynamicTableFactory.IDENTIFIER);
- options.put("topic", SINK_TOPIC);
- options.put("properties.group.id", "dummy");
- options.put("properties.bootstrap.servers", "dummy");
- optionModifier.accept(options);
-
- final RowType rowType = (RowType) SINK_SCHEMA.toSinkRowDataType().getLogicalType();
- final String valueFormat =
- options.getOrDefault(
- FactoryUtil.FORMAT.key(),
- options.get(KafkaConnectorOptions.VALUE_FORMAT.key()));
- final String keyFormat = options.get(KafkaConnectorOptions.KEY_FORMAT.key());
-
- KafkaDynamicSink sink = (KafkaDynamicSink) createTableSink(SINK_SCHEMA, options);
-
- if (AVRO_CONFLUENT.equals(valueFormat)) {
- SerializationSchema<RowData> actualValueEncoder =
- sink.valueEncodingFormat.createRuntimeEncoder(
- new SinkRuntimeProviderContext(false), SINK_SCHEMA.toSinkRowDataType());
- assertThat(actualValueEncoder)
- .isEqualTo(createConfluentAvroSerSchema(rowType, expectedValueSubject));
- }
-
- if (AVRO_CONFLUENT.equals(keyFormat)) {
- assertThat(sink.keyEncodingFormat).isNotNull();
- SerializationSchema<RowData> actualKeyEncoder =
- sink.keyEncodingFormat.createRuntimeEncoder(
- new SinkRuntimeProviderContext(false), SINK_SCHEMA.toSinkRowDataType());
- assertThat(actualKeyEncoder)
- .isEqualTo(createConfluentAvroSerSchema(rowType, expectedKeySubject));
- }
- }
-
- private SerializationSchema<RowData> createConfluentAvroSerSchema(
- RowType rowType, String subject) {
- return new AvroRowDataSerializationSchema(
- rowType,
- ConfluentRegistryAvroSerializationSchema.forGeneric(
- subject, AvroSchemaConverter.convertToSchema(rowType), TEST_REGISTRY_URL),
- RowDataToAvroConverters.createConverter(rowType));
- }
-
- // --------------------------------------------------------------------------------------------
- // Negative tests
- // --------------------------------------------------------------------------------------------
-
- @Test
- public void testCreateSourceTableWithoutPK() {
- thrown.expect(ValidationException.class);
- thrown.expect(
- containsCause(
- new ValidationException(
- "'upsert-kafka' tables require to define a PRIMARY KEY constraint. "
- + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. "
- + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys.")));
-
- ResolvedSchema illegalSchema =
- ResolvedSchema.of(
- Column.physical("window_start", DataTypes.STRING()),
- Column.physical("region", DataTypes.STRING()),
- Column.physical("view_count", DataTypes.BIGINT()));
- createTableSource(illegalSchema, getFullSourceOptions());
- }
-
- @Test
- public void testCreateSinkTableWithoutPK() {
- thrown.expect(ValidationException.class);
- thrown.expect(
- containsCause(
- new ValidationException(
- "'upsert-kafka' tables require to define a PRIMARY KEY constraint. "
- + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. "
- + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys.")));
-
- ResolvedSchema illegalSchema =
- ResolvedSchema.of(
- Column.physical("region", DataTypes.STRING()),
- Column.physical("view_count", DataTypes.BIGINT()));
- createTableSink(illegalSchema, getFullSinkOptions());
- }
-
- @Test
- public void testSerWithCDCFormatAsValue() {
- thrown.expect(ValidationException.class);
- thrown.expect(
- containsCause(
- new ValidationException(
- String.format(
- "'upsert-kafka' connector doesn't support '%s' as value format, "
- + "because '%s' is not in insert-only mode.",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.IDENTIFIER))));
-
- createTableSink(
- SINK_SCHEMA,
- getModifiedOptions(
- getFullSinkOptions(),
- options ->
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.CHANGELOG_MODE.key()),
- "I;UA;UB;D")));
- }
-
- @Test
- public void testDeserWithCDCFormatAsValue() {
- thrown.expect(ValidationException.class);
- thrown.expect(
- containsCause(
- new ValidationException(
- String.format(
- "'upsert-kafka' connector doesn't support '%s' as value format, "
- + "because '%s' is not in insert-only mode.",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.IDENTIFIER))));
-
- createTableSource(
- SOURCE_SCHEMA,
- getModifiedOptions(
- getFullSourceOptions(),
- options ->
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER,
- TestFormatFactory.CHANGELOG_MODE.key()),
- "I;UA;UB;D")));
- }
-
- @Test
- public void testInvalidSinkBufferFlush() {
- thrown.expect(ValidationException.class);
- thrown.expect(
- containsCause(
- new ValidationException(
- "'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' "
- + "must be set to be greater than zero together to enable"
- + " sink buffer flushing.")));
- createTableSink(
- SINK_SCHEMA,
- getModifiedOptions(
- getFullSinkOptions(),
- options -> {
- options.put("sink.buffer-flush.max-rows", "0");
- options.put("sink.buffer-flush.interval", "1s");
- }));
- }
-
- // --------------------------------------------------------------------------------------------
- // Utilities
- // --------------------------------------------------------------------------------------------
-
- /**
- * Returns the full options modified by the given consumer {@code optionModifier}.
- *
- * @param optionModifier Consumer to modify the options
- */
- private static Map<String, String> getModifiedOptions(
- Map<String, String> options, Consumer<Map<String, String>> optionModifier) {
- optionModifier.accept(options);
- return options;
- }
-
- private static Map<String, String> getFullSourceOptions() {
- // table options
- Map<String, String> options = new HashMap<>();
- options.put("connector", UpsertKafkaDynamicTableFactory.IDENTIFIER);
- options.put("topic", SOURCE_TOPIC);
- options.put("properties.bootstrap.servers", "dummy");
- // key format options
- options.put("key.format", TestFormatFactory.IDENTIFIER);
- options.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- ",");
- options.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()),
- "true");
- options.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.CHANGELOG_MODE.key()),
- "I");
- // value format options
- options.put("value.format", TestFormatFactory.IDENTIFIER);
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- ",");
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()),
- "true");
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.CHANGELOG_MODE.key()),
- "I");
- return options;
- }
-
- private static Map<String, String> getFullSinkOptions() {
- Map<String, String> options = new HashMap<>();
- options.put("connector", UpsertKafkaDynamicTableFactory.IDENTIFIER);
- options.put("topic", SINK_TOPIC);
- options.put("properties.bootstrap.servers", "dummy");
- // key format options
- options.put("value.format", TestFormatFactory.IDENTIFIER);
- options.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- ",");
- options.put(
- String.format(
- "key.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.CHANGELOG_MODE.key()),
- "I");
- // value format options
- options.put("key.format", TestFormatFactory.IDENTIFIER);
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()),
- ",");
- options.put(
- String.format(
- "value.%s.%s",
- TestFormatFactory.IDENTIFIER, TestFormatFactory.CHANGELOG_MODE.key()),
- "I");
- return options;
- }
-
- private KafkaDynamicSource createExpectedScanSource(
- DataType producedDataType,
- DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
- DecodingFormat<DeserializationSchema<RowData>> valueDecodingFormat,
- int[] keyFields,
- int[] valueFields,
- String keyPrefix,
- String topic,
- Properties properties) {
- return new KafkaDynamicSource(
- producedDataType,
- keyDecodingFormat,
- new UpsertKafkaDynamicTableFactory.DecodingFormatWrapper(valueDecodingFormat),
- keyFields,
- valueFields,
- keyPrefix,
- Collections.singletonList(topic),
- null,
- properties,
- StartupMode.EARLIEST,
- Collections.emptyMap(),
- 0,
- BoundedMode.UNBOUNDED,
- Collections.emptyMap(),
- 0,
- true,
- FactoryMocks.IDENTIFIER.asSummaryString());
- }
-
- private static KafkaDynamicSink createExpectedSink(
- DataType consumedDataType,
- EncodingFormat<SerializationSchema<RowData>> keyEncodingFormat,
- EncodingFormat<SerializationSchema<RowData>> valueEncodingFormat,
- int[] keyProjection,
- int[] valueProjection,
- String keyPrefix,
- String topic,
- Properties properties,
- DeliveryGuarantee deliveryGuarantee,
- SinkBufferFlushMode flushMode,
- Integer parallelism) {
- return new KafkaDynamicSink(
- consumedDataType,
- consumedDataType,
- keyEncodingFormat,
- new UpsertKafkaDynamicTableFactory.EncodingFormatWrapper(valueEncodingFormat),
- keyProjection,
- valueProjection,
- keyPrefix,
- topic,
- properties,
- null,
- deliveryGuarantee,
- true,
- flushMode,
- parallelism,
- null);
- }
-
- private void assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) {
- assertThat(provider).isInstanceOf(DataStreamScanProvider.class);
- final DataStreamScanProvider dataStreamScanProvider = (DataStreamScanProvider) provider;
- final Transformation<RowData> transformation =
- dataStreamScanProvider
- .produceDataStream(
- n -> Optional.empty(),
- StreamExecutionEnvironment.createLocalEnvironment())
- .getTransformation();
- assertThat(transformation).isInstanceOf(SourceTransformation.class);
- SourceTransformation<RowData, KafkaPartitionSplit, KafkaSourceEnumState>
- sourceTransformation =
- (SourceTransformation<RowData, KafkaPartitionSplit, KafkaSourceEnumState>)
- transformation;
- assertThat(sourceTransformation.getSource()).isInstanceOf(KafkaSource.class);
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java
deleted file mode 100644
index 109f440..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java
+++ /dev/null
@@ -1,882 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.Schema;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.table.utils.LegacyRowResource;
-import org.apache.flink.types.Row;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.time.Duration;
-import java.time.LocalDateTime;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.api.common.typeinfo.Types.INT;
-import static org.apache.flink.api.common.typeinfo.Types.LOCAL_DATE_TIME;
-import static org.apache.flink.api.common.typeinfo.Types.ROW_NAMED;
-import static org.apache.flink.api.common.typeinfo.Types.STRING;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.comparedWithKeyAndOrder;
-import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.waitingExpectedResults;
-import static org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow;
-import static org.apache.flink.table.utils.TableTestMatchers.deepEqualTo;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.HamcrestCondition.matching;
-
-/** Upsert-kafka IT cases. */
-@RunWith(Parameterized.class)
-public class UpsertKafkaTableITCase extends KafkaTableTestBase {
-
- private static final String JSON_FORMAT = "json";
- private static final String CSV_FORMAT = "csv";
- private static final String AVRO_FORMAT = "avro";
-
- @Parameterized.Parameter public String format;
-
- @Parameterized.Parameters(name = "format = {0}")
- public static Object[] parameters() {
- return new Object[] {JSON_FORMAT, CSV_FORMAT, AVRO_FORMAT};
- }
-
- @Rule public final LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE;
-
- private static final String USERS_TOPIC = "users";
- private static final String WORD_COUNT_TOPIC = "word_count";
-
- @Test
- public void testAggregate() throws Exception {
- String topic = WORD_COUNT_TOPIC + "_" + format;
- createTestTopic(topic, 4, 1);
- // ------------- test ---------------
- wordCountToUpsertKafka(topic);
- wordFreqToUpsertKafka(topic);
- // ------------- clean up ---------------
- deleteTestTopic(topic);
- }
-
- @Test
- public void testTemporalJoin() throws Exception {
- String topic = USERS_TOPIC + "_" + format;
- createTestTopic(topic, 2, 1);
- // ------------- test ---------------
- // Kafka DefaultPartitioner's hash strategy is slightly different from Flink
- // KeyGroupStreamPartitioner,
- // which causes the records in the different Flink partitions are written into the same
- // Kafka partition.
- // When reading from the out-of-order Kafka partition, we need to set suitable watermark
- // interval to
- // tolerate the disorderliness.
- // For convenience, we just set the parallelism 1 to make all records are in the same Flink
- // partition and
- // use the Kafka DefaultPartition to repartition the records.
- env.setParallelism(1);
- writeChangelogToUpsertKafkaWithMetadata(topic);
- env.setParallelism(2);
- temporalJoinUpsertKafka(topic);
- // ------------- clean up ---------------
- deleteTestTopic(topic);
- }
-
- @Test
- public void testBufferedUpsertSink() throws Exception {
- final String topic = "buffered_upsert_topic_" + format;
- createTestTopic(topic, 1, 1);
- String bootstraps = getBootstrapServers();
- env.setParallelism(1);
-
- Table table =
- tEnv.fromDataStream(
- env.fromElements(
- Row.of(
- 1,
- LocalDateTime.parse("2020-03-08T13:12:11.12"),
- "payload 1"),
- Row.of(
- 2,
- LocalDateTime.parse("2020-03-09T13:12:11.12"),
- "payload 2"),
- Row.of(
- 3,
- LocalDateTime.parse("2020-03-10T13:12:11.12"),
- "payload 3"),
- Row.of(
- 3,
- LocalDateTime.parse("2020-03-11T13:12:11.12"),
- "payload"))
- .returns(
- ROW_NAMED(
- new String[] {"k_id", "ts", "payload"},
- INT,
- LOCAL_DATE_TIME,
- STRING)),
- Schema.newBuilder()
- .column("k_id", DataTypes.INT())
- .column("ts", DataTypes.TIMESTAMP(3))
- .column("payload", DataTypes.STRING())
- .watermark("ts", "ts")
- .build());
-
- final String createTable =
- String.format(
- "CREATE TABLE upsert_kafka (\n"
- + " `k_id` INTEGER,\n"
- + " `ts` TIMESTAMP(3),\n"
- + " `payload` STRING,\n"
- + " PRIMARY KEY (k_id) NOT ENFORCED"
- + ") WITH (\n"
- + " 'connector' = 'upsert-kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'key.format' = '%s',\n"
- + " 'key.fields-prefix' = 'k_',\n"
- + " 'sink.buffer-flush.max-rows' = '2',\n"
- + " 'sink.buffer-flush.interval' = '100000',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'EXCEPT_KEY'\n"
- + ")",
- topic, bootstraps, format, format);
-
- tEnv.executeSql(createTable);
-
- table.executeInsert("upsert_kafka").await();
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM upsert_kafka"), 3);
- final List<Row> expected =
- Arrays.asList(
- changelogRow(
- "+I",
- 1,
- LocalDateTime.parse("2020-03-08T13:12:11.120"),
- "payload 1"),
- changelogRow(
- "+I",
- 2,
- LocalDateTime.parse("2020-03-09T13:12:11.120"),
- "payload 2"),
- changelogRow(
- "+I",
- 3,
- LocalDateTime.parse("2020-03-11T13:12:11.120"),
- "payload"));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testSourceSinkWithKeyAndPartialValue() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "key_partial_value_topic_" + format;
- createTestTopic(topic, 1, 1); // use single partition to guarantee orders in tests
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
-
- // k_user_id and user_id have different data types to verify the correct mapping,
- // fields are reordered on purpose
- final String createTable =
- String.format(
- "CREATE TABLE upsert_kafka (\n"
- + " `k_user_id` BIGINT,\n"
- + " `name` STRING,\n"
- + " `timestamp` TIMESTAMP(3) METADATA,\n"
- + " `k_event_id` BIGINT,\n"
- + " `user_id` INT,\n"
- + " `payload` STRING,\n"
- + " PRIMARY KEY (k_event_id, k_user_id) NOT ENFORCED"
- + ") WITH (\n"
- + " 'connector' = 'upsert-kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'key.format' = '%s',\n"
- + " 'key.fields-prefix' = 'k_',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'EXCEPT_KEY'\n"
- + ")",
- topic, bootstraps, format, format);
-
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO upsert_kafka\n"
- + "VALUES\n"
- + " (1, 'name 1', TIMESTAMP '2020-03-08 13:12:11.123', 100, 41, 'payload 1'),\n"
- + " (2, 'name 2', TIMESTAMP '2020-03-09 13:12:11.123', 101, 42, 'payload 2'),\n"
- + " (3, 'name 3', TIMESTAMP '2020-03-10 13:12:11.123', 102, 43, 'payload 3'),\n"
- + " (2, 'name 2', TIMESTAMP '2020-03-11 13:12:11.123', 101, 42, 'payload')";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM upsert_kafka"), 5);
-
- final List<Row> expected =
- Arrays.asList(
- changelogRow(
- "+I",
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 100L,
- 41,
- "payload 1"),
- changelogRow(
- "+I",
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 101L,
- 42,
- "payload 2"),
- changelogRow(
- "+I",
- 3L,
- "name 3",
- LocalDateTime.parse("2020-03-10T13:12:11.123"),
- 102L,
- 43,
- "payload 3"),
- changelogRow(
- "-U",
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 101L,
- 42,
- "payload 2"),
- changelogRow(
- "+U",
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-11T13:12:11.123"),
- 101L,
- 42,
- "payload"));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- @Test
- public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception {
- // we always use a different topic name for each parameterized topic,
- // in order to make sure the topic can be created.
- final String topic = "key_full_value_topic_" + format;
- createTestTopic(topic, 1, 1); // use single partition to guarantee orders in tests
-
- // ---------- Produce an event time stream into Kafka -------------------
- String bootstraps = getBootstrapServers();
-
- // compared to the partial value test we cannot support both k_user_id and user_id in a full
- // value due to duplicate names after key prefix stripping,
- // fields are reordered on purpose,
- // fields for keys and values are overlapping
- final String createTable =
- String.format(
- "CREATE TABLE upsert_kafka (\n"
- + " `user_id` BIGINT,\n"
- + " `name` STRING,\n"
- + " `timestamp` TIMESTAMP(3) METADATA,\n"
- + " `event_id` BIGINT,\n"
- + " `payload` STRING,\n"
- + " PRIMARY KEY (event_id, user_id) NOT ENFORCED"
- + ") WITH (\n"
- + " 'connector' = 'upsert-kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'key.format' = '%s',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'ALL',\n"
- + " 'sink.parallelism' = '4'" // enable different parallelism to
- // check ordering
- + ")",
- topic, bootstraps, format, format);
-
- tEnv.executeSql(createTable);
-
- String initialValues =
- "INSERT INTO upsert_kafka\n"
- + "VALUES\n"
- + " (1, 'name 1', TIMESTAMP '2020-03-08 13:12:11.123', 100, 'payload 1'),\n"
- + " (2, 'name 2', TIMESTAMP '2020-03-09 13:12:11.123', 101, 'payload 2'),\n"
- + " (3, 'name 3', TIMESTAMP '2020-03-10 13:12:11.123', 102, 'payload 3'),\n"
- + " (1, 'name 1', TIMESTAMP '2020-03-11 13:12:11.123', 100, 'payload')";
- tEnv.executeSql(initialValues).await();
-
- // ---------- Consume stream from Kafka -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM upsert_kafka"), 5);
-
- final List<Row> expected =
- Arrays.asList(
- changelogRow(
- "+I",
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 100L,
- "payload 1"),
- changelogRow(
- "+I",
- 2L,
- "name 2",
- LocalDateTime.parse("2020-03-09T13:12:11.123"),
- 101L,
- "payload 2"),
- changelogRow(
- "+I",
- 3L,
- "name 3",
- LocalDateTime.parse("2020-03-10T13:12:11.123"),
- 102L,
- "payload 3"),
- changelogRow(
- "-U",
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-08T13:12:11.123"),
- 100L,
- "payload 1"),
- changelogRow(
- "+U",
- 1L,
- "name 1",
- LocalDateTime.parse("2020-03-11T13:12:11.123"),
- 100L,
- "payload"));
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
-
- // ------------- cleanup -------------------
-
- deleteTestTopic(topic);
- }
-
- private void wordCountToUpsertKafka(String wordCountTable) throws Exception {
- String bootstraps = getBootstrapServers();
-
- // ------------- test data ---------------
-
- final List<Row> inputData =
- Arrays.stream("Good good study day day up Good boy".split(" "))
- .map(Row::of)
- .collect(Collectors.toList());
-
- // ------------- create table ---------------
-
- final String createSource =
- String.format(
- "CREATE TABLE words_%s ("
- + " `word` STRING"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'data-id' = '%s'"
- + ")",
- format, TestValuesTableFactory.registerData(inputData));
- tEnv.executeSql(createSource);
- final String createSinkTable =
- String.format(
- "CREATE TABLE %s (\n"
- + " `word` STRING,\n"
- + " `count` BIGINT,\n"
- + " PRIMARY KEY (`word`) NOT ENFORCED\n"
- + ") WITH (\n"
- + " 'connector' = 'upsert-kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'key.format' = '%s',\n"
- + " 'value.format' = '%s'"
- + ")",
- wordCountTable, wordCountTable, bootstraps, format, format);
- tEnv.executeSql(createSinkTable);
- String initialValues =
- "INSERT INTO "
- + wordCountTable
- + " "
- + "SELECT LOWER(`word`), count(*) as `count` "
- + "FROM words_"
- + format
- + " "
- + "GROUP BY LOWER(`word`)";
- tEnv.executeSql(initialValues).await();
-
- // ---------- read from the upsert sink -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM " + wordCountTable), 11);
-
- final Map<Row, List<Row>> expected = new HashMap<>();
- expected.put(
- Row.of("good"),
- Arrays.asList(
- changelogRow("+I", "good", 1L),
- changelogRow("-U", "good", 1L),
- changelogRow("+U", "good", 2L),
- changelogRow("-U", "good", 2L),
- changelogRow("+U", "good", 3L)));
- expected.put(Row.of("study"), Collections.singletonList(changelogRow("+I", "study", 1L)));
- expected.put(
- Row.of("day"),
- Arrays.asList(
- changelogRow("+I", "day", 1L),
- changelogRow("-U", "day", 1L),
- changelogRow("+U", "day", 2L)));
- expected.put(Row.of("up"), Collections.singletonList(changelogRow("+I", "up", 1L)));
- expected.put(Row.of("boy"), Collections.singletonList(changelogRow("+I", "boy", 1L)));
-
- comparedWithKeyAndOrder(expected, result, new int[] {0});
-
- // ---------- read the raw data from kafka -------------------
- // check we only write the upsert data into Kafka
- String rawWordCountTable = "raw_word_count";
- tEnv.executeSql(
- String.format(
- "CREATE TABLE %s (\n"
- + " `key_word` STRING NOT NULL,\n"
- + " `word` STRING NOT NULL,\n"
- + " `count` BIGINT\n"
- + ") WITH (\n"
- + " 'connector' = 'kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'scan.startup.mode' = 'earliest-offset',\n"
- + " 'key.format' = '%s',\n"
- + " 'key.fields' = 'key_word',\n"
- + " 'key.fields-prefix' = 'key_',\n"
- + " 'value.format' = '%s',\n"
- + " 'value.fields-include' = 'EXCEPT_KEY'"
- + ")",
- rawWordCountTable, wordCountTable, bootstraps, format, format));
-
- final List<Row> result2 =
- collectRows(tEnv.sqlQuery("SELECT * FROM " + rawWordCountTable), 8);
- final Map<Row, List<Row>> expected2 = new HashMap<>();
- expected2.put(
- Row.of("good"),
- Arrays.asList(
- Row.of("good", "good", 1L),
- Row.of("good", "good", 2L),
- Row.of("good", "good", 3L)));
- expected2.put(Row.of("study"), Collections.singletonList(Row.of("study", "study", 1L)));
- expected2.put(
- Row.of("day"), Arrays.asList(Row.of("day", "day", 1L), Row.of("day", "day", 2L)));
- expected2.put(Row.of("up"), Collections.singletonList(Row.of("up", "up", 1L)));
- expected2.put(Row.of("boy"), Collections.singletonList(Row.of("boy", "boy", 1L)));
-
- comparedWithKeyAndOrder(expected2, result2, new int[] {0});
- }
-
- private void wordFreqToUpsertKafka(String wordCountTable) throws Exception {
- // ------------- test data ---------------
-
- final List<String> expectedData = Arrays.asList("3,1", "2,1");
-
- // ------------- create table ---------------
-
- final String createSinkTable =
- "CREATE TABLE sink_"
- + format
- + "(\n"
- + " `count` BIGINT,\n"
- + " `freq` BIGINT,\n"
- + " PRIMARY KEY (`count`) NOT ENFORCED\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'sink-insert-only' = 'false'\n"
- + ")";
- tEnv.executeSql(createSinkTable);
- final TableResult query =
- tEnv.executeSql(
- "INSERT INTO sink_"
- + format
- + "\n"
- + "SELECT `count`, count(*) as `freq`\n"
- + "FROM "
- + wordCountTable
- + "\n"
- + "GROUP BY `count`\n"
- + "having count(*) < 2");
-
- // ---------- consume stream from sink -------------------
-
- waitingExpectedResults("sink_" + format, expectedData, Duration.ofSeconds(10));
- query.getJobClient().get().cancel();
- }
-
- private void writeChangelogToUpsertKafkaWithMetadata(String userTable) throws Exception {
- String bootstraps = getBootstrapServers();
-
- // ------------- test data ---------------
-
- // Prepare data for upsert kafka
- // Keep every partition has more than 1 record and the last record in every partition has
- // larger event time
- // than left stream event time to trigger the join.
- List<Row> changelogData =
- Arrays.asList(
- changelogRow(
- "+U",
- 100L,
- "Bob",
- "Beijing",
- LocalDateTime.parse("2020-08-15T00:00:01")),
- changelogRow(
- "+U",
- 101L,
- "Alice",
- "Shanghai",
- LocalDateTime.parse("2020-08-15T00:00:02")),
- changelogRow(
- "+U",
- 102L,
- "Greg",
- "Berlin",
- LocalDateTime.parse("2020-08-15T00:00:03")),
- changelogRow(
- "+U",
- 103L,
- "Richard",
- "Berlin",
- LocalDateTime.parse("2020-08-16T00:01:05")),
- changelogRow(
- "+U",
- 101L,
- "Alice",
- "Wuhan",
- LocalDateTime.parse("2020-08-16T00:02:00")),
- changelogRow(
- "+U",
- 104L,
- "Tomato",
- "Hongkong",
- LocalDateTime.parse("2020-08-16T00:05:05")),
- changelogRow(
- "+U",
- 105L,
- "Tim",
- "Shenzhen",
- LocalDateTime.parse("2020-08-16T00:06:00")),
- // Keep the timestamp in the records are in the ascending order.
- // It will keep the records in the kafka partition are in the order.
- // It has the same effects by adjusting the watermark strategy.
- changelogRow(
- "+U",
- 103L,
- "Richard",
- "London",
- LocalDateTime.parse("2020-08-16T01:01:05")),
- changelogRow(
- "+U",
- 101L,
- "Alice",
- "Hangzhou",
- LocalDateTime.parse("2020-08-16T01:04:05")),
- changelogRow(
- "+U",
- 104L,
- "Tomato",
- "Shenzhen",
- LocalDateTime.parse("2020-08-16T01:05:05")),
- changelogRow(
- "+U",
- 105L,
- "Tim",
- "Hongkong",
- LocalDateTime.parse("2020-08-16T01:06:00")));
-
- // ------------- create table ---------------
-
- final String createChangelog =
- String.format(
- "CREATE TABLE users_changelog_%s ("
- + " user_id BIGINT,"
- + " user_name STRING,"
- + " region STRING,"
- + " modification_time TIMESTAMP(3),"
- + " PRIMARY KEY (user_id) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'data-id' = '%s',"
- + " 'changelog-mode' = 'UA,D',"
- + " 'disable-lookup' = 'true'"
- + ")",
- format, TestValuesTableFactory.registerData(changelogData));
- tEnv.executeSql(createChangelog);
-
- // we verified computed column, watermark, metadata in this case too
- final String createSinkTable =
- String.format(
- "CREATE TABLE %s (\n"
- + " `user_id` BIGINT,\n"
- + " `user_name` STRING,\n"
- + " `region` STRING,\n"
- + " upper_region AS UPPER(`region`),\n"
- + " modification_time TIMESTAMP(3) METADATA FROM 'timestamp',\n"
- + " watermark for modification_time as modification_time,\n"
- + " PRIMARY KEY (`user_id`) NOT ENFORCED\n"
- + ") WITH (\n"
- + " 'connector' = 'upsert-kafka',\n"
- + " 'topic' = '%s',\n"
- + " 'properties.bootstrap.servers' = '%s',\n"
- + " 'key.format' = '%s',\n"
- + " 'value.format' = '%s'"
- + ")",
- userTable, userTable, bootstraps, format, format);
- tEnv.executeSql(createSinkTable);
- String initialValues =
- "INSERT INTO " + userTable + " " + "SELECT * " + "FROM users_changelog_" + format;
- tEnv.executeSql(initialValues).await();
-
- // ---------- consume stream from sink -------------------
-
- final List<Row> result = collectRows(tEnv.sqlQuery("SELECT * FROM " + userTable), 16);
-
- List<Row> expected =
- Arrays.asList(
- changelogRow(
- "+I",
- 100L,
- "Bob",
- "Beijing",
- "BEIJING",
- LocalDateTime.parse("2020-08-15T00:00:01")),
- changelogRow(
- "+I",
- 101L,
- "Alice",
- "Shanghai",
- "SHANGHAI",
- LocalDateTime.parse("2020-08-15T00:00:02")),
- changelogRow(
- "-U",
- 101L,
- "Alice",
- "Shanghai",
- "SHANGHAI",
- LocalDateTime.parse("2020-08-15T00:00:02")),
- changelogRow(
- "+U",
- 101L,
- "Alice",
- "Wuhan",
- "WUHAN",
- LocalDateTime.parse("2020-08-16T00:02:00")),
- changelogRow(
- "-U",
- 101L,
- "Alice",
- "Wuhan",
- "WUHAN",
- LocalDateTime.parse("2020-08-16T00:02:00")),
- changelogRow(
- "+U",
- 101L,
- "Alice",
- "Hangzhou",
- "HANGZHOU",
- LocalDateTime.parse("2020-08-16T01:04:05")),
- changelogRow(
- "+I",
- 102L,
- "Greg",
- "Berlin",
- "BERLIN",
- LocalDateTime.parse("2020-08-15T00:00:03")),
- changelogRow(
- "+I",
- 103L,
- "Richard",
- "Berlin",
- "BERLIN",
- LocalDateTime.parse("2020-08-16T00:01:05")),
- changelogRow(
- "-U",
- 103L,
- "Richard",
- "Berlin",
- "BERLIN",
- LocalDateTime.parse("2020-08-16T00:01:05")),
- changelogRow(
- "+U",
- 103L,
- "Richard",
- "London",
- "LONDON",
- LocalDateTime.parse("2020-08-16T01:01:05")),
- changelogRow(
- "+I",
- 104L,
- "Tomato",
- "Hongkong",
- "HONGKONG",
- LocalDateTime.parse("2020-08-16T00:05:05")),
- changelogRow(
- "-U",
- 104L,
- "Tomato",
- "Hongkong",
- "HONGKONG",
- LocalDateTime.parse("2020-08-16T00:05:05")),
- changelogRow(
- "+U",
- 104L,
- "Tomato",
- "Shenzhen",
- "SHENZHEN",
- LocalDateTime.parse("2020-08-16T01:05:05")),
- changelogRow(
- "+I",
- 105L,
- "Tim",
- "Shenzhen",
- "SHENZHEN",
- LocalDateTime.parse("2020-08-16T00:06")),
- changelogRow(
- "-U",
- 105L,
- "Tim",
- "Shenzhen",
- "SHENZHEN",
- LocalDateTime.parse("2020-08-16T00:06")),
- changelogRow(
- "+U",
- 105L,
- "Tim",
- "Hongkong",
- "HONGKONG",
- LocalDateTime.parse("2020-08-16T01:06")));
-
- // we ignore the orders for easier comparing, as we already verified ordering in
- // testAggregate()
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
- }
-
- private void temporalJoinUpsertKafka(String userTable) throws Exception {
- // ------------- test data ---------------
- List<Row> input =
- Arrays.asList(
- Row.of(10001L, 100L, LocalDateTime.parse("2020-08-15T00:00:02")),
- Row.of(10002L, 101L, LocalDateTime.parse("2020-08-15T00:00:03")),
- Row.of(10002L, 102L, LocalDateTime.parse("2020-08-15T00:00:04")),
- Row.of(10002L, 101L, LocalDateTime.parse("2020-08-16T00:02:01")),
- Row.of(10004L, 104L, LocalDateTime.parse("2020-08-16T00:04:00")),
- Row.of(10003L, 101L, LocalDateTime.parse("2020-08-16T00:04:06")),
- Row.of(10004L, 104L, LocalDateTime.parse("2020-08-16T00:05:06")));
-
- List<Row> expected =
- Arrays.asList(
- Row.of(
- 10001L,
- 100L,
- LocalDateTime.parse("2020-08-15T00:00:02"),
- "Bob",
- "BEIJING",
- LocalDateTime.parse("2020-08-15T00:00:01")),
- Row.of(
- 10002L,
- 101L,
- LocalDateTime.parse("2020-08-15T00:00:03"),
- "Alice",
- "SHANGHAI",
- LocalDateTime.parse("2020-08-15T00:00:02")),
- Row.of(
- 10002L,
- 102L,
- LocalDateTime.parse("2020-08-15T00:00:04"),
- "Greg",
- "BERLIN",
- LocalDateTime.parse("2020-08-15T00:00:03")),
- Row.of(
- 10002L,
- 101L,
- LocalDateTime.parse("2020-08-16T00:02:01"),
- "Alice",
- "WUHAN",
- LocalDateTime.parse("2020-08-16T00:02:00")),
- Row.of(
- 10004L,
- 104L,
- LocalDateTime.parse("2020-08-16T00:04:00"),
- null,
- null,
- null),
- Row.of(
- 10003L,
- 101L,
- LocalDateTime.parse("2020-08-16T00:04:06"),
- "Alice",
- "WUHAN",
- LocalDateTime.parse("2020-08-16T00:02:00")),
- Row.of(
- 10004L,
- 104L,
- LocalDateTime.parse("2020-08-16T00:05:06"),
- "Tomato",
- "HONGKONG",
- LocalDateTime.parse("2020-08-16T00:05:05")));
-
- // ------------- create table ---------------
-
- tEnv.executeSql(
- String.format(
- "CREATE TABLE pageviews_%s(\n"
- + " `page_id` BIGINT,\n"
- + " `user_id` BIGINT,\n"
- + " `viewtime` TIMESTAMP(3),\n"
- + " `proctime` as proctime(),\n"
- + " watermark for `viewtime` as `viewtime`\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'data-id' = '%s'\n"
- + ")",
- format, TestValuesTableFactory.registerData(input)));
-
- final List<Row> result =
- collectRows(
- tEnv.sqlQuery(
- String.format(
- "SELECT p.page_id, p.user_id, p.viewtime, u.user_name, u.upper_region, u.modification_time\n"
- + "FROM pageviews_%s AS p\n"
- + "LEFT JOIN %s FOR SYSTEM_TIME AS OF p.viewtime AS u\n"
- + "ON p.user_id = u.user_id",
- format, userTable)),
- 7);
-
- assertThat(result).satisfies(matching(deepEqualTo(expected, true)));
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ClusterCommunicationUtils.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ClusterCommunicationUtils.java
deleted file mode 100644
index 1fc6769..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ClusterCommunicationUtils.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.runtime.client.JobStatusMessage;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/** Utilities for communicating with a cluster through a {@link ClusterClient}. */
-public class ClusterCommunicationUtils {
-
- public static void waitUntilJobIsRunning(ClusterClient<?> client) throws Exception {
- while (getRunningJobs(client).isEmpty()) {
- Thread.sleep(50);
- }
- }
-
- public static void waitUntilNoJobIsRunning(ClusterClient<?> client) throws Exception {
- while (!getRunningJobs(client).isEmpty()) {
- Thread.sleep(50);
- }
- }
-
- public static List<JobID> getRunningJobs(ClusterClient<?> client) throws Exception {
- Collection<JobStatusMessage> statusMessages = client.listJobs().get();
- return statusMessages.stream()
- .filter(status -> !status.getJobState().isGloballyTerminalState())
- .map(JobStatusMessage::getJobId)
- .collect(Collectors.toList());
- }
-
- private ClusterCommunicationUtils() {}
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
deleted file mode 100644
index 3671f2f..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-/** Test data generators. */
-@SuppressWarnings("serial")
-public class DataGenerators {
-
- public static void generateRandomizedIntegerSequence(
- StreamExecutionEnvironment env,
- KafkaTestEnvironment testServer,
- String topic,
- final int numPartitions,
- final int numElements,
- final boolean randomizeOrder)
- throws Exception {
- env.setParallelism(numPartitions);
- env.setRestartStrategy(RestartStrategies.noRestart());
-
- DataStream<Integer> stream =
- env.addSource(
- new RichParallelSourceFunction<Integer>() {
-
- private volatile boolean running = true;
-
- @Override
- public void run(SourceContext<Integer> ctx) {
- // create a sequence
- int[] elements = new int[numElements];
- for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask();
- i < numElements;
- i++,
- val +=
- getRuntimeContext()
- .getNumberOfParallelSubtasks()) {
-
- elements[i] = val;
- }
-
- // scramble the sequence
- if (randomizeOrder) {
- Random rnd = new Random();
- for (int i = 0; i < elements.length; i++) {
- int otherPos = rnd.nextInt(elements.length);
-
- int tmp = elements[i];
- elements[i] = elements[otherPos];
- elements[otherPos] = tmp;
- }
- }
-
- // emit the sequence
- int pos = 0;
- while (running && pos < elements.length) {
- ctx.collect(elements[pos++]);
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
- });
-
- Properties props = new Properties();
- props.putAll(
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(
- testServer.getBrokerConnectionString()));
- Properties secureProps = testServer.getSecureProperties();
- if (secureProps != null) {
- props.putAll(testServer.getSecureProperties());
- }
- // Ensure the producer enables idempotence.
- props.putAll(testServer.getIdempotentProducerConfig());
-
- stream = stream.rebalance();
- testServer.produceIntoKafka(
- stream,
- topic,
- new TypeInformationSerializationSchema<>(
- BasicTypeInfo.INT_TYPE_INFO, env.getConfig()),
- props,
- new FlinkKafkaPartitioner<Integer>() {
- @Override
- public int partition(
- Integer next,
- byte[] serializedKey,
- byte[] serializedValue,
- String topic,
- int[] partitions) {
- return next % partitions.length;
- }
- });
-
- env.execute("Scrambles int sequence generator");
- }
-
- // ------------------------------------------------------------------------
-
- /**
- * A generator that continuously writes strings into the configured topic. The generation is
- * stopped if an exception occurs or {@link #shutdown()} is called.
- */
- public static class InfiniteStringsGenerator extends Thread {
-
- private final KafkaTestEnvironment server;
-
- private final String topic;
-
- private volatile Throwable error;
-
- private volatile boolean running = true;
-
- public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) {
- this.server = server;
- this.topic = topic;
- }
-
- @Override
- public void run() {
- // we manually feed data into the Kafka sink
- RichFunction producer = null;
- try {
- Properties producerProperties =
- FlinkKafkaProducerBase.getPropertiesFromBrokerList(
- server.getBrokerConnectionString());
- producerProperties.setProperty("retries", "3");
- Transformation<String> mockTransform = new MockTransformation();
- DataStream<String> stream =
- new DataStream<>(new DummyStreamExecutionEnvironment(), mockTransform);
-
- StreamSink<String> sink =
- server.getProducerSink(
- topic,
- new SimpleStringSchema(),
- producerProperties,
- new FlinkFixedPartitioner<>());
-
- OneInputStreamOperatorTestHarness<String, Object> testHarness =
- new OneInputStreamOperatorTestHarness<>(sink);
-
- testHarness.open();
-
- final StringBuilder bld = new StringBuilder();
- final Random rnd = new Random();
-
- while (running) {
- bld.setLength(0);
-
- int len = rnd.nextInt(100) + 1;
- for (int i = 0; i < len; i++) {
- bld.append((char) (rnd.nextInt(20) + 'a'));
- }
-
- String next = bld.toString();
- testHarness.processElement(new StreamRecord<>(next));
- }
- } catch (Throwable t) {
- this.error = t;
- } finally {
- if (producer != null) {
- try {
- producer.close();
- } catch (Throwable t) {
- // ignore
- }
- }
- }
- }
-
- public void shutdown() {
- this.running = false;
- this.interrupt();
- }
-
- public Throwable getError() {
- return this.error;
- }
-
- private static class MockTransformation extends Transformation<String> {
- public MockTransformation() {
- super("MockTransform", BasicTypeInfo.STRING_TYPE_INFO, 1);
- }
-
- @Override
- public List<Transformation<?>> getTransitivePredecessors() {
- return null;
- }
-
- @Override
- public List<Transformation<?>> getInputs() {
- return Collections.emptyList();
- }
- }
-
- private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
-
- @Override
- public JobExecutionResult execute(StreamGraph streamGraph) throws Exception {
- return null;
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
deleted file mode 100644
index fab374a..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A {@link RichMapFunction} that fails after the configured number of records have been processed.
- *
- * @param <T>
- */
-public class FailingIdentityMapper<T> extends RichMapFunction<T, T>
- implements ListCheckpointed<Integer>, CheckpointListener, Runnable {
-
- private static final Logger LOG = LoggerFactory.getLogger(FailingIdentityMapper.class);
-
- private static final long serialVersionUID = 6334389850158707313L;
-
- public static volatile boolean failedBefore;
-
- private final int failCount;
- private int numElementsTotal;
- private int numElementsThisTime;
-
- private boolean failer;
- private boolean hasBeenCheckpointed;
-
- private Thread printer;
- private volatile boolean printerRunning = true;
-
- public FailingIdentityMapper(int failCount) {
- this.failCount = failCount;
- }
-
- @Override
- public void open(Configuration parameters) {
- failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
- printer = new Thread(this, "FailingIdentityMapper Status Printer");
- printer.start();
- }
-
- @Override
- public T map(T value) throws Exception {
- numElementsTotal++;
- numElementsThisTime++;
-
- if (!failedBefore) {
- Thread.sleep(10);
-
- if (failer && numElementsTotal >= failCount) {
- failedBefore = true;
- throw new Exception("Artificial Test Failure");
- }
- }
- return value;
- }
-
- @Override
- public void close() throws Exception {
- printerRunning = false;
- if (printer != null) {
- printer.interrupt();
- printer = null;
- }
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) {
- this.hasBeenCheckpointed = true;
- }
-
- @Override
- public void notifyCheckpointAborted(long checkpointId) {}
-
- @Override
- public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
- return Collections.singletonList(numElementsTotal);
- }
-
- @Override
- public void restoreState(List<Integer> state) throws Exception {
- if (state.isEmpty() || state.size() > 1) {
- throw new RuntimeException(
- "Test failed due to unexpected recovered state size " + state.size());
- }
- this.numElementsTotal = state.get(0);
- }
-
- @Override
- public void run() {
- while (printerRunning) {
- try {
- Thread.sleep(5000);
- } catch (InterruptedException e) {
- // ignore
- }
- LOG.info(
- "============================> Failing mapper {}: count={}, totalCount={}",
- getRuntimeContext().getIndexOfThisSubtask(),
- numElementsThisTime,
- numElementsTotal);
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
deleted file mode 100644
index 7b7c225..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FakeStandardProducerConfig.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-
-import java.util.Properties;
-
-/** Test configuration for a kafka producer. */
-public class FakeStandardProducerConfig {
-
- public static Properties get() {
- Properties p = new Properties();
- p.setProperty("bootstrap.servers", "localhost:12345");
- p.setProperty("key.serializer", ByteArraySerializer.class.getName());
- p.setProperty("value.serializer", ByteArraySerializer.class.getName());
- return p;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/IntegerSource.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/IntegerSource.java
deleted file mode 100644
index 6111a76..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/IntegerSource.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.util.SerializableObject;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A Flink source that servers integers, but it completes only after a completed checkpoint after
- * serving all of the elements.
- */
-public class IntegerSource extends RichParallelSourceFunction<Integer>
- implements ListCheckpointed<Integer>, CheckpointListener {
-
- /**
- * Blocker when the generator needs to wait for the checkpoint to happen. Eager initialization
- * means it must be serializable (pick any serializable type).
- */
- private final Object blocker = new SerializableObject();
-
- /** The total number of events to generate. */
- private final int numEventsTotal;
-
- /** The current position in the sequence of numbers. */
- private int currentPosition = -1;
-
- private long lastCheckpointTriggered;
-
- private long lastCheckpointConfirmed;
-
- private boolean restored;
-
- private volatile boolean running = true;
-
- public IntegerSource(int numEventsTotal) {
- this.numEventsTotal = numEventsTotal;
- }
-
- @Override
- public void run(SourceContext<Integer> ctx) throws Exception {
-
- // each source subtask emits only the numbers where (num % parallelism == subtask_index)
- final int stepSize = getRuntimeContext().getNumberOfParallelSubtasks();
- int current =
- this.currentPosition >= 0
- ? this.currentPosition
- : getRuntimeContext().getIndexOfThisSubtask();
-
- while (this.running && current < this.numEventsTotal) {
- // emit the next element
- synchronized (ctx.getCheckpointLock()) {
- ctx.collect(current);
- current += stepSize;
- this.currentPosition = current;
- }
- // give some time to trigger checkpoint while we are not holding the lock (to prevent
- // starvation)
- if (!restored && current % 10 == 0) {
- Thread.sleep(1);
- }
- }
-
- // after we are done, we need to wait for two more checkpoint to complete
- // before finishing the program - that is to be on the safe side that
- // the sink also got the "commit" notification for all relevant checkpoints
- // and committed the data
- final long lastCheckpoint;
- synchronized (ctx.getCheckpointLock()) {
- lastCheckpoint = this.lastCheckpointTriggered;
- }
-
- synchronized (this.blocker) {
- while (this.lastCheckpointConfirmed <= lastCheckpoint + 1) {
- this.blocker.wait();
- }
- }
- }
-
- @Override
- public void cancel() {
- this.running = false;
- }
-
- @Override
- public List<Integer> snapshotState(long checkpointId, long checkpointTimestamp)
- throws Exception {
- this.lastCheckpointTriggered = checkpointId;
-
- return Collections.singletonList(this.currentPosition);
- }
-
- @Override
- public void restoreState(List<Integer> state) throws Exception {
- this.currentPosition = state.get(0);
-
- // at least one checkpoint must have happened so far
- this.lastCheckpointTriggered = 1L;
- this.lastCheckpointConfirmed = 1L;
- this.restored = true;
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- synchronized (blocker) {
- this.lastCheckpointConfirmed = checkpointId;
- blocker.notifyAll();
- }
- }
-
- @Override
- public void notifyCheckpointAborted(long checkpointId) {}
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
deleted file mode 100644
index 170968a..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/** {@link MapFunction} that verifies that he partitioning is identical. */
-public class PartitionValidatingMapper implements MapFunction<Integer, Integer> {
-
- private static final long serialVersionUID = 1088381231244959088L;
-
- /* the partitions from which this function received data */
- private final Set<Integer> myPartitions = new HashSet<>();
-
- private final int numPartitions;
- private final int maxPartitions;
-
- public PartitionValidatingMapper(int numPartitions, int maxPartitions) {
- this.numPartitions = numPartitions;
- this.maxPartitions = maxPartitions;
- }
-
- @Override
- public Integer map(Integer value) throws Exception {
- // validate that the partitioning is identical
- int partition = value % numPartitions;
- myPartitions.add(partition);
- if (myPartitions.size() > maxPartitions) {
- throw new Exception(
- "Error: Elements from too many different partitions: "
- + myPartitions
- + ". Expect elements only from "
- + maxPartitions
- + " partitions");
- }
- return value;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestPartitionDiscoverer.java
deleted file mode 100644
index 5731273..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestPartitionDiscoverer.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Utility {@link AbstractPartitionDiscoverer} for tests that allows mocking the sequence of
- * consecutive metadata fetch calls to Kafka.
- */
-public class TestPartitionDiscoverer extends AbstractPartitionDiscoverer {
-
- private final KafkaTopicsDescriptor topicsDescriptor;
-
- private final List<List<String>> mockGetAllTopicsReturnSequence;
- private final List<List<KafkaTopicPartition>> mockGetAllPartitionsForTopicsReturnSequence;
-
- private int getAllTopicsInvokeCount = 0;
- private int getAllPartitionsForTopicsInvokeCount = 0;
-
- public TestPartitionDiscoverer(
- KafkaTopicsDescriptor topicsDescriptor,
- int indexOfThisSubtask,
- int numParallelSubtasks,
- List<List<String>> mockGetAllTopicsReturnSequence,
- List<List<KafkaTopicPartition>> mockGetAllPartitionsForTopicsReturnSequence) {
-
- super(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks);
-
- this.topicsDescriptor = topicsDescriptor;
- this.mockGetAllTopicsReturnSequence = mockGetAllTopicsReturnSequence;
- this.mockGetAllPartitionsForTopicsReturnSequence =
- mockGetAllPartitionsForTopicsReturnSequence;
- }
-
- @Override
- protected List<String> getAllTopics() {
- assertThat(topicsDescriptor.isTopicPattern()).isTrue();
- return mockGetAllTopicsReturnSequence.get(getAllTopicsInvokeCount++);
- }
-
- @Override
- protected List<KafkaTopicPartition> getAllPartitionsForTopics(List<String> topics) {
- if (topicsDescriptor.isFixedTopics()) {
- assertThat(topics).isEqualTo(topicsDescriptor.getFixedTopics());
- } else {
- assertThat(topics)
- .isEqualTo(
- mockGetAllTopicsReturnSequence.get(
- getAllPartitionsForTopicsInvokeCount - 1));
- }
- return mockGetAllPartitionsForTopicsReturnSequence.get(
- getAllPartitionsForTopicsInvokeCount++);
- }
-
- @Override
- protected void initializeConnections() {
- // nothing to do
- }
-
- @Override
- protected void wakeupConnections() {
- // nothing to do
- }
-
- @Override
- protected void closeConnections() {
- // nothing to do
- }
-
- // ---------------------------------------------------------------------------------
- // Utilities to create mocked, fixed results for a sequences of metadata fetches
- // ---------------------------------------------------------------------------------
-
- public static List<List<String>> createMockGetAllTopicsSequenceFromFixedReturn(
- final List<String> fixed) {
- @SuppressWarnings("unchecked")
- List<List<String>> mockSequence = mock(List.class);
- when(mockSequence.get(anyInt()))
- .thenAnswer(
- new Answer<List<String>>() {
- @Override
- public List<String> answer(InvocationOnMock invocationOnMock)
- throws Throwable {
- return new ArrayList<>(fixed);
- }
- });
-
- return mockSequence;
- }
-
- public static List<List<KafkaTopicPartition>>
- createMockGetAllPartitionsFromTopicsSequenceFromFixedReturn(
- final List<KafkaTopicPartition> fixed) {
- @SuppressWarnings("unchecked")
- List<List<KafkaTopicPartition>> mockSequence = mock(List.class);
- when(mockSequence.get(anyInt()))
- .thenAnswer(
- new Answer<List<KafkaTopicPartition>>() {
- @Override
- public List<KafkaTopicPartition> answer(
- InvocationOnMock invocationOnMock) throws Throwable {
- return new ArrayList<>(fixed);
- }
- });
-
- return mockSequence;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java
deleted file mode 100644
index 1c2e8e7..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TestSourceContext.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/** Test {@link org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. */
-public class TestSourceContext<T> implements SourceFunction.SourceContext<T> {
-
- private final Object checkpointLock = new Object();
- private final Object watermarkLock = new Object();
-
- private volatile StreamRecord<T> latestElement;
- private volatile Watermark currentWatermark;
-
- @Override
- public void collect(T element) {
- this.latestElement = new StreamRecord<>(element);
- }
-
- @Override
- public void collectWithTimestamp(T element, long timestamp) {
- this.latestElement = new StreamRecord<>(element, timestamp);
- }
-
- @Override
- public void emitWatermark(Watermark mark) {
- synchronized (watermarkLock) {
- currentWatermark = mark;
- watermarkLock.notifyAll();
- }
- }
-
- @Override
- public void markAsTemporarilyIdle() {
- // do nothing
- }
-
- @Override
- public Object getCheckpointLock() {
- return checkpointLock;
- }
-
- @Override
- public void close() {
- // do nothing
- }
-
- public StreamRecord<T> getLatestElement() {
- return latestElement;
- }
-
- public boolean hasWatermark() {
- return currentWatermark != null;
- }
-
- public Watermark getLatestWatermark() throws InterruptedException {
- synchronized (watermarkLock) {
- while (currentWatermark == null) {
- watermarkLock.wait();
- }
- Watermark wm = currentWatermark;
- currentWatermark = null;
- return wm;
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
deleted file mode 100644
index 638d339..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-
-/**
- * An identity map function that sleeps between elements, throttling the processing speed.
- *
- * @param <T> The type mapped.
- */
-public class ThrottledMapper<T> implements MapFunction<T, T> {
-
- private static final long serialVersionUID = 467008933767159126L;
-
- private final int sleep;
-
- public ThrottledMapper(int sleep) {
- this.sleep = sleep;
- }
-
- @Override
- public T map(T value) throws Exception {
- Thread.sleep(this.sleep);
- return value;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java
deleted file mode 100644
index bc1db67..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2FlinkPartitioner.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-
-/**
- * Special partitioner that uses the first field of a 2-tuple as the partition, and that expects a
- * specific number of partitions.
- */
-public class Tuple2FlinkPartitioner extends FlinkKafkaPartitioner<Tuple2<Integer, Integer>> {
- private static final long serialVersionUID = -3589898230375281549L;
-
- private final int expectedPartitions;
-
- public Tuple2FlinkPartitioner(int expectedPartitions) {
- this.expectedPartitions = expectedPartitions;
- }
-
- @Override
- public int partition(
- Tuple2<Integer, Integer> next,
- byte[] key,
- byte[] value,
- String targetTopic,
- int[] partitions) {
- if (partitions.length != expectedPartitions) {
- throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
- }
-
- return next.f0;
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
deleted file mode 100644
index 7b0faf6..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.testutils;
-
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.test.util.SuccessException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.List;
-
-/** A {@link RichSinkFunction} that verifies that no duplicate records are generated. */
-public class ValidatingExactlyOnceSink extends RichSinkFunction<Integer>
- implements ListCheckpointed<Tuple2<Integer, BitSet>>, Runnable, CheckpointListener {
-
- private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class);
-
- private static final long serialVersionUID = 1748426382527469932L;
-
- private final int numElementsTotal;
- private final boolean waitForFinalCheckpoint;
-
- private BitSet duplicateChecker = new BitSet(); // this is checkpointed
-
- private int numElements; // this is checkpointed
-
- private Thread printer;
- private volatile boolean printerRunning = true;
-
- public ValidatingExactlyOnceSink(int numElementsTotal) {
- this(numElementsTotal, false);
- }
-
- public ValidatingExactlyOnceSink(int numElementsTotal, boolean waitForFinalCheckpoint) {
- this.numElementsTotal = numElementsTotal;
- this.waitForFinalCheckpoint = waitForFinalCheckpoint;
- }
-
- @Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- printer = new Thread(this, "Validating Sink Status Printer");
- printer.start();
- }
-
- @Override
- public void invoke(Integer value) throws Exception {
- numElements++;
-
- if (duplicateChecker.get(value)) {
- throw new Exception("Received a duplicate: " + value);
- }
- duplicateChecker.set(value);
- if (!waitForFinalCheckpoint) {
- checkFinish();
- }
- }
-
- @Override
- public List<Tuple2<Integer, BitSet>> snapshotState(long checkpointId, long timestamp)
- throws Exception {
- LOG.info("Snapshot of counter " + numElements + " at checkpoint " + checkpointId);
- return Collections.singletonList(new Tuple2<>(numElements, duplicateChecker));
- }
-
- @Override
- public void restoreState(List<Tuple2<Integer, BitSet>> state) throws Exception {
- if (state.isEmpty() || state.size() > 1) {
- throw new RuntimeException(
- "Test failed due to unexpected recovered state size " + state.size());
- }
-
- Tuple2<Integer, BitSet> s = state.get(0);
- LOG.info("restoring num elements to {}", s.f0);
- this.numElements = s.f0;
- this.duplicateChecker = s.f1;
- }
-
- @Override
- public void close() throws Exception {
- super.close();
-
- printerRunning = false;
- if (printer != null) {
- printer.interrupt();
- printer = null;
- }
- }
-
- @Override
- public void run() {
- while (printerRunning) {
- try {
- Thread.sleep(5000);
- } catch (InterruptedException e) {
- // ignore
- }
- LOG.info(
- "============================> Sink {}: numElements={}, numElementsTotal={}",
- getRuntimeContext().getIndexOfThisSubtask(),
- numElements,
- numElementsTotal);
- }
- }
-
- @Override
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
- checkFinish();
- }
-
- private void checkFinish() throws Exception {
- if (numElements == numElementsTotal) {
- // validate
- if (duplicateChecker.cardinality() != numElementsTotal) {
- throw new Exception("Duplicate checker has wrong cardinality");
- } else if (duplicateChecker.nextClearBit(0) != numElementsTotal) {
- throw new Exception("Received sparse sequence");
- } else {
- throw new SuccessException();
- }
- }
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/archunit.properties b/flink-connectors/flink-connector-kafka/src/test/resources/archunit.properties
deleted file mode 100644
index 15be88c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/archunit.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# 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.
-#
-
-# By default we allow removing existing violations, but fail when new violations are added.
-freeze.store.default.allowStoreUpdate=true
-
-# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
-#freeze.store.default.allowStoreCreation=true
-
-# Enable this to add allow new violations to be recorded.
-# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
-# violation, please try to avoid creating the violation. If the violation was created due to a
-# shortcoming of the rule, file a JIRA issue so the rule can be improved.
-#freeze.refreeze=true
-
-freeze.store.default.path=archunit-violations
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/canal-data.txt b/flink-connectors/flink-connector-kafka/src/test/resources/canal-data.txt
deleted file mode 100644
index 0f74e28..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/canal-data.txt
+++ /dev/null
@@ -1,11 +0,0 @@
-{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"3.14"},{"id":"102","name":"car battery","description":"12V car battery","weight":"8.1"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"},{"id":"104","name":"hammer","description":"12oz carpenter's hammer","weight":"0.75"},{"id":"105","name":"hammer","description":"14oz carpenter's hammer","weight":"0.875"},{"id":"106","name":"hammer","description":"16oz carpenter's hammer","weight":"1.0"},{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.3"},{"id":"108","name":"jacket","description":"water resistent black wind breaker","weight":"0.1"},{"id":"109","name":"spare tire","description":"24 inch spare tire","weight":"22.2"}],"database":"inventory","es":1589373515000,"id":3,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373515477,"type":"INSERT"}
-{"data":[{"id":"106","name":"hammer","description":"18oz carpenter hammer","weight":"1.0"}],"database":"inventory","es":1589373546000,"id":4,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"16oz carpenter's hammer"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373546301,"type":"UPDATE"}
-{"data":[{"id":"107","name":"rocks","description":"box of assorted rocks","weight":"5.1"}],"database":"inventory","es":1589373549000,"id":5,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.3"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373549489,"type":"UPDATE"}
-{"data":[{"id":"110","name":"jacket","description":"water resistent white wind breaker","weight":"0.2"}],"database":"inventory","es":1589373552000,"id":6,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373552882,"type":"INSERT"}
-{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.18"}],"database":"inventory","es":1589373555000,"id":7,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373555457,"type":"INSERT"}
-{"data":[{"id":"110","name":"jacket","description":"new water resistent white wind breaker","weight":"0.5"}],"database":"inventory","es":1589373558000,"id":8,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"description":"water resistent white wind breaker","weight":"0.2"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373558230,"type":"UPDATE"}
-{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373560000,"id":9,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"5.18"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373560798,"type":"UPDATE"}
-{"data":[{"id":"111","name":"scooter","description":"Big 2-wheel scooter ","weight":"5.17"}],"database":"inventory","es":1589373563000,"id":10,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373563798,"type":"DELETE"}
-{"data":[{"id":"101","name":"scooter","description":"Small 2-wheel scooter","weight":"5.17"},{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"}],"database":"inventory","es":1589373753000,"id":11,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":[{"weight":"3.14"},{"weight":"8.1"}],"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589373753939,"type":"UPDATE"}
-{"data":null,"database":"inventory","es":1589373566000,"id":13,"isDdl":true,"mysqlType":null,"old":null,"pkNames":null,"sql":"CREATE TABLE `xj_`.`user02` (`uid` int(0) NOT NULL,`uname` varchar(255) NULL, PRIMARY KEY (`uid`))","sqlType":null,"table":"user02","ts":1589373566000,"type":"CREATE"}
-{"data":[{"id":"102","name":"car battery","description":"12V car battery","weight":"5.17"},{"id":"103","name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":"0.8"}],"database":"inventory","es":1589374013000,"id":12,"isDdl":false,"mysqlType":{"id":"INTEGER","name":"VARCHAR(255)","description":"VARCHAR(512)","weight":"FLOAT"},"old":null,"pkNames":["id"],"sql":"","sqlType":{"id":4,"name":12,"description":12,"weight":7},"table":"products2","ts":1589374013680,"type":"DELETE"}
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot
deleted file mode 100644
index 85a71dc..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data
deleted file mode 100644
index 3efe488..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/debezium-data-schema-exclude.txt b/flink-connectors/flink-connector-kafka/src/test/resources/debezium-data-schema-exclude.txt
deleted file mode 100644
index 3763369..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/debezium-data-schema-exclude.txt
+++ /dev/null
@@ -1,16 +0,0 @@
-{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null}
-{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null}
-{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null}
-{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null}
-{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null}
-{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null}
-{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null}
-{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null}
-{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null}
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot
deleted file mode 100644
index f3e6c74..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot
deleted file mode 100644
index f80a429..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot
deleted file mode 100644
index 0a22c69..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot
deleted file mode 100644
index c4f5416..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-empty-state-snapshot
deleted file mode 100644
index d575aff..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-snapshot
deleted file mode 100644
index 8e98f43..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.10-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-empty-state-snapshot
deleted file mode 100644
index 135c37b..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-snapshot
deleted file mode 100644
index 652877c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.11-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-empty-state-snapshot
deleted file mode 100644
index 319cccf..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-snapshot
deleted file mode 100644
index ef00af6..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.12-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-empty-state-snapshot
deleted file mode 100644
index b472cdd..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-snapshot
deleted file mode 100644
index ac2ac1c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.13-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-empty-state-snapshot
deleted file mode 100644
index 2a0ccef..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-snapshot
deleted file mode 100644
index f1a94b6..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.14-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot
deleted file mode 100644
index f6d3a17..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot
deleted file mode 100644
index 7039d10..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-empty-state-snapshot
deleted file mode 100644
index 3f9d56d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-snapshot
deleted file mode 100644
index bb0d53c..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.16-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot
deleted file mode 100644
index 128dcc8..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot
deleted file mode 100644
index 44e8e11..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-empty-state-snapshot
deleted file mode 100644
index 14eaf25..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-snapshot
deleted file mode 100644
index ffb7f09..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.8-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-empty-state-snapshot
deleted file mode 100644
index 4a328d2..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-empty-state-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-snapshot
deleted file mode 100644
index 34ef21f..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.9-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.10-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.10-snapshot
deleted file mode 100644
index e7a4f0d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.10-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.11-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.11-snapshot
deleted file mode 100644
index b4d285d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.11-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.12-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.12-snapshot
deleted file mode 100644
index ebaf5e0..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.12-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.13-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.13-snapshot
deleted file mode 100644
index 9b15b9e..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.13-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.14-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.14-snapshot
deleted file mode 100644
index 9a24bce..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.14-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot
deleted file mode 100644
index 35af28d..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.16-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.16-snapshot
deleted file mode 100644
index de58ce3..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.16-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot
deleted file mode 100644
index 183e18e..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.8-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.8-snapshot
deleted file mode 100644
index b3f7632..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.8-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.9-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.9-snapshot
deleted file mode 100644
index c860898..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.9-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-kafka/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 6ab97f2..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,41 +0,0 @@
-################################################################################
-# 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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
-
-logger.kafka.name = kafka
-logger.kafka.level = OFF
-logger.kafka2.name = state.change
-logger.kafka2.level = OFF
-
-logger.zookeeper.name = org.apache.zookeeper
-logger.zookeeper.level = OFF
-logger.I0Itec.name = org.I0Itec
-logger.I0Itec.level = OFF
-
-logger.migration.name = org.apache.flink.test.migration
-logger.migration.level = INFO
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/maxwell-data.txt b/flink-connectors/flink-connector-kafka/src/test/resources/maxwell-data.txt
deleted file mode 100644
index ecba573..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/maxwell-data.txt
+++ /dev/null
@@ -1,20 +0,0 @@
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":1,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":2,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":3,"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":4,"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":5,"data":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1.0}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":6,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"xoffset":7,"data":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.1}}
-{"database":"test","table":"product","type":"insert","ts":1596684883,"xid":7125,"commit":true,"data":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.2}}
-{"database":"test","table":"product","type":"update","ts":1596684893,"xid":7152,"commit":true,"data":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1.0},"old":{"description":"16oz carpenter's hammer"}}
-{"database":"test","table":"product","type":"update","ts":1596684897,"xid":7169,"commit":true,"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.1},"old":{"weight":5.3}}
-{"database":"test","table":"product","type":"insert","ts":1596684900,"xid":7186,"commit":true,"data":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.2}}
-{"database":"test","table":"product","type":"insert","ts":1596684904,"xid":7201,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.18}}
-{"database":"test","table":"product","type":"update","ts":1596684906,"xid":7216,"commit":true,"data":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"old":{"description":"water resistent white wind breaker","weight":0.2}}
-{"database":"test","table":"product","type":"update","ts":1596684912,"xid":7235,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17},"old":{"weight":5.18}}
-{"database":"test","table":"product","type":"delete","ts":1596684914,"xid":7250,"commit":true,"data":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.17}}
-{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"xoffset":0,"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":5.17},"old":{"weight":3.14}}
-{"database":"test","table":"product","type":"update","ts":1596684928,"xid":7291,"commit":true,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17},"old":{"weight":8.1}}
-{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"xoffset":0,"data":{"id":102,"name":"car battery","description":"12V car battery","weight":5.17}}
-{"database":"test","table":"product","type":"delete","ts":1596684938,"xid":7322,"commit":true,"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8}}
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/product_changelog.txt b/flink-connectors/flink-connector-kafka/src/test/resources/product_changelog.txt
deleted file mode 100644
index 530c4de..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/product_changelog.txt
+++ /dev/null
@@ -1,7 +0,0 @@
-{"before":null,"after":{"product_id":"p_001","product_name":"scooter","product_price":11.11},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1601510400000,"transaction":null}
-{"before":null,"after":{"product_id":"p_002","product_name":"basketball","product_price":23.11},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1601510400000,"transaction":null}
-{"before":{"product_id":"p_001","product_name":"scooter","product_price":11.11},"after":{"product_id":"p_001","product_name":"scooter","product_price":12.99},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1601553600000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1601553603456,"transaction":null}
-{"before":{"product_id":"p_002","product_name":"basketball","product_price":23.11},"after":{"product_id":"p_002","product_name":"basketball","product_price":19.99},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1601553600000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1601553603456,"transaction":null}
-{"before":{"product_id":"p_001","product_name":"scooter","product_price":12.99},"after":{"product_id":"p_001","product_name":"scooter","product_price":11.99},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1601575200000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1601575203456,"transaction":null}
-{"before":{"product_id":"p_002","product_name":"basketball","product_price":19.99},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1601575200000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1601575203456,"transaction":null}
-{"before":{"product_id":"p_001","product_name":"scooter","product_price":11.99},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1601576200000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1601575203456,"transaction":null}
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.11/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.12/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.13/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.14/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.15/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.16/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot
deleted file mode 100644
index b95eacb..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data b/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data
deleted file mode 100644
index 0936509..0000000
--- a/flink-connectors/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data
+++ /dev/null
Binary files differ
diff --git a/flink-connectors/flink-sql-connector-kafka/pom.xml b/flink-connectors/flink-sql-connector-kafka/pom.xml
deleted file mode 100644
index edfdb67..0000000
--- a/flink-connectors/flink-sql-connector-kafka/pom.xml
+++ /dev/null
@@ -1,102 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>flink-connectors</artifactId>
- <groupId>org.apache.flink</groupId>
- <version>1.18-SNAPSHOT</version>
- </parent>
-
- <artifactId>flink-sql-connector-kafka</artifactId>
- <name>Flink : Connectors : SQL : Kafka</name>
-
- <packaging>jar</packaging>
-
- <properties>
- <japicmp.skip>true</japicmp.skip>
- </properties>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
- <optional>${flink.markBundledAsOptional}</optional>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-test-utils</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <executions>
- <execution>
- <id>shade-flink</id>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <artifactSet>
- <includes>
- <include>org.apache.flink:flink-connector-base</include>
- <include>org.apache.flink:flink-connector-kafka</include>
- <include>org.apache.kafka:*</include>
- </includes>
- </artifactSet>
- <filters>
- <filter>
- <artifact>org.apache.kafka:*</artifact>
- <excludes>
- <exclude>kafka/kafka-version.properties</exclude>
- <exclude>LICENSE</exclude>
- <!-- Does not contain anything relevant.
- Cites a binary dependency on jersey, but this is neither reflected in the
- dependency graph, nor are any jersey files bundled. -->
- <exclude>NOTICE</exclude>
- <exclude>common/**</exclude>
- </excludes>
- </filter>
- </filters>
- <relocations>
- <relocation>
- <pattern>org.apache.kafka</pattern>
- <shadedPattern>org.apache.flink.kafka.shaded.org.apache.kafka</shadedPattern>
- </relocation>
- </relocations>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
diff --git a/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE
deleted file mode 100644
index 6b0d931..0000000
--- a/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE
+++ /dev/null
@@ -1,9 +0,0 @@
-flink-sql-connector-kafka
-Copyright 2014-2023 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
-- org.apache.kafka:kafka-clients:3.2.3
diff --git a/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java b/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java
deleted file mode 100644
index cb3a761..0000000
--- a/flink-connectors/flink-sql-connector-kafka/src/test/java/org/apache/flink/connectors/kafka/PackagingITCase.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connectors.kafka;
-
-import org.apache.flink.packaging.PackagingTestUtils;
-import org.apache.flink.table.factories.Factory;
-import org.apache.flink.test.resources.ResourceTestUtils;
-
-import org.junit.jupiter.api.Test;
-
-import java.nio.file.Path;
-import java.util.Arrays;
-
-class PackagingITCase {
-
- @Test
- void testPackaging() throws Exception {
- final Path jar = ResourceTestUtils.getResource(".*/flink-sql-connector-kafka[^/]*\\.jar");
-
- PackagingTestUtils.assertJarContainsOnlyFilesMatching(
- jar, Arrays.asList("org/apache/flink/", "META-INF/"));
- PackagingTestUtils.assertJarContainsServiceEntry(jar, Factory.class);
- }
-}
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index eb6ac9f..8f03840 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -40,7 +40,6 @@
<module>flink-connector-hbase-1.4</module>
<module>flink-connector-hbase-2.2</module>
<module>flink-connector-hive</module>
- <module>flink-connector-kafka</module>
<module>flink-connector-base</module>
<module>flink-file-sink-common</module>
<module>flink-connector-files</module>
@@ -85,7 +84,6 @@
<module>flink-sql-connector-hbase-2.2</module>
<module>flink-sql-connector-hive-2.3.9</module>
<module>flink-sql-connector-hive-3.1.3</module>
- <module>flink-sql-connector-kafka</module>
</modules>
</profile>
</profiles>
diff --git a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml
index fea104d..ac64874 100644
--- a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml
+++ b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml
@@ -51,7 +51,7 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml
deleted file mode 100644
index cbdfa8b..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml
+++ /dev/null
@@ -1,247 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <parent>
- <artifactId>flink-end-to-end-tests</artifactId>
- <groupId>org.apache.flink</groupId>
- <version>1.18-SNAPSHOT</version>
- </parent>
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>flink-end-to-end-tests-common-kafka</artifactId>
- <name>Flink : E2E Tests : Common Kafka</name>
-
- <repositories>
- <repository>
- <id>confluent</id>
- <url>https://packages.confluent.io/maven/</url>
- </repository>
- </repositories>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-end-to-end-tests-common</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-test-utils-junit</artifactId>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.junit.jupiter</groupId>
- <artifactId>junit-jupiter</artifactId>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-kafka-test</artifactId>
- <version>${project.version}</version>
- <!-- ensure the test jars are built beforehand -->
- <scope>provided</scope>
- <exclusions>
- <exclusion>
- <!-- For dependency convergence; conflicting dependencies are not loaded at the same time -->
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka-clients</artifactId>
- <version>3.2.3</version>
- </dependency>
-
- <!-- The following dependencies are for connector/format sql-jars that
- we copy using the maven-dependency-plugin. When extending the test
- to cover more connectors/formats, add a dependency here and an entry
- to the dependency-plugin configuration below.
- This ensures that all modules we actually need (as defined by the
- dependency-plugin configuration) are built before this module. -->
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-client-test</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <!-- Used by maven-dependency-plugin -->
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-avro</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <!-- Used by maven-dependency-plugin -->
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-avro-confluent-registry</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.testcontainers</groupId>
- <artifactId>kafka</artifactId>
- </dependency>
-
- <dependency>
- <!-- https://mvnrepository.com/artifact/io.confluent/kafka-avro-serializer -->
- <groupId>io.confluent</groupId>
- <artifactId>kafka-avro-serializer</artifactId>
- <version>7.2.2</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>io.confluent</groupId>
- <artifactId>kafka-schema-registry-client</artifactId>
- <version>7.2.2</version>
- <scope>test</scope>
- <exclusions>
- <exclusion>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka-clients</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.apache.avro</groupId>
- <artifactId>avro</artifactId>
- <version>${avro.version}</version>
- <scope>test</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-test-utils</artifactId>
- <version>${project.version}</version>
- <exclusions>
- <exclusion>
- <groupId>com.google.guava</groupId>
- <artifactId>guava</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <executions>
- <execution>
- <id>copy</id>
- <phase>pre-integration-test</phase>
- <goals>
- <goal>copy</goal>
- </goals>
- </execution>
- </executions>
- <configuration>
- <artifactItems>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-kafka-test</artifactId>
- <version>${project.version}</version>
- <type>jar</type>
- <overWrite>true</overWrite>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
-
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-client-test</artifactId>
- <version>${project.version}</version>
- <destFileName>SqlToolbox.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-avro</artifactId>
- <version>${project.version}</version>
- <destFileName>avro.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-avro-confluent-registry</artifactId>
- <version>${project.version}</version>
- <destFileName>avro-confluent.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-sql-connector-kafka</artifactId>
- <version>${project.version}</version>
- <destFileName>sql-kafka.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
- <destFileName>kafka-connector.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-test-utils</artifactId>
- <version>${project.version}</version>
- <destFileName>flink-connector-testing.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- <artifactItem>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka-clients</artifactId>
- <version>3.2.3</version>
- <destFileName>kafka-clients.jar</destFileName>
- <type>jar</type>
- <outputDirectory>${project.build.directory}/dependencies</outputDirectory>
- </artifactItem>
- </artifactItems>
- </configuration>
- </plugin>
- </plugins>
- </build>
-</project>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaContainerClient.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaContainerClient.java
deleted file mode 100644
index d3f45e0..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaContainerClient.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.core.testutils.CommonTestUtils;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.BytesDeserializer;
-import org.apache.kafka.common.serialization.BytesSerializer;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.utils.Bytes;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/** A utility class that exposes common methods over a {@link KafkaContainer}. */
-public class KafkaContainerClient {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaContainerClient.class);
- private final KafkaContainer container;
-
- public KafkaContainerClient(KafkaContainer container) {
- this.container = container;
- }
-
- public void createTopic(int replicationFactor, int numPartitions, String topic) {
- Map<String, Object> properties = new HashMap<>();
- properties.put(
- CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
- try (AdminClient admin = AdminClient.create(properties)) {
- admin.createTopics(
- Collections.singletonList(
- new NewTopic(topic, numPartitions, (short) replicationFactor)))
- .all()
- .get();
- } catch (Exception e) {
- throw new IllegalStateException(
- String.format(
- "Fail to create topic [%s partitions: %d replication factor: %d].",
- topic, numPartitions, replicationFactor),
- e);
- }
- }
-
- public <T> void sendMessages(String topic, Serializer<T> valueSerializer, T... messages) {
- Properties props = new Properties();
- props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
- props.put(ProducerConfig.ACKS_CONFIG, "all");
-
- try (Producer<Bytes, T> producer =
- new KafkaProducer<>(props, new BytesSerializer(), valueSerializer)) {
- for (T message : messages) {
- producer.send(new ProducerRecord<>(topic, message));
- }
- }
- }
-
- public <T> List<T> readMessages(
- int expectedNumMessages,
- String groupId,
- String topic,
- Deserializer<T> valueDeserializer)
- throws Exception {
- Properties props = new Properties();
- props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
- props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
- props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
- props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-
- final List<T> messages = Collections.synchronizedList(new ArrayList<>(expectedNumMessages));
- try (Consumer<Bytes, T> consumer =
- new KafkaConsumer<>(props, new BytesDeserializer(), valueDeserializer)) {
- waitUntilTopicAvailableThenAssign(topic, consumer, Duration.ofSeconds(60));
- // Keep polling until getting expected number of messages
- final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
- while (deadline.hasTimeLeft() && messages.size() < expectedNumMessages) {
- LOG.info(
- "Waiting for messages. Received {}/{}.",
- messages.size(),
- expectedNumMessages);
- ConsumerRecords<Bytes, T> records = consumer.poll(Duration.ofMillis(1000));
- for (ConsumerRecord<Bytes, T> record : records) {
- messages.add(record.value());
- }
- }
- if (messages.size() != expectedNumMessages) {
- throw new IOException("Could not read expected number of messages.");
- }
- return messages;
- }
- }
-
- private void waitUntilTopicAvailableThenAssign(
- String topic, Consumer<?, ?> consumer, Duration timeout) throws Exception {
- // Wait until reading topic is available
- CommonTestUtils.waitUtil(
- () -> consumer.listTopics(Duration.ofSeconds(1)).containsKey(topic),
- timeout,
- String.format("Cannot get information for topic \"%s\" within timeout", topic));
- // Assign all partitions of the reading topic
- List<PartitionInfo> partitions = consumer.listTopics().get(topic);
- List<TopicPartition> tps = new ArrayList<>();
- partitions.forEach(partition -> tps.add(new TopicPartition(topic, partition.partition())));
- consumer.assign(tps);
- // Seek offsets to beginning in order to consume all records in topic
- consumer.seekToBeginning(tps);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java
deleted file mode 100644
index b0b53b8..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.tests.util.util.FactoryUtils;
-import org.apache.flink.util.ExternalResource;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.List;
-
-/** Generic interface for interacting with Kafka. */
-public interface KafkaResource extends ExternalResource {
-
- /**
- * Creates a topic with the given name, replication factor and number of partitions.
- *
- * @param replicationFactor replication factor
- * @param numPartitions number of partitions
- * @param topic desired topic name
- * @throws IOException
- */
- void createTopic(int replicationFactor, int numPartitions, String topic) throws IOException;
-
- /**
- * Sends the given messages to the given topic.
- *
- * @param topic topic name
- * @param messages messages to send
- * @throws IOException
- */
- void sendMessages(String topic, String... messages) throws IOException;
-
- /**
- * Sends the given keyed messages to the given topic. The messages themselves should contain the
- * specified {@code keySeparator}.
- *
- * @param topic topic name
- * @param keySeparator the separator used to parse key from value in the messages
- * @param messages messages to send
- * @throws IOException
- */
- void sendKeyedMessages(String topic, String keySeparator, String... messages)
- throws IOException;
-
- /**
- * Returns the kafka bootstrap server addresses.
- *
- * @return kafka bootstrap server addresses
- */
- Collection<InetSocketAddress> getBootstrapServerAddresses();
-
- /**
- * Returns the address of Zookeeper.
- *
- * @return zookeeper address
- */
- InetSocketAddress getZookeeperAddress();
-
- /**
- * Reads {@code expectedNumMessages} from the given topic. If we can't read the expected number
- * of messages we throw an exception.
- *
- * @param expectedNumMessages expected number of messages that should be read
- * @param groupId group id to identify consumer
- * @param topic topic name
- * @return read messages
- * @throws IOException
- */
- List<String> readMessage(int expectedNumMessages, String groupId, String topic)
- throws IOException;
-
- /**
- * Modifies the number of partitions for the given topic.
- *
- * @param numPartitions desired number of partitions
- * @param topic topic to modify
- * @throws IOException
- */
- void setNumPartitions(int numPartitions, String topic) throws IOException;
-
- /**
- * Returns the current number of partitions for the given topic.
- *
- * @param topic topic name
- * @return number of partitions for the given topic
- * @throws IOException
- */
- int getNumPartitions(String topic) throws IOException;
-
- /**
- * Returns the current partition offset for the given partition of the given topic.
- *
- * @param topic topic name
- * @param partition topic partition
- * @return partition offset for the given partition
- * @throws IOException
- */
- long getPartitionOffset(String topic, int partition) throws IOException;
-
- /**
- * Returns the configured KafkaResource implementation, or a {@link
- * LocalStandaloneKafkaResource} if none is configured.
- *
- * @return configured KafkaResource, or {@link LocalStandaloneKafkaResource} if none is
- * configured
- */
- static KafkaResource get(final String version) {
- return FactoryUtils.loadAndInvokeFactory(
- KafkaResourceFactory.class,
- factory -> factory.create(version),
- LocalStandaloneKafkaResourceFactory::new);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java
deleted file mode 100644
index eaa310a..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import java.util.Optional;
-
-/** A factory for {@link KafkaResource} implementations. */
-@FunctionalInterface
-public interface KafkaResourceFactory {
-
- /**
- * Returns a {@link KafkaResource} instance. If the instance could not be instantiated (for
- * example, because a mandatory parameter was missing), then an empty {@link Optional} should be
- * returned.
- *
- * @return KafkaResource instance
- * @throws Exception if the instance could not be instantiated
- */
- KafkaResource create(String kafkaVersion) throws Exception;
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java
deleted file mode 100644
index cb2ab0d..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java
+++ /dev/null
@@ -1,452 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.test.util.FileUtils;
-import org.apache.flink.tests.util.AutoClosableProcess;
-import org.apache.flink.tests.util.CommandLineWrapper;
-import org.apache.flink.tests.util.TestUtils;
-import org.apache.flink.tests.util.activation.OperatingSystemRestriction;
-import org.apache.flink.tests.util.cache.DownloadCache;
-import org.apache.flink.util.OperatingSystem;
-
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Consumer;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * {@link KafkaResource} that downloads kafka and sets up a local kafka cluster with the bundled
- * zookeeper.
- */
-public class LocalStandaloneKafkaResource implements KafkaResource {
-
- private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneKafkaResource.class);
- private static final Pattern ZK_DATA_DIR_PATTERN = Pattern.compile(".*(dataDir=).*");
- private static final Pattern KAFKA_LOG_DIR_PATTERN = Pattern.compile(".*(log\\.dirs=).*");
-
- private static final String ZOOKEEPER_HOST = "localhost";
- private static final int ZOOKEEPER_PORT = 2181;
- private static final String ZOOKEEPER_ADDRESS = ZOOKEEPER_HOST + ':' + ZOOKEEPER_PORT;
- private static final String KAFKA_HOST = "localhost";
- private static final int KAFKA_PORT = 9092;
- private static final String KAFKA_ADDRESS = KAFKA_HOST + ':' + KAFKA_PORT;
-
- private final TemporaryFolder tmp = new TemporaryFolder();
-
- private final DownloadCache downloadCache = DownloadCache.get();
- private final String kafkaVersion;
- private Path kafkaDir;
- @Nullable private Path logBackupDirectory;
-
- LocalStandaloneKafkaResource(final String kafkaVersion, @Nullable Path logBackupDirectory) {
- OperatingSystemRestriction.forbid(
- String.format(
- "The %s relies on UNIX utils and shell scripts.",
- getClass().getSimpleName()),
- OperatingSystem.WINDOWS);
- this.kafkaVersion = kafkaVersion;
- this.logBackupDirectory = logBackupDirectory;
- }
-
- private static String getKafkaDownloadUrl(final String kafkaVersion) {
- return String.format(
- "https://archive.apache.org/dist/kafka/%s/kafka_2.11-%s.tgz",
- kafkaVersion, kafkaVersion);
- }
-
- @Override
- public void before() throws Exception {
- tmp.create();
- downloadCache.before();
-
- this.kafkaDir = tmp.newFolder("kafka").toPath().toAbsolutePath();
- setupKafkaDist();
- setupKafkaCluster();
- }
-
- private void setupKafkaDist() throws IOException {
- final Path downloadDirectory = tmp.newFolder("getOrDownload").toPath();
- final Path kafkaArchive =
- downloadCache.getOrDownload(getKafkaDownloadUrl(kafkaVersion), downloadDirectory);
-
- LOG.info("Kafka location: {}", kafkaDir.toAbsolutePath());
- AutoClosableProcess.runBlocking(
- CommandLineWrapper.tar(kafkaArchive)
- .extract()
- .zipped()
- .strip(1)
- .targetDir(kafkaDir)
- .build());
-
- LOG.info("Updating ZooKeeper properties");
- FileUtils.replace(
- kafkaDir.resolve(Paths.get("config", "zookeeper.properties")),
- ZK_DATA_DIR_PATTERN,
- matcher ->
- matcher.replaceAll("$1" + kafkaDir.resolve("zookeeper").toAbsolutePath()));
-
- LOG.info("Updating Kafka properties");
- FileUtils.replace(
- kafkaDir.resolve(Paths.get("config", "server.properties")),
- KAFKA_LOG_DIR_PATTERN,
- matcher -> matcher.replaceAll("$1" + kafkaDir.resolve("kafka").toAbsolutePath()));
- }
-
- private void setupKafkaCluster() throws IOException {
- LOG.info("Starting zookeeper");
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "zookeeper-server-start.sh")).toString(),
- "-daemon",
- kafkaDir.resolve(Paths.get("config", "zookeeper.properties")).toString());
- LOG.info("Starting kafka");
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "kafka-server-start.sh")).toString(),
- "-daemon",
- kafkaDir.resolve(Paths.get("config", "server.properties")).toString());
-
- while (!isZookeeperRunning(kafkaDir)) {
- try {
- LOG.info("Waiting for ZooKeeper to start.");
- Thread.sleep(500L);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- while (!isKafkaRunning(kafkaDir)) {
- try {
- LOG.info("Waiting for Kafka to start.");
- Thread.sleep(500L);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- }
-
- @Override
- public void afterTestSuccess() {
- shutdownResource();
- downloadCache.afterTestSuccess();
- tmp.delete();
- }
-
- @Override
- public void afterTestFailure() {
- shutdownResource();
- backupLogs();
- downloadCache.afterTestFailure();
- tmp.delete();
- }
-
- private void shutdownResource() {
- try {
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "kafka-server-stop.sh")).toString());
- while (isKafkaRunning(kafkaDir)) {
- try {
- Thread.sleep(500L);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- } catch (IOException ioe) {
- LOG.warn("Error while shutting down kafka.", ioe);
- }
- try {
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "zookeeper-server-stop.sh")).toString());
- while (isZookeeperRunning(kafkaDir)) {
- try {
- Thread.sleep(500L);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- } catch (IOException ioe) {
- LOG.warn("Error while shutting down zookeeper.", ioe);
- }
- }
-
- private void backupLogs() {
- if (logBackupDirectory != null) {
- final Path targetDirectory =
- logBackupDirectory.resolve("kafka-" + UUID.randomUUID().toString());
- try {
- Files.createDirectories(targetDirectory);
- TestUtils.copyDirectory(kafkaDir.resolve("logs"), targetDirectory);
- LOG.info("Backed up logs to {}.", targetDirectory);
- } catch (IOException e) {
- LOG.warn("An error has occurred while backing up logs to {}.", targetDirectory, e);
- }
- }
- }
-
- private static boolean isZookeeperRunning(final Path kafkaDir) {
- try {
- queryBrokerStatus(kafkaDir, line -> {});
- return true;
- } catch (final IOException ioe) {
- // we get an exception if zookeeper isn't running
- return false;
- }
- }
-
- private static boolean isKafkaRunning(final Path kafkaDir) throws IOException {
- try {
- final AtomicBoolean atomicBrokerStarted = new AtomicBoolean(false);
- queryBrokerStatus(
- kafkaDir,
- line -> {
- atomicBrokerStarted.compareAndSet(false, line.contains("\"port\":"));
- });
- return atomicBrokerStarted.get();
- } catch (final IOException ioe) {
- // we get an exception if zookeeper isn't running
- return false;
- }
- }
-
- private static void queryBrokerStatus(
- final Path kafkaDir, final Consumer<String> stderrProcessor) throws IOException {
- AutoClosableProcess.create(
- kafkaDir.resolve(Paths.get("bin", "zookeeper-shell.sh")).toString(),
- ZOOKEEPER_ADDRESS,
- "get",
- "/brokers/ids/0")
- .setStdoutProcessor(stderrProcessor)
- .runBlocking();
- }
-
- @Override
- public void createTopic(int replicationFactor, int numPartitions, String topic)
- throws IOException {
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(),
- "--create",
- "--zookeeper",
- ZOOKEEPER_ADDRESS,
- "--replication-factor",
- String.valueOf(replicationFactor),
- "--partitions",
- String.valueOf(numPartitions),
- "--topic",
- topic);
- }
-
- @Override
- public void sendMessages(String topic, String... messages) throws IOException {
- List<String> args = createSendMessageArguments(topic);
- sendMessagesAndWait(args, messages);
- }
-
- @Override
- public void sendKeyedMessages(String topic, String keySeparator, String... messages)
- throws IOException {
- List<String> args = new ArrayList<>(createSendMessageArguments(topic));
- args.add("--property");
- args.add("parse.key=true");
- args.add("--property");
- args.add("key.separator=" + keySeparator);
-
- sendMessagesAndWait(args, messages);
- }
-
- private List<String> createSendMessageArguments(String topic) {
- return Arrays.asList(
- kafkaDir.resolve(Paths.get("bin", "kafka-console-producer.sh")).toString(),
- "--broker-list",
- KAFKA_ADDRESS,
- "--topic",
- topic);
- }
-
- private void sendMessagesAndWait(List<String> kafkaArgs, String... messages)
- throws IOException {
- try (AutoClosableProcess autoClosableProcess =
- AutoClosableProcess.runNonBlocking(kafkaArgs.toArray(new String[0]))) {
-
- try (PrintStream printStream =
- new PrintStream(
- autoClosableProcess.getProcess().getOutputStream(),
- true,
- StandardCharsets.UTF_8.name())) {
- for (final String message : messages) {
- printStream.println(message);
- }
- printStream.flush();
- }
-
- try {
- // wait until the process shuts down on it's own
- // this is the only reliable way to ensure the producer has actually processed our
- // input
- autoClosableProcess.getProcess().waitFor();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
-
- @Override
- public List<String> readMessage(int expectedNumMessages, String groupId, String topic)
- throws IOException {
- final List<String> messages =
- Collections.synchronizedList(new ArrayList<>(expectedNumMessages));
-
- try (final AutoClosableProcess kafka =
- AutoClosableProcess.create(
- kafkaDir.resolve(Paths.get("bin", "kafka-console-consumer.sh"))
- .toString(),
- "--bootstrap-server",
- KAFKA_ADDRESS,
- "--from-beginning",
- "--max-messages",
- String.valueOf(expectedNumMessages),
- "--topic",
- topic,
- "--consumer-property",
- "group.id=" + groupId)
- .setStdoutProcessor(messages::add)
- .runNonBlocking()) {
-
- final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
- while (deadline.hasTimeLeft() && messages.size() < expectedNumMessages) {
- try {
- LOG.info(
- "Waiting for messages. Received {}/{}.",
- messages.size(),
- expectedNumMessages);
- Thread.sleep(500);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- if (messages.size() != expectedNumMessages) {
- throw new IOException("Could not read expected number of messages.");
- }
- return messages;
- }
- }
-
- @Override
- public void setNumPartitions(int numPartitions, String topic) throws IOException {
- AutoClosableProcess.runBlocking(
- kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(),
- "--alter",
- "--topic",
- topic,
- "--partitions",
- String.valueOf(numPartitions),
- "--zookeeper",
- ZOOKEEPER_ADDRESS);
- }
-
- @Override
- public int getNumPartitions(String topic) throws IOException {
- final Pattern partitionCountPattern = Pattern.compile(".*PartitionCount:\\s*([0-9]+).*");
- final AtomicReference<Integer> partitionCountFound = new AtomicReference<>(-1);
- AutoClosableProcess.create(
- kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(),
- "--describe",
- "--topic",
- topic,
- "--zookeeper",
- ZOOKEEPER_ADDRESS)
- .setStdoutProcessor(
- line -> {
- final Matcher matcher = partitionCountPattern.matcher(line);
- if (matcher.matches()) {
- partitionCountFound.compareAndSet(
- -1, Integer.parseInt(matcher.group(1)));
- }
- })
- .runBlocking();
- return partitionCountFound.get();
- }
-
- @Override
- public long getPartitionOffset(String topic, int partition) throws IOException {
- final Pattern partitionOffsetPattern = Pattern.compile(".*:.*:([0-9]+)");
- final AtomicReference<Integer> partitionOffsetFound = new AtomicReference<>(-1);
- AutoClosableProcess.create(
- kafkaDir.resolve(Paths.get("bin", "kafka-run-class.sh")).toString(),
- "kafka.tools.GetOffsetShell",
- "--broker-list",
- KAFKA_ADDRESS,
- "--topic",
- topic,
- "--partitions",
- String.valueOf(partition),
- "--time",
- "-1")
- .setStdoutProcessor(
- line -> {
- final Matcher matcher = partitionOffsetPattern.matcher(line);
- if (matcher.matches()) {
- partitionOffsetFound.compareAndSet(
- -1, Integer.parseInt(matcher.group(1)));
- }
- })
- .runBlocking();
-
- final int partitionOffset = partitionOffsetFound.get();
- if (partitionOffset == -1) {
- throw new IOException("Could not determine partition offset.");
- }
- return partitionOffset;
- }
-
- @Override
- public Collection<InetSocketAddress> getBootstrapServerAddresses() {
- return Collections.singletonList(
- InetSocketAddress.createUnresolved(KAFKA_HOST, KAFKA_PORT));
- }
-
- @Override
- public InetSocketAddress getZookeeperAddress() {
- return InetSocketAddress.createUnresolved(KAFKA_HOST, KAFKA_PORT);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java
deleted file mode 100644
index afad6ba..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.test.parameters.ParameterProperty;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.Optional;
-
-/** A {@link KafkaResourceFactory} for the {@link LocalStandaloneKafkaResourceFactory}. */
-public final class LocalStandaloneKafkaResourceFactory implements KafkaResourceFactory {
- private static final Logger LOG =
- LoggerFactory.getLogger(LocalStandaloneKafkaResourceFactory.class);
-
- private static final ParameterProperty<Path> DISTRIBUTION_LOG_BACKUP_DIRECTORY =
- new ParameterProperty<>("logBackupDir", Paths::get);
-
- @Override
- public KafkaResource create(final String kafkaVersion) {
- Optional<Path> logBackupDirectory = DISTRIBUTION_LOG_BACKUP_DIRECTORY.get();
- if (!logBackupDirectory.isPresent()) {
- LOG.warn(
- "Property {} not set, logs will not be backed up in case of test failures.",
- DISTRIBUTION_LOG_BACKUP_DIRECTORY.getPropertyName());
- }
- return new LocalStandaloneKafkaResource(kafkaVersion, logBackupDirectory.orElse(null));
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java
deleted file mode 100644
index b22e8a3..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory;
-import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment;
-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.SinkTestSuiteBase;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.util.DockerImageVersions;
-
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.util.Arrays;
-
-/** Kafka sink E2E test based on connector testing framework. */
-@SuppressWarnings("unused")
-public class KafkaSinkE2ECase extends SinkTestSuiteBase<String> {
- private static final String KAFKA_HOSTNAME = "kafka";
-
- @TestSemantics
- CheckpointingMode[] semantics =
- new CheckpointingMode[] {
- CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE
- };
-
- // Defines TestEnvironment
- @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment(1, 6);
-
- // Defines ConnectorExternalSystem
- @TestExternalSystem
- DefaultContainerizedExternalSystem<KafkaContainer> kafka =
- DefaultContainerizedExternalSystem.builder()
- .fromContainer(
- new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA))
- .withNetworkAliases(KAFKA_HOSTNAME))
- .bindWithFlinkContainer(flink.getFlinkContainers().getJobManager())
- .build();
-
- // Defines 2 External context Factories, so test cases will be invoked twice using these two
- // kinds of external contexts.
- @TestContext
- KafkaSinkExternalContextFactory contextFactory =
- new KafkaSinkExternalContextFactory(
- kafka.getContainer(),
- Arrays.asList(
- ResourceTestUtils.getResource("kafka-connector.jar")
- .toAbsolutePath()
- .toUri()
- .toURL(),
- ResourceTestUtils.getResource("kafka-clients.jar")
- .toAbsolutePath()
- .toUri()
- .toURL(),
- ResourceTestUtils.getResource("flink-connector-testing.jar")
- .toAbsolutePath()
- .toUri()
- .toURL()));
-
- public KafkaSinkE2ECase() throws Exception {}
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java
deleted file mode 100644
index 4a036df..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContextFactory;
-import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment;
-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.streaming.api.CheckpointingMode;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.util.DockerImageVersions;
-
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.util.Arrays;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION;
-import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC;
-
-/** Kafka E2E test based on connector testing framework. */
-public class KafkaSourceE2ECase extends SourceTestSuiteBase<String> {
- private static final String KAFKA_HOSTNAME = "kafka";
-
- @TestSemantics
- CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
-
- // Defines TestEnvironment
- @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment(1, 6);
-
- // Defines ConnectorExternalSystem
- @TestExternalSystem
- DefaultContainerizedExternalSystem<KafkaContainer> kafka =
- DefaultContainerizedExternalSystem.builder()
- .fromContainer(
- new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA))
- .withNetworkAliases(KAFKA_HOSTNAME))
- .bindWithFlinkContainer(flink.getFlinkContainers().getJobManager())
- .build();
-
- // Defines 2 External context Factories, so test cases will be invoked twice using these two
- // kinds of external contexts.
- @SuppressWarnings("unused")
- @TestContext
- KafkaSourceExternalContextFactory singleTopic =
- new KafkaSourceExternalContextFactory(
- kafka.getContainer(),
- Arrays.asList(
- ResourceTestUtils.getResource("kafka-connector.jar").toUri().toURL(),
- ResourceTestUtils.getResource("kafka-clients.jar").toUri().toURL()),
- PARTITION);
-
- @SuppressWarnings("unused")
- @TestContext
- KafkaSourceExternalContextFactory multipleTopic =
- new KafkaSourceExternalContextFactory(
- kafka.getContainer(),
- Arrays.asList(
- ResourceTestUtils.getResource("kafka-connector.jar").toUri().toURL(),
- ResourceTestUtils.getResource("kafka-clients.jar").toUri().toURL()),
- TOPIC);
-
- public KafkaSourceE2ECase() throws Exception {}
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
deleted file mode 100644
index e8a563e..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.test.util.SQLJobSubmission;
-import org.apache.flink.tests.util.cache.DownloadCache;
-import org.apache.flink.tests.util.flink.ClusterController;
-import org.apache.flink.tests.util.flink.FlinkResource;
-import org.apache.flink.tests.util.flink.FlinkResourceSetup;
-import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.apache.flink.tests.util.TestUtils.readCsvResultFiles;
-import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
-import static org.junit.Assert.assertThat;
-
-/** End-to-end test for the kafka SQL connectors. */
-@RunWith(Parameterized.class)
-@Ignore("FLINK-21796")
-public class SQLClientKafkaITCase extends TestLogger {
-
- private static final Logger LOG = LoggerFactory.getLogger(SQLClientKafkaITCase.class);
-
- private static final String KAFKA_E2E_SQL = "kafka_e2e.sql";
-
- @Parameterized.Parameters(name = "{index}: kafka-version:{0} kafka-sql-version:{1}")
- public static Collection<Object[]> data() {
- return Arrays.asList(new Object[][] {{"3.2.3", "universal", "kafka", ".*kafka.jar"}});
- }
-
- private static Configuration getConfiguration() {
- // we have to enable checkpoint to trigger flushing for filesystem sink
- final Configuration flinkConfig = new Configuration();
- flinkConfig.setString("execution.checkpointing.interval", "5s");
- return flinkConfig;
- }
-
- @Rule
- public final FlinkResource flink =
- new LocalStandaloneFlinkResourceFactory()
- .create(
- FlinkResourceSetup.builder()
- .addConfiguration(getConfiguration())
- .build());
-
- @Rule public final KafkaResource kafka;
-
- @Rule public final TemporaryFolder tmp = new TemporaryFolder();
-
- private final String kafkaVersion;
- private final String kafkaSQLVersion;
- private final String kafkaIdentifier;
- private Path result;
-
- @ClassRule public static final DownloadCache DOWNLOAD_CACHE = DownloadCache.get();
-
- private static final Path sqlAvroJar = ResourceTestUtils.getResource(".*avro.jar");
- private static final Path sqlToolBoxJar = ResourceTestUtils.getResource(".*SqlToolbox.jar");
- private final List<Path> apacheAvroJars = new ArrayList<>();
- private final Path sqlConnectorKafkaJar;
-
- public SQLClientKafkaITCase(
- String kafkaVersion,
- String kafkaSQLVersion,
- String kafkaIdentifier,
- String kafkaSQLJarPattern) {
- this.kafka = KafkaResource.get(kafkaVersion);
- this.kafkaVersion = kafkaVersion;
- this.kafkaSQLVersion = kafkaSQLVersion;
- this.kafkaIdentifier = kafkaIdentifier;
-
- this.sqlConnectorKafkaJar = ResourceTestUtils.getResource(kafkaSQLJarPattern);
- }
-
- @Before
- public void before() throws Exception {
- DOWNLOAD_CACHE.before();
- Path tmpPath = tmp.getRoot().toPath();
- LOG.info("The current temporary path: {}", tmpPath);
- this.result = tmpPath.resolve("result");
- }
-
- @Test
- public void testKafka() throws Exception {
- try (ClusterController clusterController = flink.startCluster(2)) {
- // Create topic and send message
- String testJsonTopic = "test-json-" + kafkaVersion + "-" + UUID.randomUUID().toString();
- String testAvroTopic = "test-avro-" + kafkaVersion + "-" + UUID.randomUUID().toString();
- kafka.createTopic(1, 1, testJsonTopic);
- String[] messages =
- new String[] {
- "{\"rowtime\": \"2018-03-12 08:00:00\", \"user\": \"Alice\", \"event\": { \"type\": \"WARNING\", \"message\": \"This is a warning.\"}}",
- "{\"rowtime\": \"2018-03-12 08:10:00\", \"user\": \"Alice\", \"event\": { \"type\": \"WARNING\", \"message\": \"This is a warning.\"}}",
- "{\"rowtime\": \"2018-03-12 09:00:00\", \"user\": \"Bob\", \"event\": { \"type\": \"WARNING\", \"message\": \"This is another warning.\"}}",
- "{\"rowtime\": \"2018-03-12 09:10:00\", \"user\": \"Alice\", \"event\": { \"type\": \"INFO\", \"message\": \"This is a info.\"}}",
- "{\"rowtime\": \"2018-03-12 09:20:00\", \"user\": \"Steve\", \"event\": { \"type\": \"INFO\", \"message\": \"This is another info.\"}}",
- "{\"rowtime\": \"2018-03-12 09:30:00\", \"user\": \"Steve\", \"event\": { \"type\": \"INFO\", \"message\": \"This is another info.\"}}",
- "{\"rowtime\": \"2018-03-12 09:30:00\", \"user\": null, \"event\": { \"type\": \"WARNING\", \"message\": \"This is a bad message because the user is missing.\"}}",
- "{\"rowtime\": \"2018-03-12 10:40:00\", \"user\": \"Bob\", \"event\": { \"type\": \"ERROR\", \"message\": \"This is an error.\"}}"
- };
- kafka.sendMessages(testJsonTopic, messages);
-
- // Create topic test-avro
- kafka.createTopic(1, 1, testAvroTopic);
-
- // Initialize the SQL statements from "kafka_e2e.sql" file
- Map<String, String> varsMap = new HashMap<>();
- varsMap.put("$KAFKA_IDENTIFIER", this.kafkaIdentifier);
- varsMap.put("$TOPIC_JSON_NAME", testJsonTopic);
- varsMap.put("$TOPIC_AVRO_NAME", testAvroTopic);
- varsMap.put("$RESULT", this.result.toAbsolutePath().toString());
- varsMap.put(
- "$KAFKA_BOOTSTRAP_SERVERS",
- StringUtils.join(kafka.getBootstrapServerAddresses().toArray(), ","));
- List<String> sqlLines = initializeSqlLines(varsMap);
-
- // Execute SQL statements in "kafka_e2e.sql" file
- executeSqlStatements(clusterController, sqlLines);
-
- // Wait until all the results flushed to the CSV file.
- LOG.info("Verify the CSV result.");
- checkCsvResultFile();
- LOG.info("The Kafka({}) SQL client test run successfully.", this.kafkaSQLVersion);
- }
- }
-
- private void executeSqlStatements(ClusterController clusterController, List<String> sqlLines)
- throws Exception {
- LOG.info("Executing Kafka {} end-to-end SQL statements.", kafkaSQLVersion);
- clusterController.submitSQLJob(
- new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
- .addJar(sqlAvroJar)
- .addJars(apacheAvroJars)
- .addJar(sqlConnectorKafkaJar)
- .addJar(sqlToolBoxJar)
- .build(),
- Duration.ofMinutes(2L));
- }
-
- private List<String> initializeSqlLines(Map<String, String> vars) throws IOException {
- URL url = SQLClientKafkaITCase.class.getClassLoader().getResource(KAFKA_E2E_SQL);
- if (url == null) {
- throw new FileNotFoundException(KAFKA_E2E_SQL);
- }
-
- List<String> lines = Files.readAllLines(new File(url.getFile()).toPath());
- List<String> result = new ArrayList<>();
- for (String line : lines) {
- for (Map.Entry<String, String> var : vars.entrySet()) {
- line = line.replace(var.getKey(), var.getValue());
- }
- result.add(line);
- }
-
- return result;
- }
-
- private void checkCsvResultFile() throws Exception {
- boolean success = false;
- final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
- while (deadline.hasTimeLeft()) {
- if (Files.exists(result)) {
- List<String> lines = readCsvResultFiles(result);
- if (lines.size() == 4) {
- success = true;
- assertThat(
- lines.toArray(new String[0]),
- arrayContainingInAnyOrder(
- "2018-03-12 08:00:00.000,Alice,This was a warning.,2,Success constant folding.",
- "2018-03-12 09:00:00.000,Bob,This was another warning.,1,Success constant folding.",
- "2018-03-12 09:00:00.000,Steve,This was another info.,2,Success constant folding.",
- "2018-03-12 09:00:00.000,Alice,This was a info.,1,Success constant folding."));
- break;
- } else {
- LOG.info(
- "The target CSV {} does not contain enough records, current {} records, left time: {}s",
- result,
- lines.size(),
- deadline.timeLeft().getSeconds());
- }
- } else {
- LOG.info("The target CSV {} does not exist now", result);
- }
- Thread.sleep(500);
- }
- Assert.assertTrue("Did not get expected results before timeout.", success);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java
deleted file mode 100644
index ce81d88..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.connector.testframe.container.FlinkContainers;
-import org.apache.flink.connector.testframe.container.TestcontainersSettings;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.test.util.SQLJobSubmission;
-import org.apache.flink.tests.util.kafka.containers.SchemaRegistryContainer;
-import org.apache.flink.util.DockerImageVersions;
-
-import io.confluent.kafka.schemaregistry.avro.AvroSchema;
-import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
-import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
-import io.confluent.kafka.serializers.KafkaAvroDeserializer;
-import io.confluent.kafka.serializers.KafkaAvroSerializer;
-import org.apache.avro.Schema;
-import org.apache.avro.SchemaBuilder;
-import org.apache.avro.generic.GenericRecordBuilder;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-import org.testcontainers.containers.output.Slf4jLogConsumer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.nio.file.Path;
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.assertThat;
-
-/** End-to-end test for SQL client using Avro Confluent Registry format. */
-public class SQLClientSchemaRegistryITCase {
- private static final Logger LOG = LoggerFactory.getLogger(SQLClientSchemaRegistryITCase.class);
- private static final Slf4jLogConsumer LOG_CONSUMER = new Slf4jLogConsumer(LOG);
-
- public static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- public static final String INTER_CONTAINER_REGISTRY_ALIAS = "registry";
- private static final Path sqlAvroJar = ResourceTestUtils.getResource(".*avro.jar");
- private static final Path sqlAvroRegistryJar =
- ResourceTestUtils.getResource(".*avro-confluent.jar");
- private static final Path sqlToolBoxJar = ResourceTestUtils.getResource(".*SqlToolbox.jar");
- private final Path sqlConnectorKafkaJar = ResourceTestUtils.getResource(".*kafka.jar");
-
- @ClassRule public static final Network NETWORK = Network.newNetwork();
-
- @ClassRule public static final Timeout TIMEOUT = new Timeout(10, TimeUnit.MINUTES);
-
- @ClassRule
- public static final KafkaContainer KAFKA =
- new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA))
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS)
- .withLogConsumer(LOG_CONSUMER);
-
- @ClassRule
- public static final SchemaRegistryContainer REGISTRY =
- new SchemaRegistryContainer(DockerImageName.parse(DockerImageVersions.SCHEMA_REGISTRY))
- .withKafka(INTER_CONTAINER_KAFKA_ALIAS + ":9092")
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_REGISTRY_ALIAS)
- .dependsOn(KAFKA);
-
- public final TestcontainersSettings testcontainersSettings =
- TestcontainersSettings.builder().network(NETWORK).logger(LOG).dependsOn(KAFKA).build();
-
- public final FlinkContainers flink =
- FlinkContainers.builder().withTestcontainersSettings(testcontainersSettings).build();
-
- private KafkaContainerClient kafkaClient;
- private CachedSchemaRegistryClient registryClient;
-
- @Before
- public void setUp() throws Exception {
- flink.start();
- kafkaClient = new KafkaContainerClient(KAFKA);
- registryClient = new CachedSchemaRegistryClient(REGISTRY.getSchemaRegistryUrl(), 10);
- }
-
- @After
- public void tearDown() {
- flink.stop();
- }
-
- @Test
- public void testReading() throws Exception {
- String testCategoryTopic = "test-category-" + UUID.randomUUID().toString();
- String testResultsTopic = "test-results-" + UUID.randomUUID().toString();
- kafkaClient.createTopic(1, 1, testCategoryTopic);
- Schema categoryRecord =
- SchemaBuilder.record("org.apache.flink.avro.generated.record")
- .fields()
- .requiredLong("category_id")
- .optionalString("name")
- .endRecord();
- String categorySubject = testCategoryTopic + "-value";
- registryClient.register(categorySubject, new AvroSchema(categoryRecord));
- GenericRecordBuilder categoryBuilder = new GenericRecordBuilder(categoryRecord);
- KafkaAvroSerializer valueSerializer = new KafkaAvroSerializer(registryClient);
- kafkaClient.sendMessages(
- testCategoryTopic,
- valueSerializer,
- categoryBuilder.set("category_id", 1L).set("name", "electronics").build());
-
- List<String> sqlLines =
- Arrays.asList(
- "CREATE TABLE category (",
- " category_id BIGINT,",
- " name STRING,",
- " description STRING", // new field, should create new schema version, but
- // still should
- // be able to read old version
- ") WITH (",
- " 'connector' = 'kafka',",
- " 'properties.bootstrap.servers' = '"
- + INTER_CONTAINER_KAFKA_ALIAS
- + ":9092',",
- " 'topic' = '" + testCategoryTopic + "',",
- " 'scan.startup.mode' = 'earliest-offset',",
- " 'properties.group.id' = 'test-group',",
- " 'format' = 'avro-confluent',",
- " 'avro-confluent.url' = 'http://"
- + INTER_CONTAINER_REGISTRY_ALIAS
- + ":8082'",
- ");",
- "",
- "CREATE TABLE results (",
- " category_id BIGINT,",
- " name STRING,",
- " description STRING",
- ") WITH (",
- " 'connector' = 'kafka',",
- " 'properties.bootstrap.servers' = '"
- + INTER_CONTAINER_KAFKA_ALIAS
- + ":9092',",
- " 'properties.group.id' = 'test-group',",
- " 'topic' = '" + testResultsTopic + "',",
- " 'format' = 'csv',",
- " 'csv.null-literal' = 'null'",
- ");",
- "",
- "INSERT INTO results SELECT * FROM category;");
-
- executeSqlStatements(sqlLines);
- List<String> categories =
- kafkaClient.readMessages(
- 1, "test-group", testResultsTopic, new StringDeserializer());
- assertThat(categories, equalTo(Collections.singletonList("1,electronics,null")));
- }
-
- @Test
- public void testWriting() throws Exception {
- String testUserBehaviorTopic = "test-user-behavior-" + UUID.randomUUID().toString();
- // Create topic test-avro
- kafkaClient.createTopic(1, 1, testUserBehaviorTopic);
-
- String behaviourSubject = testUserBehaviorTopic + "-value";
- List<String> sqlLines =
- Arrays.asList(
- "CREATE TABLE user_behavior (",
- " user_id BIGINT NOT NULL,",
- " item_id BIGINT,",
- " category_id BIGINT,",
- " behavior STRING,",
- " ts TIMESTAMP(3)",
- ") WITH (",
- " 'connector' = 'kafka',",
- " 'properties.bootstrap.servers' = '"
- + INTER_CONTAINER_KAFKA_ALIAS
- + ":9092',",
- " 'topic' = '" + testUserBehaviorTopic + "',",
- " 'format' = 'avro-confluent',",
- " 'avro-confluent.url' = 'http://"
- + INTER_CONTAINER_REGISTRY_ALIAS
- + ":8082"
- + "'",
- ");",
- "",
- "INSERT INTO user_behavior VALUES (1, 1, 1, 'buy', TO_TIMESTAMP(FROM_UNIXTIME(1234)));");
-
- executeSqlStatements(sqlLines);
-
- List<Integer> versions = getAllVersions(behaviourSubject);
- assertThat(versions.size(), equalTo(1));
- List<Object> userBehaviors =
- kafkaClient.readMessages(
- 1,
- "test-group",
- testUserBehaviorTopic,
- new KafkaAvroDeserializer(registryClient));
-
- String schemaString =
- registryClient.getByVersion(behaviourSubject, versions.get(0), false).getSchema();
- Schema userBehaviorSchema = new Schema.Parser().parse(schemaString);
- GenericRecordBuilder recordBuilder = new GenericRecordBuilder(userBehaviorSchema);
- assertThat(
- userBehaviors,
- equalTo(
- Collections.singletonList(
- recordBuilder
- .set("user_id", 1L)
- .set("item_id", 1L)
- .set("category_id", 1L)
- .set("behavior", "buy")
- .set("ts", 1234000L)
- .build())));
- }
-
- private List<Integer> getAllVersions(String behaviourSubject) throws Exception {
- Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
- Exception ex =
- new IllegalStateException(
- "Could not query schema registry. Negative deadline provided.");
- while (deadline.hasTimeLeft()) {
- try {
- return registryClient.getAllVersions(behaviourSubject);
- } catch (RestClientException e) {
- ex = e;
- }
- }
- throw ex;
- }
-
- private void executeSqlStatements(List<String> sqlLines) throws Exception {
- flink.submitSQLJob(
- new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
- .addJars(
- sqlAvroJar, sqlAvroRegistryJar, sqlConnectorKafkaJar, sqlToolBoxJar)
- .build());
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java
deleted file mode 100644
index 726ecee..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.connector.kafka.testutils.KafkaUtil;
-import org.apache.flink.connector.testframe.container.FlinkContainers;
-import org.apache.flink.connector.testframe.container.FlinkContainersSettings;
-import org.apache.flink.connector.testframe.container.TestcontainersSettings;
-import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.test.util.JobSubmission;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.admin.AdminClient;
-import org.apache.kafka.clients.admin.NewTopic;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-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.serialization.IntegerDeserializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
-import org.apache.kafka.common.serialization.VoidDeserializer;
-import org.apache.kafka.common.serialization.VoidSerializer;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junit.jupiter.api.extension.RegisterExtension;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.KafkaContainer;
-import org.testcontainers.containers.Network;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-import java.nio.ByteBuffer;
-import java.nio.file.Path;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer;
-import static org.apache.flink.util.DockerImageVersions.KAFKA;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** smoke test for the kafka connectors. */
-@ExtendWith({TestLoggerExtension.class})
-@Testcontainers
-class SmokeKafkaITCase {
-
- private static final Logger LOG = LoggerFactory.getLogger(SmokeKafkaITCase.class);
- private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
- private static final Network NETWORK = Network.newNetwork();
- private static final String EXAMPLE_JAR_MATCHER = "flink-streaming-kafka-test.*";
-
- @Container
- public static final KafkaContainer KAFKA_CONTAINER =
- createKafkaContainer(KAFKA, LOG)
- .withEmbeddedZookeeper()
- .withNetwork(NETWORK)
- .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS);
-
- public static final TestcontainersSettings TESTCONTAINERS_SETTINGS =
- TestcontainersSettings.builder().logger(LOG).dependsOn(KAFKA_CONTAINER).build();
-
- @RegisterExtension
- public static final FlinkContainers FLINK =
- FlinkContainers.builder()
- .withFlinkContainersSettings(
- FlinkContainersSettings.basedOn(getConfiguration()))
- .withTestcontainersSettings(TESTCONTAINERS_SETTINGS)
- .build();
-
- private static AdminClient admin;
- private static KafkaProducer<Void, Integer> producer;
-
- private static Configuration getConfiguration() {
- // modify configuration to have enough slots
- final Configuration flinkConfig = new Configuration();
- flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3);
- flinkConfig.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true);
- return flinkConfig;
- }
-
- @BeforeAll
- static void setUp() {
- final Map<String, Object> adminProperties = new HashMap<>();
- adminProperties.put(
- CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
- KAFKA_CONTAINER.getBootstrapServers());
- admin = AdminClient.create(adminProperties);
- final Properties producerProperties = new Properties();
- producerProperties.putAll(adminProperties);
- producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, VoidSerializer.class);
- producerProperties.put(
- ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
- producer = new KafkaProducer<>(producerProperties);
- }
-
- @AfterAll
- static void teardown() {
- admin.close();
- producer.close();
- }
-
- @Test
- public void testKafka() throws Exception {
- final Path kafkaExampleJar = ResourceTestUtils.getResource(EXAMPLE_JAR_MATCHER);
-
- final String inputTopic = "test-input-" + "-" + UUID.randomUUID();
- final String outputTopic = "test-output" + "-" + UUID.randomUUID();
-
- // create the required topics
- final short replicationFactor = 1;
- admin.createTopics(
- Lists.newArrayList(
- new NewTopic(inputTopic, 1, replicationFactor),
- new NewTopic(outputTopic, 1, replicationFactor)))
- .all()
- .get();
-
- producer.send(new ProducerRecord<>(inputTopic, 1));
- producer.send(new ProducerRecord<>(inputTopic, 2));
- producer.send(new ProducerRecord<>(inputTopic, 3));
-
- // run the Flink job
- FLINK.submitJob(
- new JobSubmission.JobSubmissionBuilder(kafkaExampleJar)
- .setDetached(false)
- .addArgument("--input-topic", inputTopic)
- .addArgument("--output-topic", outputTopic)
- .addArgument("--prefix", "PREFIX")
- .addArgument(
- "--bootstrap.servers",
- String.join(
- ",",
- KAFKA_CONTAINER.getBootstrapServers(),
- KAFKA_CONTAINER.getNetworkAliases().stream()
- .map(
- host ->
- String.join(
- ":",
- host,
- Integer.toString(9092)))
- .collect(Collectors.joining(","))))
- .addArgument("--group.id", "myconsumer")
- .addArgument("--auto.offset.reset", "earliest")
- .addArgument("--transaction.timeout.ms", "900000")
- .addArgument("--flink.partition-discovery.interval-millis", "1000")
- .build());
- final Properties consumerProperties = new Properties();
- consumerProperties.put(
- CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
- KAFKA_CONTAINER.getBootstrapServers());
- consumerProperties.put(
- ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, VoidDeserializer.class);
- consumerProperties.put(
- ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class);
- List<Integer> records =
- KafkaUtil.drainAllRecordsFromTopic(outputTopic, consumerProperties).stream()
- .map(r -> ByteBuffer.wrap(r.value()).getInt())
- .collect(Collectors.toList());
- assertThat(records).hasSize(3).containsExactly(1, 2, 3);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/containers/SchemaRegistryContainer.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/containers/SchemaRegistryContainer.java
deleted file mode 100644
index e71d7ac..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/containers/SchemaRegistryContainer.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.kafka.containers;
-
-import org.testcontainers.containers.GenericContainer;
-import org.testcontainers.utility.DockerImageName;
-
-/**
- * A container over an Confluent Schema Registry. It runs the schema registry on port 8082 in the
- * docker network so that it does not overlap with Flink cluster.
- */
-public class SchemaRegistryContainer extends GenericContainer<SchemaRegistryContainer> {
-
- public SchemaRegistryContainer(DockerImageName imageName) {
- super(imageName);
- withExposedPorts(8082);
- }
-
- public SchemaRegistryContainer withKafka(String kafkaBootstrapServer) {
- withEnv(
- "SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS",
- "PLAINTEXT://" + kafkaBootstrapServer);
- return this;
- }
-
- @Override
- protected void configure() {
- withEnv("SCHEMA_REGISTRY_HOST_NAME", getNetworkAliases().get(0));
- withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:8082");
- }
-
- public String getSchemaRegistryUrl() {
- return "http://" + getHost() + ":" + getMappedPort(8082);
- }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_e2e.sql b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_e2e.sql
deleted file mode 100644
index 3f830d2..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_e2e.sql
+++ /dev/null
@@ -1,75 +0,0 @@
--- 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.
-
-CREATE TABLE JsonTable (
- rowtime TIMESTAMP(3),
- `user` STRING,
- event ROW<`type` STRING, message STRING>,
- WATERMARK FOR rowtime AS rowtime - INTERVAL '2' SECOND
-) WITH (
- 'connector' = '$KAFKA_IDENTIFIER',
- 'topic' = '$TOPIC_JSON_NAME',
- 'scan.startup.mode' = 'earliest-offset',
- 'properties.bootstrap.servers' = '$KAFKA_BOOTSTRAP_SERVERS',
- 'format' = 'json'
-);
-
-CREATE TABLE AvroTable (
- event_timestamp STRING,
- `user` STRING,
- message STRING,
- duplicate_count BIGINT
-) WITH (
- 'connector' = '$KAFKA_IDENTIFIER',
- 'topic' = '$TOPIC_AVRO_NAME',
- 'scan.startup.mode' = 'earliest-offset',
- 'properties.bootstrap.servers' = '$KAFKA_BOOTSTRAP_SERVERS',
- 'format' = 'avro'
-);
-
-CREATE TABLE CsvTable (
- event_timestamp STRING,
- `user` STRING,
- message STRING,
- duplicate_count BIGINT,
- constant STRING
-) WITH (
- 'connector' = 'filesystem',
- 'path' = '$RESULT',
- 'sink.rolling-policy.rollover-interval' = '2s',
- 'sink.rolling-policy.check-interval' = '2s',
- 'format' = 'csv',
- 'csv.disable-quote-character' = 'true'
-);
-
-CREATE FUNCTION RegReplace AS 'org.apache.flink.table.toolbox.StringRegexReplaceFunction';
-
-INSERT INTO AvroTable -- read from Kafka Json, window aggregation, and write into Kafka Avro
-SELECT
- CAST(TUMBLE_START(rowtime, INTERVAL '1' HOUR) AS VARCHAR) AS event_timestamp,
- `user`,
- RegReplace(event.message, ' is ', ' was ') AS message,
- COUNT(*) AS duplicate_count
-FROM JsonTable
-WHERE `user` IS NOT NULL
-GROUP BY
- `user`,
- event.message,
- TUMBLE(rowtime, INTERVAL '1' HOUR);
-
-INSERT INTO CsvTable -- read from Kafka Avro, and write into Filesystem Csv
-SELECT AvroTable.*, RegReplace('Test constant folding.', 'Test', 'Success') AS constant
-FROM AvroTable;
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 358fd81..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,34 +0,0 @@
-################################################################################
-# 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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
-
-# It is recommended to uncomment these lines when enabling the logger. The below package used
-# by testcontainers is quite verbose
-#logger.yarn.name = org.testcontainers.shaded.com.github.dockerjava.core
-#logger.yarn.level = WARN
-#logger.yarn.appenderRef.console.ref = TestLogger
diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml
index 6e0c494..5e816c6 100644
--- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml
+++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml
@@ -54,7 +54,7 @@
<!-- Used by maven-dependency-plugin -->
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
<scope>provided</scope>
</dependency>
@@ -124,7 +124,7 @@
<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
<type>jar</type>
</artifactItem>
<artifactItem>
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml
deleted file mode 100644
index 2905feb..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml
+++ /dev/null
@@ -1,43 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <parent>
- <artifactId>flink-end-to-end-tests</artifactId>
- <groupId>org.apache.flink</groupId>
- <version>1.18-SNAPSHOT</version>
- </parent>
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>flink-streaming-kafka-test-base</artifactId>
- <name>Flink : E2E Tests : Streaming Kafka base</name>
-
- <packaging>jar</packaging>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
-
-</project>
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java
deleted file mode 100644
index 6093edf..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.kafka.test.base;
-
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
-import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import javax.annotation.Nullable;
-
-/**
- * A custom {@link AssignerWithPeriodicWatermarks}, that simply assumes that the input stream
- * records are strictly ascending.
- *
- * <p>Flink also ships some built-in convenience assigners, such as the {@link
- * BoundedOutOfOrdernessTimestampExtractor} and {@link AscendingTimestampExtractor}
- */
-public class CustomWatermarkExtractor implements AssignerWithPeriodicWatermarks<KafkaEvent> {
-
- private static final long serialVersionUID = -742759155861320823L;
-
- private long currentTimestamp = Long.MIN_VALUE;
-
- @Override
- public long extractTimestamp(KafkaEvent event, long previousElementTimestamp) {
- // the inputs are assumed to be of format (message,timestamp)
- this.currentTimestamp = event.getTimestamp();
- return event.getTimestamp();
- }
-
- @Nullable
- @Override
- public Watermark getCurrentWatermark() {
- return new Watermark(
- currentTimestamp == Long.MIN_VALUE ? Long.MIN_VALUE : currentTimestamp - 1);
- }
-}
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java
deleted file mode 100644
index 9b0971d..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.kafka.test.base;
-
-/**
- * The event type used in the {@link KafkaExample}.
- *
- * <p>This is a Java POJO, which Flink recognizes and will allow "by-name" field referencing when
- * keying a {@link org.apache.flink.streaming.api.datastream.DataStream} of such a type. For a
- * demonstration of this, see the code in {@link KafkaExample}.
- */
-public class KafkaEvent {
-
- private String word;
- private int frequency;
- private long timestamp;
-
- public KafkaEvent() {}
-
- public KafkaEvent(String word, int frequency, long timestamp) {
- this.word = word;
- this.frequency = frequency;
- this.timestamp = timestamp;
- }
-
- public String getWord() {
- return word;
- }
-
- public void setWord(String word) {
- this.word = word;
- }
-
- public int getFrequency() {
- return frequency;
- }
-
- public void setFrequency(int frequency) {
- this.frequency = frequency;
- }
-
- public long getTimestamp() {
- return timestamp;
- }
-
- public void setTimestamp(long timestamp) {
- this.timestamp = timestamp;
- }
-
- public static KafkaEvent fromString(String eventStr) {
- String[] split = eventStr.split(",");
- return new KafkaEvent(split[0], Integer.valueOf(split[1]), Long.valueOf(split[2]));
- }
-
- @Override
- public String toString() {
- return word + "," + frequency + "," + timestamp;
- }
-}
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java
deleted file mode 100644
index ef9a3d0..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.kafka.test.base;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import java.io.IOException;
-
-/**
- * The serialization schema for the {@link KafkaEvent} type. This class defines how to transform a
- * Kafka record's bytes to a {@link KafkaEvent}, and vice-versa.
- */
-public class KafkaEventSchema
- implements DeserializationSchema<KafkaEvent>, SerializationSchema<KafkaEvent> {
-
- private static final long serialVersionUID = 6154188370181669758L;
-
- @Override
- public byte[] serialize(KafkaEvent event) {
- return event.toString().getBytes();
- }
-
- @Override
- public KafkaEvent deserialize(byte[] message) throws IOException {
- return KafkaEvent.fromString(new String(message));
- }
-
- @Override
- public boolean isEndOfStream(KafkaEvent nextElement) {
- return false;
- }
-
- @Override
- public TypeInformation<KafkaEvent> getProducedType() {
- return TypeInformation.of(KafkaEvent.class);
- }
-}
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java
deleted file mode 100644
index a6fa83e..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.kafka.test.base;
-
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-/** The util class for kafka example. */
-public class KafkaExampleUtil {
-
- public static StreamExecutionEnvironment prepareExecutionEnv(ParameterTool parameterTool)
- throws Exception {
-
- if (parameterTool.getNumberOfParameters() < 4) {
- System.out.println(
- "Missing parameters!\n"
- + "Usage: Kafka --input-topic <topic> --output-topic <topic> "
- + "--bootstrap.servers <kafka brokers> "
- + "--group.id <some id>");
- throw new Exception(
- "Missing parameters!\n"
- + "Usage: Kafka --input-topic <topic> --output-topic <topic> "
- + "--bootstrap.servers <kafka brokers> "
- + "--group.id <some id>");
- }
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000));
- env.enableCheckpointing(5000); // create a checkpoint every 5 seconds
- env.getConfig()
- .setGlobalJobParameters(
- parameterTool); // make parameters available in the web interface
-
- return env;
- }
-}
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java
deleted file mode 100644
index 995d94b..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.kafka.test.base;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.configuration.Configuration;
-
-/**
- * A {@link RichMapFunction} that continuously outputs the current total frequency count of a key.
- * The current total count is keyed state managed by Flink.
- */
-public class RollingAdditionMapper extends RichMapFunction<KafkaEvent, KafkaEvent> {
-
- private static final long serialVersionUID = 1180234853172462378L;
-
- private transient ValueState<Integer> currentTotalCount;
-
- @Override
- public KafkaEvent map(KafkaEvent event) throws Exception {
- Integer totalCount = currentTotalCount.value();
-
- if (totalCount == null) {
- totalCount = 0;
- }
- totalCount += event.getFrequency();
-
- currentTotalCount.update(totalCount);
-
- return new KafkaEvent(event.getWord(), totalCount, event.getTimestamp());
- }
-
- @Override
- public void open(Configuration parameters) throws Exception {
- currentTotalCount =
- getRuntimeContext()
- .getState(new ValueStateDescriptor<>("currentTotalCount", Integer.class));
- }
-}
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml
deleted file mode 100644
index 345ffef..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml
+++ /dev/null
@@ -1,87 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <parent>
- <artifactId>flink-end-to-end-tests</artifactId>
- <groupId>org.apache.flink</groupId>
- <version>1.18-SNAPSHOT</version>
- </parent>
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>flink-streaming-kafka-test</artifactId>
- <name>Flink : E2E Tests : Streaming Kafka</name>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-kafka-test-base</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
-
- <build>
- <finalName>KafkaExample</finalName>
- <plugins>
- <!-- Use the shade plugin to build a fat jar for the kafka connector test -->
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <executions>
- <execution>
- <id>shade-flink</id>
- <phase>none</phase>
- </execution>
- <execution>
- <id>fat-jar-kafka-example</id>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <shadeTestJar>false</shadeTestJar>
- <shadedArtifactAttached>false</shadedArtifactAttached>
- <createDependencyReducedPom>false</createDependencyReducedPom>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- <mainClass>org.apache.flink.streaming.kafka.test.KafkaExample</mainClass>
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-
-</project>
diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java b/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java
deleted file mode 100644
index cb90a1d..0000000
--- a/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.kafka.test;
-
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
-import org.apache.flink.connector.kafka.sink.KafkaSink;
-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.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.kafka.test.base.KafkaExampleUtil;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.IntegerDeserializer;
-import org.apache.kafka.common.serialization.IntegerSerializer;
-
-/**
- * A simple application used as smoke test example to forward messages from one topic to another
- * topic in batch mode.
- *
- * <p>Example usage: --input-topic test-input --output-topic test-output --bootstrap.servers
- * localhost:9092 --group.id myconsumer
- */
-public class KafkaExample extends KafkaExampleUtil {
-
- public static void main(String[] args) throws Exception {
- // parse input arguments
- final ParameterTool parameterTool = ParameterTool.fromArgs(args);
- StreamExecutionEnvironment env = KafkaExampleUtil.prepareExecutionEnv(parameterTool);
-
- DataStream<Integer> input =
- env.fromSource(
- KafkaSource.<Integer>builder()
- .setBootstrapServers(
- parameterTool
- .getProperties()
- .getProperty(
- ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
- .setBounded(OffsetsInitializer.latest())
- .setDeserializer(
- KafkaRecordDeserializationSchema.valueOnly(
- IntegerDeserializer.class))
- .setTopics(parameterTool.getRequired("input-topic"))
- .build(),
- WatermarkStrategy.noWatermarks(),
- "kafka-source");
-
- input.sinkTo(
- KafkaSink.<Integer>builder()
- .setBootstrapServers(
- parameterTool
- .getProperties()
- .getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
- .setRecordSerializer(
- KafkaRecordSerializationSchema.builder()
- .setTopic(parameterTool.getRequired("output-topic"))
- .setKafkaValueSerializer(IntegerSerializer.class)
- .build())
- .build());
- env.execute("Smoke Kafka Example");
- }
-}
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index f199aaa..edf788e 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -65,11 +65,8 @@
<module>flink-metrics-availability-test</module>
<module>flink-metrics-reporter-prometheus-test</module>
<module>flink-heavy-deployment-stress-test</module>
- <module>flink-streaming-kafka-test-base</module>
- <module>flink-streaming-kafka-test</module>
<module>flink-plugins-test</module>
<module>flink-tpch-test</module>
- <module>flink-end-to-end-tests-common-kafka</module>
<module>flink-tpcds-test</module>
<module>flink-netty-shuffle-memory-control-test</module>
<module>flink-python-test</module>
diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml
index 6336dfd..d1bd52e 100644
--- a/flink-examples/flink-examples-streaming/pom.xml
+++ b/flink-examples/flink-examples-streaming/pom.xml
@@ -63,7 +63,7 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
</dependency>
<dependency>
diff --git a/flink-python/pom.xml b/flink-python/pom.xml
index a027e69..58ba1cc 100644
--- a/flink-python/pom.xml
+++ b/flink-python/pom.xml
@@ -282,7 +282,7 @@
<!-- Indirectly accessed in pyflink_gateway_server -->
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-kafka</artifactId>
- <version>${project.version}</version>
+ <version>3.0.0-1.17</version>
<scope>test</scope>
</dependency>
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index 97c746a..235c104 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -74,7 +74,7 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka</artifactId>
- <version>${flink.version}</version>
+ <version>3.0.0-1.17</version>
</dependency>
-->
diff --git a/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml
index cb5932a..4a90f55 100644
--- a/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml
@@ -79,7 +79,7 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka</artifactId>
- <version>${flink.version}</version>
+ <version>3.0.0-1.17</version>
</dependency>
-->