You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by da...@apache.org on 2022/12/03 18:59:57 UTC
[flink-connector-aws] 05/08: [FLINK-29907][Connectors/Kinesis] Externalize Amazon Kinesis connectors from Flink repo
This is an automated email from the ASF dual-hosted git repository.
dannycranmer pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-connector-aws.git
commit a23e101dda08cc377602db66df5414889c32a8c8
Author: Danny Cranmer <da...@apache.org>
AuthorDate: Fri Dec 2 11:30:40 2022 +0000
[FLINK-29907][Connectors/Kinesis] Externalize Amazon Kinesis connectors from Flink repo
---
flink-connector-aws-kinesis-streams/pom.xml | 2 +-
.../28f0499c-3213-4ec2-97f7-970f052922b3 | 0
.../4c963703-6b45-4782-825a-5cc6ba1556dd | 0
.../archunit-violations/stored.rules | 4 +
flink-connector-kinesis/pom.xml | 394 ++++++
.../kinesis/FlinkDynamoDBStreamsConsumer.java | 80 ++
.../connectors/kinesis/FlinkKinesisConsumer.java | 556 ++++++++
.../connectors/kinesis/FlinkKinesisException.java | 48 +
.../connectors/kinesis/FlinkKinesisProducer.java | 500 +++++++
.../connectors/kinesis/KinesisPartitioner.java | 65 +
.../connectors/kinesis/KinesisShardAssigner.java | 54 +
.../kinesis/config/AWSConfigConstants.java | 28 +
.../kinesis/config/ConsumerConfigConstants.java | 427 ++++++
.../kinesis/config/ProducerConfigConstants.java | 51 +
.../internals/DynamoDBStreamsDataFetcher.java | 128 ++
.../kinesis/internals/KinesisDataFetcher.java | 1460 ++++++++++++++++++++
.../kinesis/internals/ShardConsumer.java | 258 ++++
.../kinesis/internals/publisher/RecordBatch.java | 95 ++
.../internals/publisher/RecordPublisher.java | 62 +
.../publisher/RecordPublisherFactory.java | 51 +
.../publisher/fanout/FanOutRecordPublisher.java | 304 ++++
.../fanout/FanOutRecordPublisherConfiguration.java | 475 +++++++
.../fanout/FanOutRecordPublisherFactory.java | 99 ++
.../publisher/fanout/FanOutShardSubscriber.java | 609 ++++++++
.../publisher/fanout/StreamConsumerRegistrar.java | 313 +++++
.../polling/AdaptivePollingRecordPublisher.java | 132 ++
.../publisher/polling/PollingRecordPublisher.java | 223 +++
.../PollingRecordPublisherConfiguration.java | 70 +
.../polling/PollingRecordPublisherFactory.java | 89 ++
.../metrics/KinesisConsumerMetricConstants.java | 48 +
.../PollingRecordPublisherMetricsReporter.java | 87 ++
.../metrics/ShardConsumerMetricsReporter.java | 90 ++
.../kinesis/model/DynamoDBStreamsShardHandle.java | 60 +
.../kinesis/model/KinesisStreamShard.java | 149 ++
.../kinesis/model/KinesisStreamShardState.java | 103 ++
.../kinesis/model/SentinelSequenceNumber.java | 76 +
.../connectors/kinesis/model/SequenceNumber.java | 112 ++
.../connectors/kinesis/model/StartingPosition.java | 118 ++
.../kinesis/model/StreamShardHandle.java | 115 ++
.../kinesis/model/StreamShardMetadata.java | 236 ++++
.../kinesis/proxy/DynamoDBStreamsProxy.java | 132 ++
.../kinesis/proxy/FullJitterBackoff.java | 61 +
.../kinesis/proxy/GetShardListResult.java | 78 ++
.../connectors/kinesis/proxy/KinesisProxy.java | 654 +++++++++
.../kinesis/proxy/KinesisProxyInterface.java | 83 ++
.../connectors/kinesis/proxy/KinesisProxyV2.java | 221 +++
.../kinesis/proxy/KinesisProxyV2Factory.java | 82 ++
.../kinesis/proxy/KinesisProxyV2Interface.java | 63 +
.../serialization/DynamoDBStreamsSchema.java | 48 +
.../KinesisDeserializationSchema.java | 81 ++
.../KinesisDeserializationSchemaWrapper.java | 86 ++
.../serialization/KinesisSerializationSchema.java | 61 +
.../kinesis/table/KinesisConnectorOptionsUtil.java | 58 +
.../kinesis/table/KinesisConsumerOptionsUtil.java | 87 ++
.../kinesis/table/KinesisDynamicSource.java | 222 +++
.../kinesis/table/KinesisDynamicTableFactory.java | 99 ++
.../table/RowDataKinesisDeserializationSchema.java | 144 ++
.../streaming/connectors/kinesis/util/AWSUtil.java | 291 ++++
.../connectors/kinesis/util/AwsV2Util.java | 81 ++
.../BeanDeserializerModifierForIgnorables.java | 81 ++
.../kinesis/util/JobManagerWatermarkTracker.java | 188 +++
.../connectors/kinesis/util/KinesisConfigUtil.java | 615 +++++++++
.../connectors/kinesis/util/RecordEmitter.java | 284 ++++
.../kinesis/util/StreamConsumerRegistrarUtil.java | 178 +++
.../connectors/kinesis/util/TimeoutLatch.java | 45 +
.../kinesis/util/UniformShardAssigner.java | 58 +
.../connectors/kinesis/util/WatermarkTracker.java | 117 ++
.../src/main/resources/META-INF/NOTICE | 294 ++++
.../resources/META-INF/licenses/LICENSE.protobuf | 32 +
...aded.software.amazon.awssdk.http.SdkHttpService | 20 +
.../org.apache.flink.table.factories.Factory | 16 +
.../awssdk/global/handlers/execution.interceptors | 1 +
.../architecture/TestCodeArchitectureTest.java | 40 +
.../kinesis/FlinkKinesisConsumerMigrationTest.java | 547 ++++++++
.../kinesis/FlinkKinesisConsumerTest.java | 1258 +++++++++++++++++
.../connectors/kinesis/FlinkKinesisITCase.java | 251 ++++
.../kinesis/FlinkKinesisProducerTest.java | 546 ++++++++
.../connectors/kinesis/KinesisConsumerTest.java | 75 +
.../examples/ConsumeFromDynamoDBStreams.java | 59 +
.../kinesis/examples/ConsumeFromKinesis.java | 55 +
.../kinesis/examples/ProduceIntoKinesis.java | 83 ++
.../internals/DynamoDBStreamsDataFetcherTest.java | 69 +
.../kinesis/internals/KinesisDataFetcherTest.java | 1117 +++++++++++++++
.../kinesis/internals/ShardConsumerFanOutTest.java | 300 ++++
.../kinesis/internals/ShardConsumerTest.java | 246 ++++
.../kinesis/internals/ShardConsumerTestUtils.java | 207 +++
.../internals/publisher/RecordBatchTest.java | 92 ++
.../FanOutRecordPublisherConfigurationTest.java | 196 +++
.../fanout/FanOutRecordPublisherTest.java | 616 +++++++++
.../fanout/FanOutShardSubscriberTest.java | 185 +++
.../fanout/StreamConsumerRegistrarTest.java | 340 +++++
.../PollingRecordPublisherConfigurationTest.java | 73 +
.../polling/PollingRecordPublisherFactoryTest.java | 71 +
.../polling/PollingRecordPublisherTest.java | 174 +++
.../manualtests/ManualConsumerProducerTest.java | 132 ++
.../kinesis/manualtests/ManualExactlyOnceTest.java | 172 +++
.../ManualExactlyOnceWithStreamReshardingTest.java | 289 ++++
.../kinesis/manualtests/ManualProducerTest.java | 95 ++
.../PollingRecordPublisherMetricsReporterTest.java | 68 +
.../metrics/ShardConsumerMetricsReporterTest.java | 84 ++
.../model/DynamoDBStreamsShardHandleTest.java | 108 ++
.../kinesis/model/SentinelSequenceNumberTest.java | 33 +
.../kinesis/model/StartingPositionTest.java | 94 ++
.../kinesis/model/StreamShardHandleTest.java | 41 +
.../connectors/kinesis/proxy/KinesisProxyTest.java | 514 +++++++
.../kinesis/proxy/KinesisProxyV2FactoryTest.java | 86 ++
.../kinesis/proxy/KinesisProxyV2Test.java | 423 ++++++
.../table/KinesisDynamicTableFactoryTest.java | 275 ++++
.../AlwaysThrowsDeserializationSchema.java | 57 +
.../ExactlyOnceValidatingConsumerThread.java | 185 +++
.../testutils/FakeKinesisBehavioursFactory.java | 686 +++++++++
.../FakeKinesisFanOutBehavioursFactory.java | 708 ++++++++++
.../KinesisEventsGeneratorProducerThread.java | 132 ++
.../kinesis/testutils/KinesisPubsubClient.java | 158 +++
.../kinesis/testutils/KinesisShardIdGenerator.java | 29 +
.../kinesis/testutils/TestRuntimeContext.java | 89 ++
.../kinesis/testutils/TestSourceContext.java | 66 +
.../connectors/kinesis/testutils/TestUtils.java | 207 +++
.../testutils/TestableFlinkKinesisConsumer.java | 69 +
.../testutils/TestableKinesisDataFetcher.java | 217 +++
...inesisDataFetcherForShardConsumerException.java | 102 ++
.../connectors/kinesis/util/AWSUtilTest.java | 206 +++
.../connectors/kinesis/util/AwsV2UtilTest.java | 167 +++
.../util/JobManagerWatermarkTrackerTest.java | 75 +
.../kinesis/util/KinesisConfigUtilTest.java | 1024 ++++++++++++++
.../connectors/kinesis/util/RecordEmitterTest.java | 136 ++
.../util/StreamConsumerRegistrarUtilTest.java | 92 ++
.../kinesis/util/UniformShardAssignerTest.java | 75 +
.../kinesis/util/WatermarkTrackerTest.java | 106 ++
.../src/test/resources/archunit.properties | 31 +
...onsumer-migration-test-flink1.10-empty-snapshot | Bin 0 -> 2862 bytes
...esis-consumer-migration-test-flink1.10-snapshot | Bin 0 -> 2930 bytes
...onsumer-migration-test-flink1.11-empty-snapshot | Bin 0 -> 2870 bytes
...esis-consumer-migration-test-flink1.11-snapshot | Bin 0 -> 2938 bytes
...onsumer-migration-test-flink1.12-empty-snapshot | Bin 0 -> 2870 bytes
...esis-consumer-migration-test-flink1.12-snapshot | Bin 0 -> 2938 bytes
...onsumer-migration-test-flink1.13-empty-snapshot | Bin 0 -> 2870 bytes
...esis-consumer-migration-test-flink1.13-snapshot | Bin 0 -> 2938 bytes
...onsumer-migration-test-flink1.14-empty-snapshot | Bin 0 -> 2870 bytes
...esis-consumer-migration-test-flink1.14-snapshot | Bin 0 -> 2938 bytes
...onsumer-migration-test-flink1.15-empty-snapshot | Bin 0 -> 2870 bytes
...esis-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 2938 bytes
...consumer-migration-test-flink1.3-empty-snapshot | Bin 0 -> 13975 bytes
...nesis-consumer-migration-test-flink1.3-snapshot | Bin 0 -> 14043 bytes
...consumer-migration-test-flink1.4-empty-snapshot | Bin 0 -> 13147 bytes
...nesis-consumer-migration-test-flink1.4-snapshot | Bin 0 -> 13215 bytes
...consumer-migration-test-flink1.7-empty-snapshot | Bin 0 -> 18539 bytes
...nesis-consumer-migration-test-flink1.7-snapshot | Bin 0 -> 18607 bytes
...consumer-migration-test-flink1.8-empty-snapshot | Bin 0 -> 2862 bytes
...nesis-consumer-migration-test-flink1.8-snapshot | Bin 0 -> 2930 bytes
...consumer-migration-test-flink1.9-empty-snapshot | Bin 0 -> 2862 bytes
...nesis-consumer-migration-test-flink1.9-snapshot | Bin 0 -> 2930 bytes
.../src/test/resources/log4j2-test.properties | 29 +
flink-connector-kinesis/src/test/resources/profile | 7 +
flink-sql-connector-aws-kinesis-streams/pom.xml | 2 +-
flink-sql-connector-kinesis/pom.xml | 121 ++
.../src/main/resources/META-INF/NOTICE | 20 +
.../flink/connectors/kinesis/PackagingITCase.java | 46 +
pom.xml | 127 +-
tools/maven/suppressions.xml | 9 +-
160 files changed, 26318 insertions(+), 6 deletions(-)
diff --git a/flink-connector-aws-kinesis-streams/pom.xml b/flink-connector-aws-kinesis-streams/pom.xml
index af280a6..0aba4c6 100644
--- a/flink-connector-aws-kinesis-streams/pom.xml
+++ b/flink-connector-aws-kinesis-streams/pom.xml
@@ -30,7 +30,7 @@ under the License.
</parent>
<artifactId>flink-connector-aws-kinesis-streams</artifactId>
- <name>Flink : Connectors : AWS : Amazon Kinesis Data Streams</name>
+ <name>Flink : Connectors : AWS : Amazon Kinesis Data Streams Sink v2</name>
<packaging>jar</packaging>
<dependencies>
diff --git a/flink-connector-kinesis/archunit-violations/28f0499c-3213-4ec2-97f7-970f052922b3 b/flink-connector-kinesis/archunit-violations/28f0499c-3213-4ec2-97f7-970f052922b3
new file mode 100644
index 0000000..e69de29
diff --git a/flink-connector-kinesis/archunit-violations/4c963703-6b45-4782-825a-5cc6ba1556dd b/flink-connector-kinesis/archunit-violations/4c963703-6b45-4782-825a-5cc6ba1556dd
new file mode 100644
index 0000000..e69de29
diff --git a/flink-connector-kinesis/archunit-violations/stored.rules b/flink-connector-kinesis/archunit-violations/stored.rules
new file mode 100644
index 0000000..fe99e99
--- /dev/null
+++ b/flink-connector-kinesis/archunit-violations/stored.rules
@@ -0,0 +1,4 @@
+#
+#Tue Feb 22 12:18:40 CET 2022
+Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=4c963703-6b45-4782-825a-5cc6ba1556dd
+ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=28f0499c-3213-4ec2-97f7-970f052922b3
diff --git a/flink-connector-kinesis/pom.xml b/flink-connector-kinesis/pom.xml
new file mode 100644
index 0000000..66d743f
--- /dev/null
+++ b/flink-connector-kinesis/pom.xml
@@ -0,0 +1,394 @@
+<?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/maven-v4_0_0.xsd">
+
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-aws-parent</artifactId>
+ <version>4.0-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>flink-connector-kinesis</artifactId>
+ <name>Flink : Connectors : AWS : Amazon Kinesis Data Streams</name>
+
+ <properties>
+ <aws.kinesis-kpl.version>0.14.1</aws.kinesis-kpl.version>
+ <aws.dynamodbstreams-kinesis-adapter.version>1.5.3</aws.dynamodbstreams-kinesis-adapter.version>
+ <hamcrest.version>1.3</hamcrest.version>
+ </properties>
+
+ <packaging>jar</packaging>
+
+ <dependencies>
+ <!-- AWS dependencies -->
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>amazon-kinesis-aggregator</artifactId>
+ <version>1.0.3</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Amazon AWS SDK v1.x dependencies -->
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-kinesis</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-sts</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-kms</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-s3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-dynamodb</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-cloudwatch</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>amazon-kinesis-producer</artifactId>
+ <version>${aws.kinesis-kpl.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>amazon-kinesis-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>dynamodb-streams-kinesis-adapter</artifactId>
+ <version>${aws.dynamodbstreams-kinesis-adapter.version}</version>
+ </dependency>
+
+ <!-- Other third-party dependencies -->
+ <dependency>
+ <!-- KPL requires jaxb-api for javax.xml.bind.DatatypeConverter -->
+ <groupId>javax.xml.bind</groupId>
+ <artifactId>jaxb-api</artifactId>
+ <!-- packaged in flink-dist -->
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+
+ <!-- Flink ecosystem -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-aws-kinesis-streams</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-aws-base</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <!-- Flink Table API 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>${flink.version}</version>
+ <scope>provided</scope>
+ <optional>true</optional>
+ </dependency>
+
+ <!-- Test dependencies -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-core</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-runtime</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.hamcrest</groupId>
+ <artifactId>hamcrest-all</artifactId>
+ <version>${hamcrest.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-aws-kinesis-streams</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-aws-base</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.testcontainers</groupId>
+ <artifactId>testcontainers</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Kinesis table factory testing -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-common</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- Amazon AWS SDK v2.x dependencies -->
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>kinesis</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>netty-nio-client</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>sts</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.httpcomponents</groupId>
+ <artifactId>httpclient</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.httpcomponents</groupId>
+ <artifactId>httpcore</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.fasterxml.jackson.dataformat</groupId>
+ <artifactId>jackson-dataformat-cbor</artifactId>
+ </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>**/org/apache/flink/streaming/connectors/kinesis/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-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>shade-flink</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <!-- required for the Kinesis e2e test -->
+ <shadeTestJar>true</shadeTestJar>
+ <promoteTransitiveDependencies>true</promoteTransitiveDependencies>
+ <artifactSet combine.children="append">
+ <includes>
+ <include>org.apache.flink:flink-connector-aws-base:*</include>
+ <include>org.apache.flink:flink-connector-aws-kinesis-streams:*</include>
+ <include>com.amazonaws:*</include>
+ <include>com.google.protobuf:*</include>
+ <include>org.apache.httpcomponents:*</include>
+ <include>software.amazon.awssdk:*</include>
+ <include>software.amazon.eventstream:*</include>
+ <include>software.amazon.ion:*</include>
+ <include>org.reactivestreams:*</include>
+ <include>io.netty:*</include>
+ <include>com.typesafe.netty:*</include>
+ </includes>
+ </artifactSet>
+ <relocations combine.children="override">
+ <!-- Do not relocate guava because it is exposed in the Kinesis API (KinesisProducer#addUserRecord).
+ Users may be using other affected API's, so relocations may break user-code -->
+ <relocation>
+ <pattern>org.apache.flink.connector.aws.config</pattern>
+ <shadedPattern>
+ org.apache.flink.kinesis.shaded.org.apache.flink.connector.aws.config
+ </shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.flink.connector.aws.util</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.org.apache.flink.connector.aws.util
+ </shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.google.protobuf</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.com.google.protobuf</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.amazonaws</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.com.amazonaws</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.apache.http</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.org.apache.http</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>software.amazon</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.software.amazon</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>io.netty</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.io.netty</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.typesafe.netty</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.com.typesafe.netty</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.reactivestreams</pattern>
+ <shadedPattern>org.apache.flink.kinesis.shaded.org.reactivestreams</shadedPattern>
+ </relocation>
+ </relocations>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>.gitkeep</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>com.amazonaws:amazon-kinesis-producer</artifact>
+ <excludes>
+ <exclude>META-INF/THIRD_PARTY_NOTICES</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>software.amazon.awssdk:*</artifact>
+ <excludes>
+ <exclude>META-INF/services/**</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>org.apache.flink:flink-connector-aws-kinesis-streams:*</artifact>
+ <excludes>
+ <exclude>profile</exclude>
+ </excludes>
+ </filter>
+ <filter>
+ <artifact>org.apache.flink:flink-connector-aws-base:*</artifact>
+ <excludes>
+ <exclude>profile</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <configuration>
+ <systemPropertyVariables>
+ <com.amazonaws.sdk.disableCbor>true</com.amazonaws.sdk.disableCbor>
+ <com.amazonaws.sdk.disableCertChecking>true</com.amazonaws.sdk.disableCertChecking>
+ </systemPropertyVariables>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java
new file mode 100644
index 0000000..fd87a62
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.internals.DynamoDBStreamsDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Consume events from DynamoDB streams.
+ *
+ * @param <T> the type of data emitted
+ */
+public class FlinkDynamoDBStreamsConsumer<T> extends FlinkKinesisConsumer<T> {
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDynamoDBStreamsConsumer.class);
+
+ /**
+ * Constructor of FlinkDynamoDBStreamsConsumer.
+ *
+ * @param stream stream to consume
+ * @param deserializer deserialization schema
+ * @param config config properties
+ */
+ public FlinkDynamoDBStreamsConsumer(
+ String stream, DeserializationSchema<T> deserializer, Properties config) {
+ super(stream, deserializer, config);
+ }
+
+ /**
+ * Constructor of FlinkDynamodbStreamConsumer.
+ *
+ * @param streams list of streams to consume
+ * @param deserializer deserialization schema
+ * @param config config properties
+ */
+ public FlinkDynamoDBStreamsConsumer(
+ List<String> streams, KinesisDeserializationSchema deserializer, Properties config) {
+ super(streams, deserializer, config);
+ }
+
+ @Override
+ protected KinesisDataFetcher<T> createFetcher(
+ List<String> streams,
+ SourceFunction.SourceContext<T> sourceContext,
+ RuntimeContext runtimeContext,
+ Properties configProps,
+ KinesisDeserializationSchema<T> deserializationSchema) {
+ return new DynamoDBStreamsDataFetcher<T>(
+ streams,
+ sourceContext,
+ runtimeContext,
+ configProps,
+ deserializationSchema,
+ getShardAssigner());
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
new file mode 100644
index 0000000..488a1f5
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
@@ -0,0 +1,556 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+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.DeserializationSchema;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.TupleTypeInfo;
+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.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.StreamConsumerRegistrarUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.WatermarkTracker;
+import org.apache.flink.util.InstantiationUtil;
+
+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.Properties;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kinesis Consumer is an exactly-once parallel streaming data source that subscribes to
+ * multiple AWS Kinesis streams within the same AWS service region, and can handle resharding of
+ * streams. Each subtask of the consumer is responsible for fetching data records from multiple
+ * Kinesis shards. The number of shards fetched by each subtask will change as shards are closed and
+ * created by Kinesis.
+ *
+ * <p>To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees,
+ * the Flink Kinesis consumer is implemented with the AWS Java SDK, instead of the officially
+ * recommended AWS Kinesis Client Library, for low-level control on the management of stream state.
+ * The Flink Kinesis Connector also supports setting the initial starting points of Kinesis streams,
+ * namely TRIM_HORIZON and LATEST.
+ *
+ * <p>Kinesis and the Flink consumer support dynamic re-sharding and shard IDs, while sequential,
+ * cannot be assumed to be consecutive. There is no perfect generic default assignment function.
+ * Default shard to subtask assignment, which is based on hash code, may result in skew, with some
+ * subtasks having many shards assigned and others none.
+ *
+ * <p>It is recommended to monitor the shard distribution and adjust assignment appropriately. A
+ * custom assigner implementation can be set via {@link #setShardAssigner(KinesisShardAssigner)} to
+ * optimize the hash function or use static overrides to limit skew.
+ *
+ * <p>In order for the consumer to emit watermarks, a timestamp assigner needs to be set via {@link
+ * #setPeriodicWatermarkAssigner(AssignerWithPeriodicWatermarks)} and the auto watermark emit
+ * interval configured via {@link
+ * org.apache.flink.api.common.ExecutionConfig#setAutoWatermarkInterval(long)}.
+ *
+ * <p>Watermarks can only advance when all shards of a subtask continuously deliver records. To
+ * avoid an inactive or closed shard to block the watermark progress, the idle timeout should be
+ * configured via configuration property {@link ConsumerConfigConstants#SHARD_IDLE_INTERVAL_MILLIS}.
+ * By default, shards won't be considered idle and watermark calculation will wait for newer records
+ * to arrive from all shards.
+ *
+ * <p>Note that re-sharding of the Kinesis stream while an application (that relies on the Kinesis
+ * records for watermarking) is running can lead to incorrect late events. This depends on how
+ * shards are assigned to subtasks and applies regardless of whether watermarks are generated in the
+ * source or a downstream operator.
+ *
+ * @param <T> the type of data emitted
+ */
+@PublicEvolving
+public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T>
+ implements ResultTypeQueryable<T>, CheckpointedFunction {
+
+ private static final long serialVersionUID = 4724006128720664870L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisConsumer.class);
+
+ // ------------------------------------------------------------------------
+ // Consumer properties
+ // ------------------------------------------------------------------------
+
+ /** The names of the Kinesis streams that we will be consuming from. */
+ private final List<String> streams;
+
+ /**
+ * Properties to parametrize settings such as AWS service region, initial position in stream,
+ * shard list retrieval behaviours, etc.
+ */
+ private final Properties configProps;
+
+ /** User supplied deserialization schema to convert Kinesis byte messages to Flink objects. */
+ private final KinesisDeserializationSchema<T> deserializer;
+
+ /** The function that determines which subtask a shard should be assigned to. */
+ private KinesisShardAssigner shardAssigner = KinesisDataFetcher.DEFAULT_SHARD_ASSIGNER;
+
+ private AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner;
+ private WatermarkTracker watermarkTracker;
+
+ // ------------------------------------------------------------------------
+ // Runtime state
+ // ------------------------------------------------------------------------
+
+ /**
+ * Per-task fetcher for Kinesis data records, where each fetcher pulls data from one or more
+ * Kinesis shards.
+ */
+ private transient KinesisDataFetcher<T> fetcher;
+
+ /** The sequence numbers to restore to upon restore from failure. */
+ private transient HashMap<StreamShardMetadata.EquivalenceWrapper, SequenceNumber>
+ sequenceNumsToRestore;
+
+ private volatile boolean running = true;
+
+ // ------------------------------------------------------------------------
+ // State for Checkpoint
+ // ------------------------------------------------------------------------
+
+ /** State name to access shard sequence number states; cannot be changed. */
+ private static final String sequenceNumsStateStoreName = "Kinesis-Stream-Shard-State";
+
+ private transient ListState<Tuple2<StreamShardMetadata, SequenceNumber>>
+ sequenceNumsStateForCheckpoint;
+
+ // ------------------------------------------------------------------------
+ // Constructors
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a new Flink Kinesis Consumer.
+ *
+ * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
+ * start streaming from are configured with a {@link Properties} instance.
+ *
+ * @param stream The single AWS Kinesis stream to read from.
+ * @param deserializer The deserializer used to convert raw bytes of Kinesis records to Java
+ * objects (without key).
+ * @param configProps The properties used to configure AWS credentials, AWS region, and initial
+ * starting position.
+ */
+ public FlinkKinesisConsumer(
+ String stream, DeserializationSchema<T> deserializer, Properties configProps) {
+ this(stream, new KinesisDeserializationSchemaWrapper<>(deserializer), configProps);
+ }
+
+ /**
+ * Creates a new Flink Kinesis Consumer.
+ *
+ * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
+ * start streaming from are configured with a {@link Properties} instance.
+ *
+ * @param stream The single AWS Kinesis stream to read from.
+ * @param deserializer The keyed deserializer used to convert raw bytes of Kinesis records to
+ * Java objects.
+ * @param configProps The properties used to configure AWS credentials, AWS region, and initial
+ * starting position.
+ */
+ public FlinkKinesisConsumer(
+ String stream, KinesisDeserializationSchema<T> deserializer, Properties configProps) {
+ this(Collections.singletonList(stream), deserializer, configProps);
+ }
+
+ /**
+ * Creates a new Flink Kinesis Consumer.
+ *
+ * <p>The AWS credentials to be used, AWS region of the Kinesis streams, initial position to
+ * start streaming from are configured with a {@link Properties} instance.
+ *
+ * @param streams The AWS Kinesis streams to read from.
+ * @param deserializer The keyed deserializer used to convert raw bytes of Kinesis records to
+ * Java objects.
+ * @param configProps The properties used to configure AWS credentials, AWS region, and initial
+ * starting position.
+ */
+ public FlinkKinesisConsumer(
+ List<String> streams,
+ KinesisDeserializationSchema<T> deserializer,
+ Properties configProps) {
+ checkNotNull(streams, "streams can not be null");
+ checkArgument(streams.size() != 0, "must be consuming at least 1 stream");
+ checkArgument(!streams.contains(""), "stream names cannot be empty Strings");
+ this.streams = streams;
+
+ this.configProps = checkNotNull(configProps, "configProps can not be null");
+
+ // check the configuration properties for any conflicting settings
+ KinesisConfigUtil.validateConsumerConfiguration(this.configProps, streams);
+
+ checkNotNull(deserializer, "deserializer can not be null");
+ checkArgument(
+ InstantiationUtil.isSerializable(deserializer),
+ "The provided deserialization schema is not serializable: "
+ + deserializer.getClass().getName()
+ + ". "
+ + "Please check that it does not contain references to non-serializable instances.");
+ this.deserializer = deserializer;
+
+ StreamConsumerRegistrarUtil.eagerlyRegisterStreamConsumers(configProps, streams);
+
+ if (LOG.isInfoEnabled()) {
+ StringBuilder sb = new StringBuilder();
+ for (String stream : streams) {
+ sb.append(stream).append(", ");
+ }
+ LOG.info(
+ "Flink Kinesis Consumer is going to read the following streams: {}",
+ sb.toString());
+ }
+ }
+
+ public KinesisShardAssigner getShardAssigner() {
+ return shardAssigner;
+ }
+
+ /**
+ * Provide a custom assigner to influence how shards are distributed over subtasks.
+ *
+ * @param shardAssigner shard assigner
+ */
+ public void setShardAssigner(KinesisShardAssigner shardAssigner) {
+ this.shardAssigner = checkNotNull(shardAssigner, "function can not be null");
+ ClosureCleaner.clean(shardAssigner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+ }
+
+ public AssignerWithPeriodicWatermarks<T> getPeriodicWatermarkAssigner() {
+ return periodicWatermarkAssigner;
+ }
+
+ /**
+ * Set the assigner that will extract the timestamp from {@link T} and calculate the watermark.
+ *
+ * @param periodicWatermarkAssigner periodic watermark assigner
+ */
+ public void setPeriodicWatermarkAssigner(
+ AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner) {
+ this.periodicWatermarkAssigner = periodicWatermarkAssigner;
+ ClosureCleaner.clean(
+ this.periodicWatermarkAssigner,
+ ExecutionConfig.ClosureCleanerLevel.RECURSIVE,
+ true);
+ }
+
+ public WatermarkTracker getWatermarkTracker() {
+ return this.watermarkTracker;
+ }
+
+ /**
+ * Set the global watermark tracker. When set, it will be used by the fetcher to align the shard
+ * consumers by event time.
+ *
+ * @param watermarkTracker
+ */
+ public void setWatermarkTracker(WatermarkTracker watermarkTracker) {
+ this.watermarkTracker = watermarkTracker;
+ ClosureCleaner.clean(
+ this.watermarkTracker, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+ }
+
+ // ------------------------------------------------------------------------
+ // Source life cycle
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void run(SourceContext<T> sourceContext) throws Exception {
+
+ // all subtasks will run a fetcher, regardless of whether or not the subtask will initially
+ // have
+ // shards to subscribe to; fetchers will continuously poll for changes in the shard list, so
+ // all subtasks
+ // can potentially have new shards to subscribe to later on
+ KinesisDataFetcher<T> fetcher =
+ createFetcher(
+ streams, sourceContext, getRuntimeContext(), configProps, deserializer);
+
+ // initial discovery
+ List<StreamShardHandle> allShards = fetcher.discoverNewShardsToSubscribe();
+
+ for (StreamShardHandle shard : allShards) {
+ StreamShardMetadata.EquivalenceWrapper kinesisStreamShard =
+ new StreamShardMetadata.EquivalenceWrapper(
+ KinesisDataFetcher.convertToStreamShardMetadata(shard));
+
+ if (sequenceNumsToRestore != null) {
+
+ if (sequenceNumsToRestore.containsKey(kinesisStreamShard)) {
+ // if the shard was already seen and is contained in the state,
+ // just use the sequence number stored in the state
+ fetcher.registerNewSubscribedShardState(
+ new KinesisStreamShardState(
+ kinesisStreamShard.getShardMetadata(),
+ shard,
+ sequenceNumsToRestore.get(kinesisStreamShard)));
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info(
+ "Subtask {} is seeding the fetcher with restored shard {},"
+ + " starting state set to the restored sequence number {}",
+ getRuntimeContext().getIndexOfThisSubtask(),
+ shard.toString(),
+ sequenceNumsToRestore.get(kinesisStreamShard));
+ }
+ } else {
+ // the shard wasn't discovered in the previous run, therefore should be consumed
+ // from the beginning
+ fetcher.registerNewSubscribedShardState(
+ new KinesisStreamShardState(
+ kinesisStreamShard.getShardMetadata(),
+ shard,
+ SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get()));
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info(
+ "Subtask {} is seeding the fetcher with new discovered shard {},"
+ + " starting state set to the SENTINEL_EARLIEST_SEQUENCE_NUM",
+ getRuntimeContext().getIndexOfThisSubtask(),
+ shard.toString());
+ }
+ }
+ } else {
+ // we're starting fresh; use the configured start position as initial state
+ SentinelSequenceNumber startingSeqNum =
+ InitialPosition.valueOf(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_INITIAL_POSITION,
+ ConsumerConfigConstants
+ .DEFAULT_STREAM_INITIAL_POSITION))
+ .toSentinelSequenceNumber();
+
+ fetcher.registerNewSubscribedShardState(
+ new KinesisStreamShardState(
+ kinesisStreamShard.getShardMetadata(),
+ shard,
+ startingSeqNum.get()));
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info(
+ "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}",
+ getRuntimeContext().getIndexOfThisSubtask(),
+ shard.toString(),
+ startingSeqNum.get());
+ }
+ }
+ }
+
+ // check that we are running before starting the fetcher
+ if (!running) {
+ return;
+ }
+
+ // expose the fetcher from this point, so that state
+ // snapshots can be taken from the fetcher's state holders
+ this.fetcher = fetcher;
+
+ // start the fetcher loop. The fetcher will stop running only when cancel() or
+ // close() is called, or an error is thrown by threads created by the fetcher
+ fetcher.runFetcher();
+
+ // check that the fetcher has terminated before fully closing
+ fetcher.awaitTermination();
+ sourceContext.close();
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+
+ KinesisDataFetcher fetcher = this.fetcher;
+
+ // this method might be called before the subtask actually starts running,
+ // so we must check if the fetcher is actually created
+ if (fetcher != null) {
+ try {
+ // interrupt the fetcher of any work
+ fetcher.shutdownFetcher();
+ } catch (Exception e) {
+ LOG.warn("Error while closing Kinesis data fetcher", e);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ cancel();
+ // safe-guard when the fetcher has been interrupted, make sure to not leak resources
+ // application might be stopped before connector subtask has been started
+ // so we must check if the fetcher is actually created
+ KinesisDataFetcher fetcher = this.fetcher;
+ if (fetcher != null) {
+ fetcher.awaitTermination();
+ }
+ this.fetcher = null;
+ super.close();
+ }
+
+ @Override
+ public TypeInformation<T> getProducedType() {
+ return deserializer.getProducedType();
+ }
+
+ // ------------------------------------------------------------------------
+ // State Snapshot & Restore
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ TypeInformation<Tuple2<StreamShardMetadata, SequenceNumber>> shardsStateTypeInfo =
+ new TupleTypeInfo<>(
+ TypeInformation.of(StreamShardMetadata.class),
+ TypeInformation.of(SequenceNumber.class));
+
+ sequenceNumsStateForCheckpoint =
+ context.getOperatorStateStore()
+ .getUnionListState(
+ new ListStateDescriptor<>(
+ sequenceNumsStateStoreName, shardsStateTypeInfo));
+
+ if (context.isRestored()) {
+ if (sequenceNumsToRestore == null) {
+ sequenceNumsToRestore = new HashMap<>();
+ for (Tuple2<StreamShardMetadata, SequenceNumber> kinesisSequenceNumber :
+ sequenceNumsStateForCheckpoint.get()) {
+ sequenceNumsToRestore.put(
+ // we wrap the restored metadata inside an equivalence wrapper that
+ // checks only stream name and shard id,
+ // so that if a shard had been closed (due to a Kinesis reshard
+ // operation, for example) since
+ // the savepoint and has a different metadata than what we last stored,
+ // we will still be able to match it in sequenceNumsToRestore. Please
+ // see FLINK-8484 for details.
+ new StreamShardMetadata.EquivalenceWrapper(kinesisSequenceNumber.f0),
+ kinesisSequenceNumber.f1);
+ }
+
+ LOG.info(
+ "Setting restore state in the FlinkKinesisConsumer. Using the following offsets: {}",
+ sequenceNumsToRestore);
+ }
+ } else {
+ LOG.info("No restore state for FlinkKinesisConsumer.");
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ if (!running) {
+ LOG.debug("snapshotState() called on closed source; returning null.");
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Snapshotting state ...");
+ }
+
+ sequenceNumsStateForCheckpoint.clear();
+
+ if (fetcher == null) {
+ if (sequenceNumsToRestore != null) {
+ for (Map.Entry<StreamShardMetadata.EquivalenceWrapper, SequenceNumber> entry :
+ sequenceNumsToRestore.entrySet()) {
+ // sequenceNumsToRestore is the restored global union state;
+ // should only snapshot shards that actually belong to us
+ int hashCode =
+ shardAssigner.assign(
+ KinesisDataFetcher.convertToStreamShardHandle(
+ entry.getKey().getShardMetadata()),
+ getRuntimeContext().getNumberOfParallelSubtasks());
+ if (KinesisDataFetcher.isThisSubtaskShouldSubscribeTo(
+ hashCode,
+ getRuntimeContext().getNumberOfParallelSubtasks(),
+ getRuntimeContext().getIndexOfThisSubtask())) {
+
+ sequenceNumsStateForCheckpoint.add(
+ Tuple2.of(entry.getKey().getShardMetadata(), entry.getValue()));
+ }
+ }
+ }
+ } else {
+ HashMap<StreamShardMetadata, SequenceNumber> lastStateSnapshot =
+ fetcher.snapshotState();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Snapshotted state, last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}",
+ lastStateSnapshot,
+ context.getCheckpointId(),
+ context.getCheckpointTimestamp());
+ }
+
+ for (Map.Entry<StreamShardMetadata, SequenceNumber> entry :
+ lastStateSnapshot.entrySet()) {
+ sequenceNumsStateForCheckpoint.add(Tuple2.of(entry.getKey(), entry.getValue()));
+ }
+ }
+ }
+ }
+
+ /**
+ * This method is exposed for tests that need to mock the KinesisDataFetcher in the consumer.
+ */
+ protected KinesisDataFetcher<T> createFetcher(
+ List<String> streams,
+ SourceFunction.SourceContext<T> sourceContext,
+ RuntimeContext runtimeContext,
+ Properties configProps,
+ KinesisDeserializationSchema<T> deserializationSchema) {
+
+ return new KinesisDataFetcher<>(
+ streams,
+ sourceContext,
+ runtimeContext,
+ configProps,
+ deserializationSchema,
+ shardAssigner,
+ periodicWatermarkAssigner,
+ watermarkTracker);
+ }
+
+ @VisibleForTesting
+ HashMap<StreamShardMetadata.EquivalenceWrapper, SequenceNumber> getRestoredState() {
+ return sequenceNumsToRestore;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisException.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisException.java
new file mode 100644
index 0000000..201324a
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisException.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A {@link RuntimeException} wrapper indicating the exception was thrown from this connector. This
+ * class is abstract, semantic subclasses should be created to indicate the type of exception.
+ */
+@Internal
+public abstract class FlinkKinesisException extends RuntimeException {
+
+ public FlinkKinesisException(final String message) {
+ super(message);
+ }
+
+ public FlinkKinesisException(final String message, final Throwable cause) {
+ super(message, cause);
+ }
+
+ /**
+ * A semantic {@link RuntimeException} thrown to indicate timeout errors in the Kinesis
+ * connector.
+ */
+ @Internal
+ public static class FlinkKinesisTimeoutException extends FlinkKinesisException {
+
+ public FlinkKinesisTimeoutException(String message) {
+ super(message);
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
new file mode 100644
index 0000000..338cd28
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java
@@ -0,0 +1,500 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+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.configuration.Configuration;
+import org.apache.flink.connector.kinesis.sink.KinesisStreamsSink;
+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.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.TimeoutLatch;
+import org.apache.flink.util.InstantiationUtil;
+
+import com.amazonaws.metrics.AwsSdkMetrics;
+import com.amazonaws.services.kinesis.producer.Attempt;
+import com.amazonaws.services.kinesis.producer.KinesisProducer;
+import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration;
+import com.amazonaws.services.kinesis.producer.UserRecordFailedException;
+import com.amazonaws.services.kinesis.producer.UserRecordResult;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The FlinkKinesisProducer allows to produce from a Flink DataStream into Kinesis.
+ *
+ * @param <OUT> Data type to produce into Kinesis Streams
+ * @deprecated This producer based on the Kinesis Producer Library KPL has been superseded. The new
+ * sink can be found in the module {@code flink-connectors/flink-connector-aws-kinesis-streams}
+ * and package {@link KinesisStreamsSink}. It is based on the AWS SDK for Java 2.x. The work to
+ * replace this sink was carried out in FLINK-24227.
+ */
+@Deprecated
+@PublicEvolving
+public class FlinkKinesisProducer<OUT> extends RichSinkFunction<OUT>
+ implements CheckpointedFunction {
+
+ public static final String KINESIS_PRODUCER_METRIC_GROUP = "kinesisProducer";
+
+ public static final String METRIC_BACKPRESSURE_CYCLES = "backpressureCycles";
+
+ public static final String METRIC_OUTSTANDING_RECORDS_COUNT = "outstandingRecordsCount";
+
+ public static final String KINESIS_PRODUCER_RELEASE_HOOK_NAME = "kinesisProducer";
+
+ private static final long serialVersionUID = 6447077318449477846L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisProducer.class);
+
+ /** Properties to parametrize settings such as AWS service region, access key etc. */
+ private final Properties configProps;
+
+ /* Flag controlling the error behavior of the producer */
+ private boolean failOnError = false;
+
+ /* Maximum length of the internal record queue before backpressuring */
+ private int queueLimit = Integer.MAX_VALUE;
+
+ /* Name of the default stream to produce to. Can be overwritten by the serialization schema */
+ private String defaultStream;
+
+ /* Default partition id. Can be overwritten by the serialization schema */
+ private String defaultPartition;
+
+ /* Schema for turning the OUT type into a byte array. */
+ private final KinesisSerializationSchema<OUT> schema;
+
+ /* Optional custom partitioner */
+ private KinesisPartitioner<OUT> customPartitioner = null;
+
+ // --------------------------- Runtime fields ---------------------------
+
+ /* Our Kinesis instance for each parallel Flink sink */
+ private transient KinesisProducer producer;
+
+ /* Backpressuring waits for this latch, triggered by record callback */
+ private transient volatile TimeoutLatch backpressureLatch;
+
+ /* Callback handling failures */
+ private transient FutureCallback<UserRecordResult> callback;
+
+ /* Counts how often we have to wait for KPL because we are above the queue limit */
+ private transient Counter backpressureCycles;
+
+ /* Field for async exception */
+ private transient volatile Throwable thrownException;
+
+ // --------------------------- Initialization and configuration ---------------------------
+
+ /**
+ * Create a new FlinkKinesisProducer. This is a constructor supporting Flink's {@see
+ * SerializationSchema}.
+ *
+ * @param schema Serialization schema for the data type
+ * @param configProps The properties used to configure KinesisProducer, including AWS
+ * credentials and AWS region
+ */
+ public FlinkKinesisProducer(final SerializationSchema<OUT> schema, Properties configProps) {
+
+ // create a simple wrapper for the serialization schema
+ this(
+ new KinesisSerializationSchema<OUT>() {
+
+ @Override
+ public void open(SerializationSchema.InitializationContext context)
+ throws Exception {
+ schema.open(context);
+ }
+
+ @Override
+ public ByteBuffer serialize(OUT element) {
+ // wrap into ByteBuffer
+ return ByteBuffer.wrap(schema.serialize(element));
+ }
+ // use default stream and hash key
+
+ @Override
+ public String getTargetStream(OUT element) {
+ return null;
+ }
+ },
+ configProps);
+ }
+
+ /**
+ * Create a new FlinkKinesisProducer. This is a constructor supporting {@see
+ * KinesisSerializationSchema}.
+ *
+ * @param schema Kinesis serialization schema for the data type
+ * @param configProps The properties used to configure KinesisProducer, including AWS
+ * credentials and AWS region
+ */
+ public FlinkKinesisProducer(KinesisSerializationSchema<OUT> schema, Properties configProps) {
+ checkNotNull(configProps, "configProps can not be null");
+ this.configProps = KinesisConfigUtil.replaceDeprecatedProducerKeys(configProps);
+
+ checkNotNull(schema, "serialization schema cannot be null");
+ checkArgument(
+ InstantiationUtil.isSerializable(schema),
+ "The provided serialization schema is not serializable: "
+ + schema.getClass().getName()
+ + ". "
+ + "Please check that it does not contain references to non-serializable instances.");
+ this.schema = schema;
+ }
+
+ /**
+ * If set to true, the producer will immediately fail with an exception on any error. Otherwise,
+ * the errors are logged and the producer goes on.
+ *
+ * @param failOnError Error behavior flag
+ */
+ public void setFailOnError(boolean failOnError) {
+ this.failOnError = failOnError;
+ }
+
+ /**
+ * The {@link KinesisProducer} holds an unbounded queue internally. To avoid memory problems
+ * under high loads, a limit can be employed above which the internal queue will be flushed,
+ * thereby applying backpressure.
+ *
+ * @param queueLimit The maximum length of the internal queue before backpressuring
+ */
+ public void setQueueLimit(int queueLimit) {
+ checkArgument(queueLimit > 0, "queueLimit must be a positive number");
+ this.queueLimit = queueLimit;
+ }
+
+ /**
+ * Set a default stream name.
+ *
+ * @param defaultStream Name of the default Kinesis stream
+ */
+ public void setDefaultStream(String defaultStream) {
+ this.defaultStream = defaultStream;
+ }
+
+ /**
+ * Set default partition id.
+ *
+ * @param defaultPartition Name of the default partition
+ */
+ public void setDefaultPartition(String defaultPartition) {
+ this.defaultPartition = defaultPartition;
+ }
+
+ public void setCustomPartitioner(KinesisPartitioner<OUT> partitioner) {
+ checkNotNull(partitioner, "partitioner cannot be null");
+ checkArgument(
+ InstantiationUtil.isSerializable(partitioner),
+ "The provided custom partitioner is not serializable: "
+ + partitioner.getClass().getName()
+ + ". "
+ + "Please check that it does not contain references to non-serializable instances.");
+
+ this.customPartitioner = partitioner;
+ }
+
+ // --------------------------- Lifecycle methods ---------------------------
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+
+ schema.open(
+ RuntimeContextInitializationContextAdapters.serializationAdapter(
+ getRuntimeContext(), metricGroup -> metricGroup.addGroup("user")));
+
+ // check and pass the configuration properties
+ KinesisProducerConfiguration producerConfig =
+ KinesisConfigUtil.getValidatedProducerConfiguration(configProps);
+
+ producer = getKinesisProducer(producerConfig);
+
+ final MetricGroup kinesisMectricGroup =
+ getRuntimeContext().getMetricGroup().addGroup(KINESIS_PRODUCER_METRIC_GROUP);
+ this.backpressureCycles = kinesisMectricGroup.counter(METRIC_BACKPRESSURE_CYCLES);
+ kinesisMectricGroup.gauge(
+ METRIC_OUTSTANDING_RECORDS_COUNT, producer::getOutstandingRecordsCount);
+
+ backpressureLatch = new TimeoutLatch();
+ callback =
+ new FutureCallback<UserRecordResult>() {
+ @Override
+ public void onSuccess(UserRecordResult result) {
+ backpressureLatch.trigger();
+ if (!result.isSuccessful()) {
+ if (failOnError) {
+ // only remember the first thrown exception
+ if (thrownException == null) {
+ thrownException =
+ new RuntimeException("Record was not sent successful");
+ }
+ } else {
+ LOG.warn("Record was not sent successful");
+ }
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ backpressureLatch.trigger();
+ if (failOnError) {
+ thrownException = t;
+ } else {
+ LOG.warn("An exception occurred while processing a record", t);
+ }
+ }
+ };
+
+ if (this.customPartitioner != null) {
+ this.customPartitioner.initialize(
+ getRuntimeContext().getIndexOfThisSubtask(),
+ getRuntimeContext().getNumberOfParallelSubtasks());
+ }
+
+ final RuntimeContext ctx = getRuntimeContext();
+ ctx.registerUserCodeClassLoaderReleaseHookIfAbsent(
+ KINESIS_PRODUCER_RELEASE_HOOK_NAME,
+ () -> this.runClassLoaderReleaseHook(ctx.getUserCodeClassLoader()));
+
+ LOG.info("Started Kinesis producer instance for region '{}'", producerConfig.getRegion());
+ }
+
+ @Override
+ public void invoke(OUT value, Context context) throws Exception {
+ if (this.producer == null) {
+ throw new RuntimeException("Kinesis producer has been closed");
+ }
+
+ checkAndPropagateAsyncError();
+ boolean didWaitForFlush = enforceQueueLimit();
+
+ if (didWaitForFlush) {
+ checkAndPropagateAsyncError();
+ }
+
+ String stream = defaultStream;
+ String partition = defaultPartition;
+
+ ByteBuffer serialized = schema.serialize(value);
+
+ // maybe set custom stream
+ String customStream = schema.getTargetStream(value);
+ if (customStream != null) {
+ stream = customStream;
+ }
+
+ String explicitHashkey = null;
+ // maybe set custom partition
+ if (customPartitioner != null) {
+ partition = customPartitioner.getPartitionId(value);
+ explicitHashkey = customPartitioner.getExplicitHashKey(value);
+ }
+
+ if (stream == null) {
+ if (failOnError) {
+ throw new RuntimeException("No target stream set");
+ } else {
+ LOG.warn("No target stream set. Skipping record");
+ return;
+ }
+ }
+
+ ListenableFuture<UserRecordResult> cb =
+ producer.addUserRecord(stream, partition, explicitHashkey, serialized);
+ Futures.addCallback(cb, callback, MoreExecutors.directExecutor());
+ }
+
+ @Override
+ public void close() throws Exception {
+ LOG.info("Closing producer");
+ super.close();
+
+ if (producer != null) {
+ LOG.info("Flushing outstanding {} records", producer.getOutstandingRecordsCount());
+ // try to flush all outstanding records
+ flushSync();
+
+ LOG.info("Flushing done. Destroying producer instance.");
+ producer.destroy();
+ producer = null;
+ }
+
+ // make sure we propagate pending errors
+ checkAndPropagateAsyncError();
+ }
+
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ // nothing to do
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ // check for asynchronous errors and fail the checkpoint if necessary
+ checkAndPropagateAsyncError();
+
+ flushSync();
+ if (producer.getOutstandingRecordsCount() > 0) {
+ throw new IllegalStateException(
+ "Number of outstanding records must be zero at this point: "
+ + producer.getOutstandingRecordsCount());
+ }
+
+ // if the flushed requests has errors, we should propagate it also and fail the checkpoint
+ checkAndPropagateAsyncError();
+ }
+
+ // --------------------------- Utilities ---------------------------
+
+ /**
+ * Creates a {@link KinesisProducer}. Exposed so that tests can inject mock producers easily.
+ */
+ @VisibleForTesting
+ protected KinesisProducer getKinesisProducer(KinesisProducerConfiguration producerConfig) {
+ return new KinesisProducer(producerConfig);
+ }
+
+ /** Check if there are any asynchronous exceptions. If so, rethrow the exception. */
+ private void checkAndPropagateAsyncError() throws Exception {
+ if (thrownException != null) {
+ String errorMessages = "";
+ if (thrownException instanceof UserRecordFailedException) {
+ List<Attempt> attempts =
+ ((UserRecordFailedException) thrownException).getResult().getAttempts();
+ for (Attempt attempt : attempts) {
+ if (attempt.getErrorMessage() != null) {
+ errorMessages += attempt.getErrorMessage() + "\n";
+ }
+ }
+ }
+ if (failOnError) {
+ throw new RuntimeException(
+ "An exception was thrown while processing a record: " + errorMessages,
+ thrownException);
+ } else {
+ LOG.warn(
+ "An exception was thrown while processing a record: {}.",
+ errorMessages,
+ thrownException);
+
+ // reset, prevent double throwing
+ thrownException = null;
+ }
+ }
+ }
+
+ /**
+ * If the internal queue of the {@link KinesisProducer} gets too long, flush some of the records
+ * until we are below the limit again. We don't want to flush _all_ records at this point since
+ * that would break record aggregation.
+ *
+ * @return boolean whether flushing occurred or not
+ */
+ private boolean enforceQueueLimit() {
+ int attempt = 0;
+ while (producer.getOutstandingRecordsCount() >= queueLimit) {
+ backpressureCycles.inc();
+ if (attempt >= 10) {
+ LOG.warn(
+ "Waiting for the queue length to drop below the limit takes unusually long, still not done after {} attempts.",
+ attempt);
+ }
+ attempt++;
+ try {
+ backpressureLatch.await(100);
+ } catch (InterruptedException e) {
+ LOG.warn("Flushing was interrupted.");
+ break;
+ }
+ }
+ return attempt > 0;
+ }
+
+ /**
+ * A reimplementation of {@link KinesisProducer#flushSync()}. This implementation releases the
+ * block on flushing if an interruption occurred.
+ */
+ private void flushSync() throws Exception {
+ while (producer.getOutstandingRecordsCount() > 0) {
+ producer.flush();
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ LOG.warn("Flushing was interrupted.");
+ break;
+ }
+ }
+ }
+
+ /**
+ * Remove references created by the producer, preventing the classloader to unload. References
+ * were analyzed as of versions: aws.kinesis-kpl.version = 0.14.0 aws.sdk.version = 1.11.754
+ * aws.sdkv2.version = 2.13.52
+ */
+ private void runClassLoaderReleaseHook(ClassLoader classLoader) {
+ AwsSdkMetrics.unregisterMetricAdminMBean();
+
+ // shutdown FileAgeManager thread pool
+ try {
+ Class<?> fileAgeManagerClazz =
+ Class.forName(
+ "com.amazonaws.services.kinesis.producer.FileAgeManager",
+ true,
+ classLoader);
+ Field instanceField = fileAgeManagerClazz.getDeclaredField("instance");
+ instanceField.setAccessible(true);
+ Object fileAgeManager = instanceField.get(null);
+
+ Field executorField = fileAgeManagerClazz.getDeclaredField("executorService");
+ executorField.setAccessible(true);
+ ExecutorService executorService = (ExecutorService) executorField.get(fileAgeManager);
+ executorService.shutdown();
+ executorService.awaitTermination(1, TimeUnit.MINUTES);
+ } catch (ClassNotFoundException
+ | NoSuchFieldException
+ | IllegalAccessException
+ | InterruptedException e) {
+ LOG.info("Unable to shutdown thread pool of KinesisProducer/FileAgeManager.", e);
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
new file mode 100644
index 0000000..c93c9da
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisPartitioner.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.kinesis.sink.PartitionKeyGenerator;
+
+import java.io.Serializable;
+
+/**
+ * An interface for partitioning records.
+ *
+ * @param <T> record type
+ */
+@PublicEvolving
+public abstract class KinesisPartitioner<T> implements Serializable, PartitionKeyGenerator<T> {
+
+ private static final long serialVersionUID = -7467294664702189780L;
+
+ /**
+ * Return a partition id based on the input.
+ *
+ * @param element Element to partition
+ * @return A string representing the partition id
+ */
+ public abstract String getPartitionId(T element);
+
+ /**
+ * Optional method for setting an explicit hash key.
+ *
+ * @param element Element to get the hash key for
+ * @return the hash key for the element
+ */
+ public String getExplicitHashKey(T element) {
+ return null;
+ }
+
+ /**
+ * Optional initializer.
+ *
+ * @param indexOfThisSubtask Index of this partitioner instance
+ * @param numberOfParallelSubtasks Total number of parallel instances
+ */
+ public void initialize(int indexOfThisSubtask, int numberOfParallelSubtasks) {}
+
+ @Override
+ public String apply(T element) {
+ return getPartitionId(element);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisShardAssigner.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisShardAssigner.java
new file mode 100644
index 0000000..322b548
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisShardAssigner.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import java.io.Serializable;
+
+/**
+ * Utility to map Kinesis shards to Flink subtask indices. Users can implement this interface to
+ * optimize distribution of shards over subtasks. See {@link #assign(StreamShardHandle, int)} for
+ * details.
+ */
+@PublicEvolving
+public interface KinesisShardAssigner extends Serializable {
+
+ /**
+ * Returns the index of the target subtask that a specific shard should be assigned to. For
+ * return values outside the subtask range, modulus operation will be applied automatically,
+ * hence it is also valid to just return a hash code.
+ *
+ * <p>The resulting distribution of shards should have the following contract:
+ *
+ * <ul>
+ * <li>1. Uniform distribution across subtasks
+ * <li>2. Deterministic, calls for a given shard always return same index.
+ * </ul>
+ *
+ * <p>The above contract is crucial and cannot be broken. Consumer subtasks rely on this
+ * contract to filter out shards that they should not subscribe to, guaranteeing that each shard
+ * of a stream will always be assigned to one subtask in a uniformly distributed manner.
+ *
+ * @param shard the shard to determine
+ * @param numParallelSubtasks total number of subtasks
+ * @return target index, if index falls outside of the range, modulus operation will be applied
+ */
+ int assign(StreamShardHandle shard, int numParallelSubtasks);
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
new file mode 100644
index 0000000..4c1bda4
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/AWSConfigConstants.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * Class inheriting from {@link org.apache.flink.connector.aws.config.AWSConfigConstants} for
+ * backward compatibility.
+ */
+@PublicEvolving
+public class AWSConfigConstants extends org.apache.flink.connector.aws.config.AWSConfigConstants {}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
new file mode 100644
index 0000000..fff44d6
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
@@ -0,0 +1,427 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+
+import java.time.Duration;
+
+/**
+ * Optional consumer specific configuration keys and default values for {@link
+ * FlinkKinesisConsumer}.
+ */
+@PublicEvolving
+public class ConsumerConfigConstants extends AWSConfigConstants {
+
+ /**
+ * The initial position to start reading shards from. This will affect the {@code
+ * ShardIteratorType} used when the consumer tasks retrieve the first shard iterator for each
+ * Kinesis shard.
+ */
+ public enum InitialPosition {
+
+ /**
+ * Start reading from the earliest possible record in the stream (excluding expired data
+ * records).
+ */
+ TRIM_HORIZON(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM),
+
+ /** Start reading from the latest incoming record. */
+ LATEST(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM),
+
+ /** Start reading from the record at the specified timestamp. */
+ AT_TIMESTAMP(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM);
+
+ private SentinelSequenceNumber sentinelSequenceNumber;
+
+ InitialPosition(SentinelSequenceNumber sentinelSequenceNumber) {
+ this.sentinelSequenceNumber = sentinelSequenceNumber;
+ }
+
+ public SentinelSequenceNumber toSentinelSequenceNumber() {
+ return this.sentinelSequenceNumber;
+ }
+ }
+
+ /** The record publisher type represents the record-consume style. */
+ public enum RecordPublisherType {
+
+ /** Consume the Kinesis records using AWS SDK v2 with the enhanced fan-out consumer. */
+ EFO,
+ /** Consume the Kinesis records using AWS SDK v1 with the get-records method. */
+ POLLING
+ }
+
+ /** The EFO registration type represents how we are going to de-/register efo consumer. */
+ public enum EFORegistrationType {
+
+ /**
+ * Delay the registration of efo consumer for taskmanager to execute. De-register the efo
+ * consumer for taskmanager to execute when task is shut down.
+ */
+ LAZY,
+ /**
+ * Register the efo consumer eagerly for jobmanager to execute. De-register the efo consumer
+ * the same way as lazy does.
+ */
+ EAGER,
+ /** Do not register efo consumer programmatically. Do not de-register either. */
+ NONE
+ }
+
+ /** The RecordPublisher type (EFO|POLLING, default is POLLING). */
+ public static final String RECORD_PUBLISHER_TYPE = "flink.stream.recordpublisher";
+
+ /** The name of the EFO consumer to register with KDS. */
+ public static final String EFO_CONSUMER_NAME = "flink.stream.efo.consumername";
+
+ /**
+ * Determine how and when consumer de-/registration is performed (LAZY|EAGER|NONE, default is
+ * LAZY).
+ */
+ public static final String EFO_REGISTRATION_TYPE = "flink.stream.efo.registration";
+
+ /** The prefix of consumer ARN for a given stream. */
+ public static final String EFO_CONSUMER_ARN_PREFIX = "flink.stream.efo.consumerarn";
+
+ /** The initial position to start reading Kinesis streams from (LATEST is used if not set). */
+ public static final String STREAM_INITIAL_POSITION = "flink.stream.initpos";
+
+ /**
+ * The initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP is set for
+ * STREAM_INITIAL_POSITION).
+ */
+ public static final String STREAM_INITIAL_TIMESTAMP = "flink.stream.initpos.timestamp";
+
+ /**
+ * The date format of initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP
+ * is set for STREAM_INITIAL_POSITION).
+ */
+ public static final String STREAM_TIMESTAMP_DATE_FORMAT =
+ "flink.stream.initpos.timestamp.format";
+
+ /** The maximum number of describeStream attempts if we get a recoverable exception. */
+ public static final String STREAM_DESCRIBE_RETRIES = "flink.stream.describe.maxretries";
+
+ /**
+ * The base backoff time between each describeStream attempt (for consuming from DynamoDB
+ * streams).
+ */
+ public static final String STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base";
+
+ /**
+ * The maximum backoff time between each describeStream attempt (for consuming from DynamoDB
+ * streams).
+ */
+ public static final String STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max";
+
+ /**
+ * The power constant for exponential backoff between each describeStream attempt (for consuming
+ * from DynamoDB streams).
+ */
+ public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.stream.describe.backoff.expconst";
+
+ /** The maximum number of listShards attempts if we get a recoverable exception. */
+ public static final String LIST_SHARDS_RETRIES = "flink.list.shards.maxretries";
+
+ /** The base backoff time between each listShards attempt. */
+ public static final String LIST_SHARDS_BACKOFF_BASE = "flink.list.shards.backoff.base";
+
+ /** The maximum backoff time between each listShards attempt. */
+ public static final String LIST_SHARDS_BACKOFF_MAX = "flink.list.shards.backoff.max";
+
+ /** The power constant for exponential backoff between each listShards attempt. */
+ public static final String LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.list.shards.backoff.expconst";
+
+ /** The maximum number of describeStreamConsumer attempts if we get a recoverable exception. */
+ public static final String DESCRIBE_STREAM_CONSUMER_RETRIES =
+ "flink.stream.describestreamconsumer.maxretries";
+
+ /** The base backoff time between each describeStreamConsumer attempt. */
+ public static final String DESCRIBE_STREAM_CONSUMER_BACKOFF_BASE =
+ "flink.stream.describestreamconsumer.backoff.base";
+
+ /** The maximum backoff time between each describeStreamConsumer attempt. */
+ public static final String DESCRIBE_STREAM_CONSUMER_BACKOFF_MAX =
+ "flink.stream.describestreamconsumer.backoff.max";
+
+ /** The power constant for exponential backoff between each describeStreamConsumer attempt. */
+ public static final String DESCRIBE_STREAM_CONSUMER_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.stream.describestreamconsumer.backoff.expconst";
+
+ /** The maximum number of registerStream attempts if we get a recoverable exception. */
+ public static final String REGISTER_STREAM_RETRIES =
+ "flink.stream.registerstreamconsumer.maxretries";
+
+ /**
+ * The maximum time in seconds to wait for a stream consumer to become active before giving up.
+ */
+ public static final String REGISTER_STREAM_TIMEOUT_SECONDS =
+ "flink.stream.registerstreamconsumer.timeout";
+
+ /** The base backoff time between each registerStream attempt. */
+ public static final String REGISTER_STREAM_BACKOFF_BASE =
+ "flink.stream.registerstreamconsumer.backoff.base";
+
+ /** The maximum backoff time between each registerStream attempt. */
+ public static final String REGISTER_STREAM_BACKOFF_MAX =
+ "flink.stream.registerstreamconsumer.backoff.max";
+
+ /** The power constant for exponential backoff between each registerStream attempt. */
+ public static final String REGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.stream.registerstreamconsumer.backoff.expconst";
+
+ /** The maximum number of deregisterStream attempts if we get a recoverable exception. */
+ public static final String DEREGISTER_STREAM_RETRIES =
+ "flink.stream.deregisterstreamconsumer.maxretries";
+
+ /** The maximum time in seconds to wait for a stream consumer to deregister before giving up. */
+ public static final String DEREGISTER_STREAM_TIMEOUT_SECONDS =
+ "flink.stream.deregisterstreamconsumer.timeout";
+
+ /** The base backoff time between each deregisterStream attempt. */
+ public static final String DEREGISTER_STREAM_BACKOFF_BASE =
+ "flink.stream.deregisterstreamconsumer.backoff.base";
+
+ /** The maximum backoff time between each deregisterStream attempt. */
+ public static final String DEREGISTER_STREAM_BACKOFF_MAX =
+ "flink.stream.deregisterstreamconsumer.backoff.max";
+
+ /** The power constant for exponential backoff between each deregisterStream attempt. */
+ public static final String DEREGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.stream.deregisterstreamconsumer.backoff.expconst";
+
+ /** The maximum number of subscribeToShard attempts if we get a recoverable exception. */
+ public static final String SUBSCRIBE_TO_SHARD_RETRIES =
+ "flink.shard.subscribetoshard.maxretries";
+
+ /** A timeout when waiting for a shard subscription to be established. */
+ public static final String SUBSCRIBE_TO_SHARD_TIMEOUT_SECONDS =
+ "flink.shard.subscribetoshard.timeout";
+
+ /** The base backoff time between each subscribeToShard attempt. */
+ public static final String SUBSCRIBE_TO_SHARD_BACKOFF_BASE =
+ "flink.shard.subscribetoshard.backoff.base";
+
+ /** The maximum backoff time between each subscribeToShard attempt. */
+ public static final String SUBSCRIBE_TO_SHARD_BACKOFF_MAX =
+ "flink.shard.subscribetoshard.backoff.max";
+
+ /** The power constant for exponential backoff between each subscribeToShard attempt. */
+ public static final String SUBSCRIBE_TO_SHARD_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.shard.subscribetoshard.backoff.expconst";
+
+ /**
+ * The maximum number of records to try to get each time we fetch records from a AWS Kinesis
+ * shard.
+ */
+ public static final String SHARD_GETRECORDS_MAX = "flink.shard.getrecords.maxrecordcount";
+
+ /** The maximum number of getRecords attempts if we get a recoverable exception. */
+ public static final String SHARD_GETRECORDS_RETRIES = "flink.shard.getrecords.maxretries";
+
+ /**
+ * The base backoff time between getRecords attempts if we get a
+ * ProvisionedThroughputExceededException.
+ */
+ public static final String SHARD_GETRECORDS_BACKOFF_BASE =
+ "flink.shard.getrecords.backoff.base";
+
+ /**
+ * The maximum backoff time between getRecords attempts if we get a
+ * ProvisionedThroughputExceededException.
+ */
+ public static final String SHARD_GETRECORDS_BACKOFF_MAX = "flink.shard.getrecords.backoff.max";
+
+ /** The power constant for exponential backoff between each getRecords attempt. */
+ public static final String SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.shard.getrecords.backoff.expconst";
+
+ /** The interval between each getRecords request to a AWS Kinesis shard in milliseconds. */
+ public static final String SHARD_GETRECORDS_INTERVAL_MILLIS =
+ "flink.shard.getrecords.intervalmillis";
+
+ /**
+ * The maximum number of getShardIterator attempts if we get
+ * ProvisionedThroughputExceededException.
+ */
+ public static final String SHARD_GETITERATOR_RETRIES = "flink.shard.getiterator.maxretries";
+
+ /**
+ * The base backoff time between getShardIterator attempts if we get a
+ * ProvisionedThroughputExceededException.
+ */
+ public static final String SHARD_GETITERATOR_BACKOFF_BASE =
+ "flink.shard.getiterator.backoff.base";
+
+ /**
+ * The maximum backoff time between getShardIterator attempts if we get a
+ * ProvisionedThroughputExceededException.
+ */
+ public static final String SHARD_GETITERATOR_BACKOFF_MAX =
+ "flink.shard.getiterator.backoff.max";
+
+ /** The power constant for exponential backoff between each getShardIterator attempt. */
+ public static final String SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT =
+ "flink.shard.getiterator.backoff.expconst";
+
+ /** The interval between each attempt to discover new shards. */
+ public static final String SHARD_DISCOVERY_INTERVAL_MILLIS =
+ "flink.shard.discovery.intervalmillis";
+
+ /** The config to turn on adaptive reads from a shard. */
+ public static final String SHARD_USE_ADAPTIVE_READS = "flink.shard.adaptivereads";
+
+ /** The interval after which to consider a shard idle for purposes of watermark generation. */
+ public static final String SHARD_IDLE_INTERVAL_MILLIS = "flink.shard.idle.interval";
+
+ /** The interval for periodically synchronizing the shared watermark state. */
+ public static final String WATERMARK_SYNC_MILLIS = "flink.watermark.sync.interval";
+
+ /** The maximum delta allowed for the reader to advance ahead of the shared global watermark. */
+ public static final String WATERMARK_LOOKAHEAD_MILLIS = "flink.watermark.lookahead.millis";
+
+ /**
+ * The maximum number of records that will be buffered before suspending consumption of a shard.
+ */
+ public static final String WATERMARK_SYNC_QUEUE_CAPACITY =
+ "flink.watermark.sync.queue.capacity";
+
+ public static final String EFO_HTTP_CLIENT_MAX_CONCURRENCY =
+ "flink.stream.efo.http-client.max-concurrency";
+
+ public static final String EFO_HTTP_CLIENT_READ_TIMEOUT_MILLIS =
+ "flink.stream.efo.http-client.read-timeout";
+
+ // ------------------------------------------------------------------------
+ // Default values for consumer configuration
+ // ------------------------------------------------------------------------
+
+ public static final String DEFAULT_STREAM_INITIAL_POSITION = InitialPosition.LATEST.toString();
+
+ public static final String DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT =
+ "yyyy-MM-dd'T'HH:mm:ss.SSSXXX";
+
+ public static final int DEFAULT_STREAM_DESCRIBE_RETRIES = 50;
+
+ public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 2000L;
+
+ public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L;
+
+ public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final long DEFAULT_LIST_SHARDS_BACKOFF_BASE = 1000L;
+
+ public static final long DEFAULT_LIST_SHARDS_BACKOFF_MAX = 5000L;
+
+ public static final double DEFAULT_LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final int DEFAULT_LIST_SHARDS_RETRIES = 10;
+
+ public static final int DEFAULT_DESCRIBE_STREAM_CONSUMER_RETRIES = 50;
+
+ public static final long DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_BASE = 2000L;
+
+ public static final long DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_MAX = 5000L;
+
+ public static final double DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final int DEFAULT_REGISTER_STREAM_RETRIES = 10;
+
+ public static final Duration DEFAULT_REGISTER_STREAM_TIMEOUT = Duration.ofSeconds(60);
+
+ public static final long DEFAULT_REGISTER_STREAM_BACKOFF_BASE = 500L;
+
+ public static final long DEFAULT_REGISTER_STREAM_BACKOFF_MAX = 2000L;
+
+ public static final double DEFAULT_REGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final int DEFAULT_DEREGISTER_STREAM_RETRIES = 10;
+
+ public static final Duration DEFAULT_DEREGISTER_STREAM_TIMEOUT = Duration.ofSeconds(60);
+
+ public static final long DEFAULT_DEREGISTER_STREAM_BACKOFF_BASE = 500L;
+
+ public static final long DEFAULT_DEREGISTER_STREAM_BACKOFF_MAX = 2000L;
+
+ public static final double DEFAULT_DEREGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final int DEFAULT_SUBSCRIBE_TO_SHARD_RETRIES = 10;
+
+ public static final Duration DEFAULT_SUBSCRIBE_TO_SHARD_TIMEOUT = Duration.ofSeconds(60);
+
+ public static final long DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_BASE = 1000L;
+
+ public static final long DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_MAX = 2000L;
+
+ public static final double DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final int DEFAULT_SHARD_GETRECORDS_MAX = 10000;
+
+ public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3;
+
+ public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L;
+
+ public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L;
+
+ public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final long DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS = 200L;
+
+ public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3;
+
+ public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L;
+
+ public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L;
+
+ public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
+
+ public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L;
+
+ public static final boolean DEFAULT_SHARD_USE_ADAPTIVE_READS = false;
+
+ public static final long DEFAULT_SHARD_IDLE_INTERVAL_MILLIS = -1;
+
+ public static final long DEFAULT_WATERMARK_SYNC_MILLIS = 30_000;
+
+ public static final int DEFAULT_EFO_HTTP_CLIENT_MAX_CONCURRENCY = 10_000;
+
+ public static final Duration DEFAULT_EFO_HTTP_CLIENT_READ_TIMEOUT = Duration.ofMinutes(6);
+
+ /**
+ * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured
+ * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved
+ * iterators.
+ */
+ public static final long MAX_SHARD_GETRECORDS_INTERVAL_MILLIS = 300000L;
+
+ /**
+ * Build the key of an EFO consumer ARN according to a stream name.
+ *
+ * @param streamName the stream name the key is built upon.
+ * @return a key of EFO consumer ARN.
+ */
+ public static String efoConsumerArn(final String streamName) {
+ return EFO_CONSUMER_ARN_PREFIX + "." + streamName;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
new file mode 100644
index 0000000..539ed93
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.config;
+
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+
+/**
+ * Optional producer specific configuration keys for {@link FlinkKinesisProducer}.
+ *
+ * @deprecated This class is deprecated in favor of the official AWS Kinesis producer configuration
+ * keys. See <a
+ * href="https://github.com/awslabs/amazon-kinesis-producer/blob/master/java/amazon-kinesis-producer-sample/default_config.properties">
+ * here</a> for the full list of available configs. For configuring the region and credentials,
+ * please use the keys in {@link
+ * org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants}.
+ */
+@Deprecated
+public class ProducerConfigConstants extends AWSConfigConstants {
+
+ /**
+ * Deprecated key.
+ *
+ * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration
+ * keys. Please use {@code CollectionMaxCount} instead.
+ */
+ @Deprecated public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount";
+
+ /**
+ * Deprecated key.
+ *
+ * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration
+ * keys. Please use {@code AggregationMaxCount} instead.
+ */
+ @Deprecated
+ public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount";
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java
new file mode 100644
index 0000000..e3926d5
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling.PollingRecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.model.DynamoDBStreamsShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.DynamoDBStreamsProxy;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Dynamodb streams data fetcher.
+ *
+ * @param <T> type of fetched data.
+ */
+public class DynamoDBStreamsDataFetcher<T> extends KinesisDataFetcher<T> {
+ private final RecordPublisherFactory recordPublisherFactory;
+
+ /**
+ * Constructor.
+ *
+ * @param streams list of streams to fetch data
+ * @param sourceContext source context
+ * @param runtimeContext runtime context
+ * @param configProps config properties
+ * @param deserializationSchema deserialization schema
+ * @param shardAssigner shard assigner
+ */
+ public DynamoDBStreamsDataFetcher(
+ List<String> streams,
+ SourceFunction.SourceContext<T> sourceContext,
+ RuntimeContext runtimeContext,
+ Properties configProps,
+ KinesisDeserializationSchema<T> deserializationSchema,
+ KinesisShardAssigner shardAssigner) {
+
+ this(
+ streams,
+ sourceContext,
+ runtimeContext,
+ configProps,
+ deserializationSchema,
+ shardAssigner,
+ DynamoDBStreamsProxy::create);
+ }
+
+ @VisibleForTesting
+ DynamoDBStreamsDataFetcher(
+ List<String> streams,
+ SourceFunction.SourceContext<T> sourceContext,
+ RuntimeContext runtimeContext,
+ Properties configProps,
+ KinesisDeserializationSchema<T> deserializationSchema,
+ KinesisShardAssigner shardAssigner,
+ FlinkKinesisProxyFactory flinkKinesisProxyFactory) {
+ super(
+ streams,
+ sourceContext,
+ sourceContext.getCheckpointLock(),
+ runtimeContext,
+ configProps,
+ deserializationSchema,
+ shardAssigner,
+ null,
+ null,
+ new AtomicReference<>(),
+ new ArrayList<>(),
+ createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
+ flinkKinesisProxyFactory,
+ null);
+
+ this.recordPublisherFactory = new PollingRecordPublisherFactory(flinkKinesisProxyFactory);
+ }
+
+ @Override
+ protected boolean shouldAdvanceLastDiscoveredShardId(
+ String shardId, String lastSeenShardIdOfStream) {
+ if (DynamoDBStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) {
+ // shardID update is valid only if the given shard id is greater
+ // than the previous last seen shard id of the stream.
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ protected RecordPublisher createRecordPublisher(
+ SequenceNumber sequenceNumber,
+ Properties configProps,
+ MetricGroup metricGroup,
+ StreamShardHandle subscribedShard)
+ throws InterruptedException {
+ StartingPosition startingPosition =
+ StartingPosition.continueFromSequenceNumber(sequenceNumber);
+ return recordPublisherFactory.create(
+ startingPosition, getConsumerConfiguration(), metricGroup, subscribedShard);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
new file mode 100644
index 0000000..4fcc80a
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java
@@ -0,0 +1,1460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback;
+import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.RecordPublisherType;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutRecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling.PollingRecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.metrics.KinesisConsumerMetricConstants;
+import org.apache.flink.streaming.connectors.kinesis.metrics.ShardConsumerMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
+import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Factory;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.RecordEmitter;
+import org.apache.flink.streaming.connectors.kinesis.util.StreamConsumerRegistrarUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.WatermarkTracker;
+import org.apache.flink.streaming.runtime.operators.windowing.TimestampedValue;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import com.amazonaws.services.kinesis.model.HashKeyRange;
+import com.amazonaws.services.kinesis.model.SequenceNumberRange;
+import com.amazonaws.services.kinesis.model.Shard;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.RECORD_PUBLISHER_TYPE;
+import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.RecordPublisherType.POLLING;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel
+ * subtask instantiates and runs a single fetcher throughout the subtask's lifetime. The fetcher
+ * accomplishes the following:
+ *
+ * <ul>
+ * <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The
+ * subscribed subset of shards, including future new shards, is non-overlapping across
+ * subtasks (no two subtasks will be subscribed to the same shard) and determinate across
+ * subtask restores (the subtask will always subscribe to the same subset of shards even after
+ * restoring)
+ * <li>2. decide where in each discovered shard should the fetcher start subscribing to
+ * <li>3. subscribe to shards by creating a single thread for each shard
+ * </ul>
+ *
+ * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for
+ * continuous shard discovery), and 2) last processed sequence numbers of each subscribed shard.
+ * Since operations on the second state will be performed by multiple threads, these operations
+ * should only be done using the handler methods provided in this class.
+ */
+@SuppressWarnings("unchecked")
+@Internal
+public class KinesisDataFetcher<T> {
+
+ public static final KinesisShardAssigner DEFAULT_SHARD_ASSIGNER =
+ (shard, subtasks) -> shard.hashCode();
+
+ private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
+
+ // ------------------------------------------------------------------------
+ // Consumer-wide settings
+ // ------------------------------------------------------------------------
+
+ /** Configuration properties for the Flink Kinesis Consumer. */
+ private final Properties configProps;
+
+ /** The list of Kinesis streams that the consumer is subscribing to. */
+ private final List<String> streams;
+
+ /**
+ * The deserialization schema we will be using to convert Kinesis records to Flink objects. Note
+ * that since this might not be thread-safe, {@link ShardConsumer}s using this must clone a copy
+ * using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
+ */
+ private final KinesisDeserializationSchema<T> deserializationSchema;
+
+ /** The function that determines which subtask a shard should be assigned to. */
+ private final KinesisShardAssigner shardAssigner;
+
+ // ------------------------------------------------------------------------
+ // Consumer metrics
+ // ------------------------------------------------------------------------
+
+ /** The metric group that all metrics should be registered to. */
+ private final MetricGroup consumerMetricGroup;
+
+ // ------------------------------------------------------------------------
+ // Subtask-specific settings
+ // ------------------------------------------------------------------------
+
+ /** Runtime context of the subtask that this fetcher was created in. */
+ private final RuntimeContext runtimeContext;
+
+ private final int totalNumberOfConsumerSubtasks;
+
+ private final int indexOfThisConsumerSubtask;
+
+ // ------------------------------------------------------------------------
+ // Executor services to run created threads
+ // ------------------------------------------------------------------------
+
+ /** Executor service to run {@link ShardConsumer}s to consume Kinesis shards. */
+ private final ExecutorService shardConsumersExecutor;
+
+ // ------------------------------------------------------------------------
+ // Managed state, accessed and updated across multiple threads
+ // ------------------------------------------------------------------------
+
+ /**
+ * The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new
+ * shards in. Note: this state will be updated if new shards are found when {@link
+ * KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
+ */
+ private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
+
+ /**
+ * The shards, along with their last processed sequence numbers, that this fetcher is subscribed
+ * to. The fetcher will add new subscribed shard states to this list as it discovers new shards.
+ * {@link ShardConsumer} threads update the last processed sequence number of subscribed shards
+ * as they fetch and process records.
+ *
+ * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on
+ * this list, all operations must be wrapped in synchronized blocks on the {@link
+ * KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use the following
+ * thread-safe methods this class provides to operate on this list:
+ *
+ * <ul>
+ * <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}
+ * <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}
+ * <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, long, int, SequenceNumber)}
+ * </ul>
+ */
+ private final List<KinesisStreamShardState> subscribedShardsState;
+
+ private final SourceFunction.SourceContext<T> sourceContext;
+
+ /** Checkpoint lock, also used to synchronize operations on subscribedShardsState. */
+ private final Object checkpointLock;
+
+ /** Reference to the first error thrown by any of the {@link ShardConsumer} threads. */
+ private final AtomicReference<Throwable> error;
+
+ /**
+ * The Kinesis proxy factory that will be used to create instances for discovery and shard
+ * consumers.
+ */
+ private final FlinkKinesisProxyFactory kinesisProxyFactory;
+
+ /**
+ * The Kinesis proxy V2 factory that will be used to create instances for EFO shard consumers.
+ */
+ private final FlinkKinesisProxyV2Factory kinesisProxyV2Factory;
+
+ /** The Kinesis proxy that the fetcher will be using to discover new shards. */
+ private final KinesisProxyInterface kinesis;
+
+ /** The factory used to create record publishers that consumer from Kinesis shards. */
+ private final RecordPublisherFactory recordPublisherFactory;
+
+ private final CompletableFuture<Void> cancelFuture = new CompletableFuture<>();
+
+ /**
+ * The current number of shards that are actively read by this fetcher.
+ *
+ * <p>This value is updated in {@link
+ * KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, and {@link
+ * KinesisDataFetcher#updateState(int, SequenceNumber)}.
+ */
+ private final AtomicInteger numberOfActiveShards = new AtomicInteger(0);
+
+ private volatile boolean running = true;
+
+ private final AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner;
+ private final WatermarkTracker watermarkTracker;
+ private final RecordEmitter recordEmitter;
+ private boolean isIdle;
+
+ /**
+ * The watermark related state for each shard consumer. Entries in this map will be created when
+ * shards are discovered. After recovery, this shard map will be recreated, possibly with
+ * different shard index keys, since those are transient and not part of checkpointed state.
+ */
+ private ConcurrentHashMap<Integer, ShardWatermarkState> shardWatermarks =
+ new ConcurrentHashMap<>();
+
+ /**
+ * The most recent watermark, calculated from the per shard watermarks. The initial value will
+ * never be emitted and also apply after recovery. The fist watermark that will be emitted is
+ * derived from actually consumed records. In case of recovery and replay, the watermark will
+ * rewind, consistent with the shard consumer sequence.
+ */
+ private long lastWatermark = Long.MIN_VALUE;
+
+ /**
+ * The next watermark used for synchronization. For purposes of global watermark calculation, we
+ * need to consider the next watermark based on the buffered records vs. the last emitted
+ * watermark to allow for progress.
+ */
+ private long nextWatermark = Long.MIN_VALUE;
+
+ /**
+ * The time span since last consumed record, after which a shard will be considered idle for
+ * purpose of watermark calculation. A positive value will allow the watermark to progress even
+ * when some shards don't receive new records.
+ */
+ private long shardIdleIntervalMillis =
+ ConsumerConfigConstants.DEFAULT_SHARD_IDLE_INTERVAL_MILLIS;
+
+ /** Factory to create Kinesis proxy instances used by a fetcher. */
+ public interface FlinkKinesisProxyFactory {
+ KinesisProxyInterface create(Properties configProps);
+ }
+
+ /** Factory to create Kinesis proxy V@ instances used by a fetcher. */
+ public interface FlinkKinesisProxyV2Factory {
+ KinesisProxyV2Interface create(Properties configProps);
+ }
+
+ /**
+ * The wrapper that holds the watermark handling related parameters of a record produced by the
+ * shard consumer thread.
+ *
+ * @param <T>
+ */
+ private static class RecordWrapper<T> extends TimestampedValue<T> {
+ int shardStateIndex;
+ SequenceNumber lastSequenceNumber;
+ long timestamp;
+ Watermark watermark;
+
+ private RecordWrapper(T record, long timestamp) {
+ super(record, timestamp);
+ this.timestamp = timestamp;
+ }
+
+ @Override
+ public long getTimestamp() {
+ return timestamp;
+ }
+ }
+
+ /** Kinesis data fetcher specific, asynchronous record emitter. */
+ private class AsyncKinesisRecordEmitter extends RecordEmitter<RecordWrapper<T>> {
+
+ private AsyncKinesisRecordEmitter() {
+ this(DEFAULT_QUEUE_CAPACITY);
+ }
+
+ private AsyncKinesisRecordEmitter(int queueCapacity) {
+ super(queueCapacity);
+ }
+
+ @Override
+ public void emit(RecordWrapper<T> record, RecordQueue<RecordWrapper<T>> queue) {
+ emitRecordAndUpdateState(record);
+ }
+ }
+
+ /** Synchronous emitter for use w/o watermark synchronization. */
+ private class SyncKinesisRecordEmitter extends AsyncKinesisRecordEmitter {
+ private final ConcurrentHashMap<Integer, RecordQueue<RecordWrapper<T>>> queues =
+ new ConcurrentHashMap<>();
+
+ @Override
+ public RecordQueue<RecordWrapper<T>> getQueue(int producerIndex) {
+ return queues.computeIfAbsent(
+ producerIndex,
+ (key) ->
+ new RecordQueue<RecordWrapper<T>>() {
+ @Override
+ public void put(RecordWrapper<T> record) {
+ emit(record, this);
+ }
+
+ @Override
+ public int getSize() {
+ return 0;
+ }
+
+ @Override
+ public RecordWrapper<T> peek() {
+ return null;
+ }
+ });
+ }
+ }
+
+ /**
+ * Creates a Kinesis Data Fetcher.
+ *
+ * @param streams the streams to subscribe to
+ * @param sourceContext context of the source function
+ * @param runtimeContext this subtask's runtime context
+ * @param configProps the consumer configuration properties
+ * @param deserializationSchema deserialization schema
+ */
+ public KinesisDataFetcher(
+ final List<String> streams,
+ final SourceFunction.SourceContext<T> sourceContext,
+ final RuntimeContext runtimeContext,
+ final Properties configProps,
+ final KinesisDeserializationSchema<T> deserializationSchema,
+ final KinesisShardAssigner shardAssigner,
+ final AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner,
+ final WatermarkTracker watermarkTracker) {
+ this(
+ streams,
+ sourceContext,
+ sourceContext.getCheckpointLock(),
+ runtimeContext,
+ configProps,
+ deserializationSchema,
+ shardAssigner,
+ periodicWatermarkAssigner,
+ watermarkTracker,
+ new AtomicReference<>(),
+ new ArrayList<>(),
+ createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
+ KinesisProxy::create,
+ KinesisProxyV2Factory::createKinesisProxyV2);
+ }
+
+ @VisibleForTesting
+ protected KinesisDataFetcher(
+ final List<String> streams,
+ final SourceFunction.SourceContext<T> sourceContext,
+ final Object checkpointLock,
+ final RuntimeContext runtimeContext,
+ final Properties configProps,
+ final KinesisDeserializationSchema<T> deserializationSchema,
+ final KinesisShardAssigner shardAssigner,
+ final AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner,
+ final WatermarkTracker watermarkTracker,
+ final AtomicReference<Throwable> error,
+ final List<KinesisStreamShardState> subscribedShardsState,
+ final HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
+ final FlinkKinesisProxyFactory kinesisProxyFactory,
+ @Nullable final FlinkKinesisProxyV2Factory kinesisProxyV2Factory) {
+ this.streams = checkNotNull(streams);
+ this.configProps = checkNotNull(configProps);
+ this.sourceContext = checkNotNull(sourceContext);
+ this.checkpointLock = checkNotNull(checkpointLock);
+ this.runtimeContext = checkNotNull(runtimeContext);
+ this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
+ this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
+ this.deserializationSchema = checkNotNull(deserializationSchema);
+ this.shardAssigner = checkNotNull(shardAssigner);
+ this.periodicWatermarkAssigner = periodicWatermarkAssigner;
+ this.watermarkTracker = watermarkTracker;
+ this.kinesisProxyFactory = checkNotNull(kinesisProxyFactory);
+ this.kinesisProxyV2Factory = kinesisProxyV2Factory;
+ this.kinesis = kinesisProxyFactory.create(configProps);
+ this.recordPublisherFactory = createRecordPublisherFactory();
+
+ this.consumerMetricGroup =
+ runtimeContext
+ .getMetricGroup()
+ .addGroup(KinesisConsumerMetricConstants.KINESIS_CONSUMER_METRICS_GROUP);
+
+ this.error = checkNotNull(error);
+ this.subscribedShardsState = checkNotNull(subscribedShardsState);
+ this.subscribedStreamsToLastDiscoveredShardIds =
+ checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
+
+ this.shardConsumersExecutor =
+ createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
+
+ this.recordEmitter = createRecordEmitter(configProps);
+
+ StreamConsumerRegistrarUtil.lazilyRegisterStreamConsumers(configProps, streams);
+ }
+
+ private RecordEmitter createRecordEmitter(Properties configProps) {
+ if (periodicWatermarkAssigner != null && watermarkTracker != null) {
+ int queueCapacity =
+ Integer.parseInt(
+ configProps.getProperty(
+ ConsumerConfigConstants.WATERMARK_SYNC_QUEUE_CAPACITY,
+ Integer.toString(
+ AsyncKinesisRecordEmitter.DEFAULT_QUEUE_CAPACITY)));
+ return new AsyncKinesisRecordEmitter(queueCapacity);
+ }
+ return new SyncKinesisRecordEmitter();
+ }
+
+ /**
+ * Create a new shard consumer. Override this method to customize shard consumer behavior in
+ * subclasses.
+ *
+ * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
+ * @param subscribedShard the shard this consumer is subscribed to
+ * @param lastSequenceNum the sequence number in the shard to start consuming
+ * @param metricGroup the metric group to report metrics to
+ * @return shard consumer
+ */
+ protected ShardConsumer<T> createShardConsumer(
+ final Integer subscribedShardStateIndex,
+ final StreamShardHandle subscribedShard,
+ final SequenceNumber lastSequenceNum,
+ final MetricGroup metricGroup,
+ final KinesisDeserializationSchema<T> shardDeserializer)
+ throws InterruptedException {
+
+ return new ShardConsumer<>(
+ this,
+ createRecordPublisher(lastSequenceNum, configProps, metricGroup, subscribedShard),
+ subscribedShardStateIndex,
+ subscribedShard,
+ lastSequenceNum,
+ new ShardConsumerMetricsReporter(metricGroup),
+ shardDeserializer);
+ }
+
+ protected RecordPublisherFactory createRecordPublisherFactory() {
+ RecordPublisherType recordPublisherType =
+ RecordPublisherType.valueOf(
+ configProps.getProperty(RECORD_PUBLISHER_TYPE, POLLING.name()));
+
+ switch (recordPublisherType) {
+ case EFO:
+ return new FanOutRecordPublisherFactory(kinesisProxyV2Factory.create(configProps));
+ case POLLING:
+ default:
+ return new PollingRecordPublisherFactory(kinesisProxyFactory);
+ }
+ }
+
+ protected RecordPublisher createRecordPublisher(
+ final SequenceNumber sequenceNumber,
+ final Properties configProps,
+ final MetricGroup metricGroup,
+ final StreamShardHandle subscribedShard)
+ throws InterruptedException {
+
+ StartingPosition startingPosition =
+ AWSUtil.getStartingPosition(sequenceNumber, configProps);
+ return recordPublisherFactory.create(
+ startingPosition, configProps, metricGroup, subscribedShard);
+ }
+
+ /**
+ * Starts the fetcher. After starting the fetcher, it can only be stopped by calling {@link
+ * KinesisDataFetcher#shutdownFetcher()}.
+ *
+ * @throws Exception the first error or exception thrown by the fetcher or any of the threads
+ * created by the fetcher.
+ */
+ public void runFetcher() throws Exception {
+
+ // check that we are running before proceeding
+ if (!running) {
+ return;
+ }
+
+ // ------------------------------------------------------------------------
+ // Procedures before starting the infinite while loop:
+ // ------------------------------------------------------------------------
+
+ // 1. check that there is at least one shard in the subscribed streams to consume from (can
+ // be done by
+ // checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not
+ // null)
+ boolean hasShards = false;
+ StringBuilder streamsWithNoShardsFound = new StringBuilder();
+ for (Map.Entry<String, String> streamToLastDiscoveredShardEntry :
+ subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
+ if (streamToLastDiscoveredShardEntry.getValue() != null) {
+ hasShards = true;
+ } else {
+ streamsWithNoShardsFound
+ .append(streamToLastDiscoveredShardEntry.getKey())
+ .append(", ");
+ }
+ }
+
+ if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
+ LOG.warn(
+ "Subtask {} has failed to find any shards for the following subscribed streams: {}",
+ indexOfThisConsumerSubtask,
+ streamsWithNoShardsFound.toString());
+ }
+
+ if (!hasShards) {
+ throw new RuntimeException(
+ "No shards can be found for all subscribed streams: " + streams);
+ }
+
+ // 2. start consuming any shard state we already have in the subscribedShardState up to
+ // this point; the
+ // subscribedShardState may already be seeded with values due to step 1., or explicitly
+ // added by the
+ // consumer using a restored state checkpoint
+ for (int seededStateIndex = 0;
+ seededStateIndex < subscribedShardsState.size();
+ seededStateIndex++) {
+ KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
+
+ // only start a consuming thread if the seeded subscribed shard has not been completely
+ // read already
+ if (!seededShardState
+ .getLastProcessedSequenceNum()
+ .equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info(
+ "Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}",
+ indexOfThisConsumerSubtask,
+ seededShardState.getStreamShardHandle().toString(),
+ seededShardState.getLastProcessedSequenceNum(),
+ seededStateIndex);
+ }
+
+ StreamShardHandle streamShardHandle =
+ subscribedShardsState.get(seededStateIndex).getStreamShardHandle();
+ KinesisDeserializationSchema<T> shardDeserializationSchema =
+ getClonedDeserializationSchema();
+ shardDeserializationSchema.open(
+ RuntimeContextInitializationContextAdapters.deserializationAdapter(
+ runtimeContext,
+ // ignore the provided metric group
+ metricGroup ->
+ consumerMetricGroup
+ .addGroup(
+ "subtaskId",
+ String.valueOf(indexOfThisConsumerSubtask))
+ .addGroup(
+ "shardId",
+ streamShardHandle.getShard().getShardId())
+ .addGroup("user")));
+ shardConsumersExecutor.submit(
+ createShardConsumer(
+ seededStateIndex,
+ streamShardHandle,
+ subscribedShardsState
+ .get(seededStateIndex)
+ .getLastProcessedSequenceNum(),
+ registerShardMetricGroup(
+ consumerMetricGroup,
+ subscribedShardsState.get(seededStateIndex)),
+ shardDeserializationSchema));
+ }
+ }
+
+ // start periodic watermark emitter, if a watermark assigner was configured
+ if (periodicWatermarkAssigner != null) {
+ long periodicWatermarkIntervalMillis =
+ runtimeContext.getExecutionConfig().getAutoWatermarkInterval();
+ if (periodicWatermarkIntervalMillis > 0) {
+ ProcessingTimeService timerService =
+ ((StreamingRuntimeContext) runtimeContext).getProcessingTimeService();
+ LOG.info(
+ "Starting periodic watermark emitter with interval {}",
+ periodicWatermarkIntervalMillis);
+ new PeriodicWatermarkEmitter(timerService, periodicWatermarkIntervalMillis).start();
+ if (watermarkTracker != null) {
+ // setup global watermark tracking
+ long watermarkSyncMillis =
+ Long.parseLong(
+ getConsumerConfiguration()
+ .getProperty(
+ ConsumerConfigConstants.WATERMARK_SYNC_MILLIS,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_WATERMARK_SYNC_MILLIS)));
+ watermarkTracker.setUpdateTimeoutMillis(
+ watermarkSyncMillis * 3); // synchronization latency
+ watermarkTracker.open(runtimeContext);
+ new WatermarkSyncCallback(timerService, watermarkSyncMillis).start();
+ // emit records ahead of watermark to offset synchronization latency
+ long lookaheadMillis =
+ Long.parseLong(
+ getConsumerConfiguration()
+ .getProperty(
+ ConsumerConfigConstants
+ .WATERMARK_LOOKAHEAD_MILLIS,
+ Long.toString(0)));
+ recordEmitter.setMaxLookaheadMillis(
+ Math.max(lookaheadMillis, watermarkSyncMillis * 3));
+
+ // record emitter depends on periodic watermark
+ // it runs in a separate thread since main thread is used for discovery
+ Runnable recordEmitterRunnable =
+ new Runnable() {
+ @Override
+ public void run() {
+ try {
+ recordEmitter.run();
+ } catch (Throwable error) {
+ // report the error that terminated the emitter loop to
+ // source thread
+ stopWithError(error);
+ }
+ }
+ };
+
+ Thread thread = new Thread(recordEmitterRunnable);
+ thread.setName("recordEmitter-" + runtimeContext.getTaskNameWithSubtasks());
+ thread.setDaemon(true);
+ thread.start();
+ }
+ }
+ this.shardIdleIntervalMillis =
+ Long.parseLong(
+ getConsumerConfiguration()
+ .getProperty(
+ ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_IDLE_INTERVAL_MILLIS)));
+ }
+
+ // ------------------------------------------------------------------------
+
+ // finally, start the infinite shard discovery and consumer launching loop;
+ // we will escape from this loop only when shutdownFetcher() or stopWithError() is called
+ // TODO: have this thread emit the records for tracking backpressure
+
+ final long discoveryIntervalMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+
+ if (this.numberOfActiveShards.get() == 0) {
+ LOG.info(
+ "Subtask {} has no active shards to read on startup; marking the subtask as temporarily idle ...",
+ indexOfThisConsumerSubtask);
+ sourceContext.markAsTemporarilyIdle();
+ }
+
+ while (running) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Subtask {} is trying to discover new shards that were created due to resharding ...",
+ indexOfThisConsumerSubtask);
+ }
+ List<StreamShardHandle> newShardsDueToResharding = discoverNewShardsToSubscribe();
+
+ for (StreamShardHandle shard : newShardsDueToResharding) {
+ // since there may be delay in discovering a new shard, all new shards due to
+ // resharding should be read starting from the earliest record possible
+ KinesisStreamShardState newShardState =
+ new KinesisStreamShardState(
+ convertToStreamShardMetadata(shard),
+ shard,
+ SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
+ int newStateIndex = registerNewSubscribedShardState(newShardState);
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info(
+ "Subtask {} has discovered a new shard {} due to resharding, and will start consuming "
+ + "the shard from sequence number {} with ShardConsumer {}",
+ indexOfThisConsumerSubtask,
+ newShardState.getStreamShardHandle().toString(),
+ newShardState.getLastProcessedSequenceNum(),
+ newStateIndex);
+ }
+
+ StreamShardHandle streamShardHandle = newShardState.getStreamShardHandle();
+ KinesisDeserializationSchema<T> shardDeserializationSchema =
+ getClonedDeserializationSchema();
+ shardDeserializationSchema.open(
+ RuntimeContextInitializationContextAdapters.deserializationAdapter(
+ runtimeContext,
+ // ignore the provided metric group
+ metricGroup ->
+ consumerMetricGroup
+ .addGroup(
+ "subtaskId",
+ String.valueOf(indexOfThisConsumerSubtask))
+ .addGroup(
+ "shardId",
+ streamShardHandle.getShard().getShardId())
+ .addGroup("user")));
+ shardConsumersExecutor.submit(
+ createShardConsumer(
+ newStateIndex,
+ newShardState.getStreamShardHandle(),
+ newShardState.getLastProcessedSequenceNum(),
+ registerShardMetricGroup(consumerMetricGroup, newShardState),
+ shardDeserializationSchema));
+ }
+
+ // we also check if we are running here so that we won't start the discovery sleep
+ // interval if the running flag was set to false during the middle of the while loop
+ if (running && discoveryIntervalMillis != 0) {
+ try {
+ cancelFuture.get(discoveryIntervalMillis, TimeUnit.MILLISECONDS);
+ LOG.debug("Cancelled discovery");
+ } catch (TimeoutException iex) {
+ // timeout is expected when fetcher is not cancelled
+ }
+ }
+ }
+
+ // make sure all resources have been terminated before leaving
+ try {
+ awaitTermination();
+ } catch (InterruptedException ie) {
+ // If there is an original exception, preserve it, since that's more important/useful.
+ this.error.compareAndSet(null, ie);
+ }
+
+ // any error thrown in the shard consumer threads will be thrown to the main thread
+ Throwable throwable = this.error.get();
+ if (throwable != null) {
+ if (throwable instanceof Exception) {
+ throw (Exception) throwable;
+ } else if (throwable instanceof Error) {
+ throw (Error) throwable;
+ } else {
+ throw new Exception(throwable);
+ }
+ }
+ }
+
+ /**
+ * Creates a snapshot of the current last processed sequence numbers of each subscribed shard.
+ *
+ * @return state snapshot
+ */
+ public HashMap<StreamShardMetadata, SequenceNumber> snapshotState() {
+ // this method assumes that the checkpoint lock is held
+ assert Thread.holdsLock(checkpointLock);
+
+ HashMap<StreamShardMetadata, SequenceNumber> stateSnapshot = new HashMap<>();
+ for (KinesisStreamShardState shardWithState : subscribedShardsState) {
+ stateSnapshot.put(
+ shardWithState.getStreamShardMetadata(),
+ shardWithState.getLastProcessedSequenceNum());
+ }
+ return stateSnapshot;
+ }
+
+ /**
+ * Starts shutting down the fetcher. Must be called to allow {@link
+ * KinesisDataFetcher#runFetcher()} to complete. Once called, the shutdown procedure will be
+ * executed and all shard consuming threads will be interrupted.
+ */
+ public void shutdownFetcher() {
+ LOG.info(
+ "Starting shutdown of shard consumer threads and AWS SDK resources of subtask {} ...",
+ indexOfThisConsumerSubtask,
+ error.get());
+
+ running = false;
+ try {
+ try {
+ deregisterStreamConsumer();
+ } catch (Exception e) {
+ LOG.warn("Encountered exception deregistering stream consumers", e);
+ }
+
+ try {
+ closeRecordPublisherFactory();
+ } catch (Exception e) {
+ LOG.warn("Encountered exception closing record publisher factory", e);
+ }
+ } finally {
+ gracefulShutdownShardConsumers();
+
+ cancelFuture.complete(null);
+
+ if (watermarkTracker != null) {
+ watermarkTracker.close();
+ }
+ this.recordEmitter.stop();
+ }
+
+ LOG.info(
+ "Shutting down the shard consumer threads of subtask {} ...",
+ indexOfThisConsumerSubtask);
+ }
+
+ /**
+ * Closes recordRecordPublisherFactory. Allows test to override this to simulate exception for
+ * shutdown logic.
+ */
+ @VisibleForTesting
+ protected void closeRecordPublisherFactory() {
+ recordPublisherFactory.close();
+ }
+
+ /**
+ * Deregisters stream consumers. Allows test to override this to simulate exception for shutdown
+ * logic.
+ */
+ @VisibleForTesting
+ protected void deregisterStreamConsumer() {
+ StreamConsumerRegistrarUtil.deregisterStreamConsumers(configProps, streams);
+ }
+
+ /** Gracefully stops shardConsumersExecutor without interrupting running threads. */
+ private void gracefulShutdownShardConsumers() {
+ shardConsumersExecutor.shutdown();
+ }
+
+ /**
+ * Returns a flag indicating if this fetcher is running.
+ *
+ * @return true if the fetch is running, false if it has been shutdown
+ */
+ boolean isRunning() {
+ return running;
+ }
+
+ /**
+ * After calling {@link KinesisDataFetcher#shutdownFetcher()}, this can be called to await the
+ * fetcher shutdown.
+ */
+ @SuppressWarnings("StatementWithEmptyBody")
+ public void awaitTermination() throws InterruptedException {
+ while (!shardConsumersExecutor.awaitTermination(1, TimeUnit.MINUTES)) {
+ // Keep waiting.
+ }
+ }
+
+ /**
+ * Called by created threads to pass on errors. Only the first thrown error is set. Once set,
+ * the shutdown process will be executed and all shard consuming threads will be interrupted.
+ */
+ protected void stopWithError(Throwable throwable) {
+ if (this.error.compareAndSet(null, throwable)) {
+ shutdownFetcher();
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Functions that update the subscribedStreamToLastDiscoveredShardIds state
+ // ------------------------------------------------------------------------
+
+ /**
+ * Updates the last discovered shard of a subscribed stream; only updates if the update is
+ * valid.
+ */
+ public void advanceLastDiscoveredShardOfStream(String stream, String shardId) {
+ String lastSeenShardIdOfStream = this.subscribedStreamsToLastDiscoveredShardIds.get(stream);
+
+ // the update is valid only if the given shard id is greater
+ // than the previous last seen shard id of the stream
+ if (lastSeenShardIdOfStream == null) {
+ // if not previously set, simply put as the last seen shard id
+ this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
+ } else if (shouldAdvanceLastDiscoveredShardId(shardId, lastSeenShardIdOfStream)) {
+ this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId);
+ }
+ }
+
+ /** Given lastSeenShardId, check if last discovered shardId should be advanced. */
+ protected boolean shouldAdvanceLastDiscoveredShardId(
+ String shardId, String lastSeenShardIdOfStream) {
+ return (StreamShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) > 0);
+ }
+
+ /**
+ * A utility function that does the following:
+ *
+ * <p>1. Find new shards for each stream that we haven't seen before 2. For each new shard,
+ * determine whether this consumer subtask should subscribe to them; if yes, it is added to the
+ * returned list of shards 3. Update the subscribedStreamsToLastDiscoveredShardIds state so that
+ * we won't get shards that we have already seen before the next time this function is called
+ */
+ public List<StreamShardHandle> discoverNewShardsToSubscribe() throws InterruptedException {
+
+ List<StreamShardHandle> newShardsToSubscribe = new LinkedList<>();
+
+ GetShardListResult shardListResult =
+ kinesis.getShardList(subscribedStreamsToLastDiscoveredShardIds);
+ if (shardListResult.hasRetrievedShards()) {
+ Set<String> streamsWithNewShards = shardListResult.getStreamsWithRetrievedShards();
+
+ for (String stream : streamsWithNewShards) {
+ List<StreamShardHandle> newShardsOfStream =
+ shardListResult.getRetrievedShardListOfStream(stream);
+ for (StreamShardHandle newShard : newShardsOfStream) {
+ int hashCode = shardAssigner.assign(newShard, totalNumberOfConsumerSubtasks);
+ if (isThisSubtaskShouldSubscribeTo(
+ hashCode, totalNumberOfConsumerSubtasks, indexOfThisConsumerSubtask)) {
+ newShardsToSubscribe.add(newShard);
+ }
+ }
+
+ advanceLastDiscoveredShardOfStream(
+ stream,
+ shardListResult.getLastSeenShardOfStream(stream).getShard().getShardId());
+ }
+ }
+
+ return newShardsToSubscribe;
+ }
+
+ // ------------------------------------------------------------------------
+ // Functions to get / set information about the consumer
+ // ------------------------------------------------------------------------
+
+ protected Properties getConsumerConfiguration() {
+ return configProps;
+ }
+
+ private KinesisDeserializationSchema<T> getClonedDeserializationSchema() {
+ try {
+ return InstantiationUtil.clone(
+ deserializationSchema, runtimeContext.getUserCodeClassLoader());
+ } catch (IOException | ClassNotFoundException ex) {
+ // this really shouldn't happen; simply wrap it around a runtime exception
+ throw new RuntimeException(ex);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Thread-safe operations for record emitting and shard state updating
+ // that assure atomicity with respect to the checkpoint lock
+ // ------------------------------------------------------------------------
+
+ /**
+ * Prepare a record and hand it over to the {@link RecordEmitter}, which may collect it
+ * asynchronously. This method is called by {@link ShardConsumer}s.
+ *
+ * @param record the record to collect
+ * @param recordTimestamp timestamp to attach to the collected record
+ * @param shardStateIndex index of the shard to update in subscribedShardsState; this index
+ * should be the returned value from {@link
+ * KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called when
+ * the shard state was registered.
+ * @param lastSequenceNumber the last sequence number value to update
+ */
+ protected void emitRecordAndUpdateState(
+ T record,
+ long recordTimestamp,
+ int shardStateIndex,
+ SequenceNumber lastSequenceNumber) {
+ ShardWatermarkState sws = shardWatermarks.get(shardStateIndex);
+ Preconditions.checkNotNull(
+ sws, "shard watermark state initialized in registerNewSubscribedShardState");
+ Watermark watermark = null;
+ if (sws.periodicWatermarkAssigner != null) {
+ recordTimestamp =
+ sws.periodicWatermarkAssigner.extractTimestamp(record, sws.lastRecordTimestamp);
+ // track watermark per record since extractTimestamp has side effect
+ watermark = sws.periodicWatermarkAssigner.getCurrentWatermark();
+ }
+ sws.lastRecordTimestamp = recordTimestamp;
+ sws.lastUpdated = getCurrentTimeMillis();
+
+ RecordWrapper<T> recordWrapper = new RecordWrapper<>(record, recordTimestamp);
+ recordWrapper.shardStateIndex = shardStateIndex;
+ recordWrapper.lastSequenceNumber = lastSequenceNumber;
+ recordWrapper.watermark = watermark;
+ try {
+ sws.emitQueue.put(recordWrapper);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Atomic operation to collect a record and update state to the sequence number of the record.
+ * This method is called from the record emitter.
+ *
+ * <p>Responsible for tracking per shard watermarks and emit timestamps extracted from the
+ * record, when a watermark assigner was configured.
+ */
+ private void emitRecordAndUpdateState(RecordWrapper<T> rw) {
+ synchronized (checkpointLock) {
+ if (rw.getValue() != null) {
+ sourceContext.collectWithTimestamp(rw.getValue(), rw.timestamp);
+ ShardWatermarkState<T> sws = shardWatermarks.get(rw.shardStateIndex);
+ sws.lastEmittedRecordWatermark = rw.watermark;
+ } else {
+ LOG.warn(
+ "Skipping non-deserializable record at sequence number {} of shard {}.",
+ rw.lastSequenceNumber,
+ subscribedShardsState.get(rw.shardStateIndex).getStreamShardHandle());
+ }
+ updateState(rw.shardStateIndex, rw.lastSequenceNumber);
+ }
+ }
+
+ /**
+ * Update the shard to last processed sequence number state. This method is called by {@link
+ * ShardConsumer}s.
+ *
+ * @param shardStateIndex index of the shard to update in subscribedShardsState; this index
+ * should be the returned value from {@link
+ * KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called when
+ * the shard state was registered.
+ * @param lastSequenceNumber the last sequence number value to update
+ */
+ protected final void updateState(int shardStateIndex, SequenceNumber lastSequenceNumber) {
+ synchronized (checkpointLock) {
+ subscribedShardsState
+ .get(shardStateIndex)
+ .setLastProcessedSequenceNum(lastSequenceNumber);
+
+ // if a shard's state is updated to be SENTINEL_SHARD_ENDING_SEQUENCE_NUM by its
+ // consumer thread,
+ // we've finished reading the shard and should determine it to be non-active
+ if (lastSequenceNumber.equals(
+ SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+ LOG.info(
+ "Subtask {} has reached the end of subscribed shard: {}",
+ indexOfThisConsumerSubtask,
+ subscribedShardsState.get(shardStateIndex).getStreamShardHandle());
+
+ // check if we need to mark the source as idle;
+ // note that on resharding, if registerNewSubscribedShardState was invoked for newly
+ // discovered shards
+ // AFTER the old shards had reached the end, the subtask's status will be
+ // automatically toggled back to
+ // be active immediately afterwards as soon as we collect records from the new
+ // shards
+ if (this.numberOfActiveShards.decrementAndGet() == 0) {
+ LOG.info(
+ "Subtask {} has reached the end of all currently subscribed shards; marking the subtask as temporarily idle ...",
+ indexOfThisConsumerSubtask);
+
+ sourceContext.markAsTemporarilyIdle();
+ }
+ }
+ }
+ }
+
+ /**
+ * Register a new subscribed shard state.
+ *
+ * @param newSubscribedShardState the new shard state that this fetcher is to be subscribed to
+ */
+ public int registerNewSubscribedShardState(KinesisStreamShardState newSubscribedShardState) {
+ synchronized (checkpointLock) {
+ subscribedShardsState.add(newSubscribedShardState);
+
+ // If a registered shard has initial state that is not
+ // SENTINEL_SHARD_ENDING_SEQUENCE_NUM (will be the case
+ // if the consumer had already finished reading a shard before we failed and restored),
+ // we determine that
+ // this subtask has a new active shard
+ if (!newSubscribedShardState
+ .getLastProcessedSequenceNum()
+ .equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
+ this.numberOfActiveShards.incrementAndGet();
+ }
+
+ int shardStateIndex = subscribedShardsState.size() - 1;
+
+ // track all discovered shards for watermark determination
+ ShardWatermarkState sws = shardWatermarks.get(shardStateIndex);
+ if (sws == null) {
+ sws = new ShardWatermarkState();
+ try {
+ sws.periodicWatermarkAssigner =
+ InstantiationUtil.clone(periodicWatermarkAssigner);
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to instantiate new WatermarkAssigner", e);
+ }
+ sws.emitQueue = recordEmitter.getQueue(shardStateIndex);
+ sws.lastUpdated = getCurrentTimeMillis();
+ sws.lastRecordTimestamp = Long.MIN_VALUE;
+ shardWatermarks.put(shardStateIndex, sws);
+ }
+
+ return shardStateIndex;
+ }
+ }
+
+ /**
+ * Return the current system time. Allow tests to override this to simulate progress for
+ * watermark logic.
+ *
+ * @return current processing time
+ */
+ @VisibleForTesting
+ protected long getCurrentTimeMillis() {
+ return System.currentTimeMillis();
+ }
+
+ /**
+ * Called periodically to emit a watermark. Checks all shards for the current event time
+ * watermark, and possibly emits the next watermark.
+ *
+ * <p>Shards that have not received an update for a certain interval are considered inactive so
+ * as to not hold back the watermark indefinitely. When all shards are inactive, the subtask
+ * will be marked as temporarily idle to not block downstream operators.
+ */
+ @VisibleForTesting
+ protected void emitWatermark() {
+ LOG.debug(
+ "Evaluating watermark for subtask {} time {}",
+ indexOfThisConsumerSubtask,
+ getCurrentTimeMillis());
+ long potentialWatermark = Long.MAX_VALUE;
+ long potentialNextWatermark = Long.MAX_VALUE;
+ long idleTime =
+ (shardIdleIntervalMillis > 0)
+ ? getCurrentTimeMillis() - shardIdleIntervalMillis
+ : Long.MAX_VALUE;
+
+ for (Map.Entry<Integer, ShardWatermarkState> e : shardWatermarks.entrySet()) {
+ Watermark w = e.getValue().lastEmittedRecordWatermark;
+ // consider only active shards, or those that would advance the watermark
+ if (w != null
+ && (e.getValue().lastUpdated >= idleTime
+ || e.getValue().emitQueue.getSize() > 0
+ || w.getTimestamp() > lastWatermark)) {
+ potentialWatermark = Math.min(potentialWatermark, w.getTimestamp());
+ // for sync, use the watermark of the next record, when available
+ // otherwise watermark may stall when record is blocked by synchronization
+ RecordEmitter.RecordQueue<RecordWrapper<T>> q = e.getValue().emitQueue;
+ RecordWrapper<T> nextRecord = q.peek();
+ Watermark nextWatermark = (nextRecord != null) ? nextRecord.watermark : w;
+ potentialNextWatermark =
+ Math.min(potentialNextWatermark, nextWatermark.getTimestamp());
+ }
+ }
+
+ // advance watermark if possible (watermarks can only be ascending)
+ if (potentialWatermark == Long.MAX_VALUE) {
+ if (shardWatermarks.isEmpty() || shardIdleIntervalMillis > 0) {
+ LOG.info(
+ "No active shard for subtask {}, marking the source idle.",
+ indexOfThisConsumerSubtask);
+ // no active shard, signal downstream operators to not wait for a watermark
+ sourceContext.markAsTemporarilyIdle();
+ isIdle = true;
+ }
+ } else {
+ if (potentialWatermark > lastWatermark) {
+ LOG.debug(
+ "Emitting watermark {} from subtask {}",
+ potentialWatermark,
+ indexOfThisConsumerSubtask);
+ sourceContext.emitWatermark(new Watermark(potentialWatermark));
+ lastWatermark = potentialWatermark;
+ isIdle = false;
+ }
+ nextWatermark = potentialNextWatermark;
+ }
+ }
+
+ /** Per shard tracking of watermark and last activity. */
+ private static class ShardWatermarkState<T> {
+ private AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner;
+ private RecordEmitter.RecordQueue<RecordWrapper<T>> emitQueue;
+ private volatile long lastRecordTimestamp;
+ private volatile long lastUpdated;
+ private volatile Watermark lastEmittedRecordWatermark;
+ }
+
+ /**
+ * The periodic watermark emitter. In its given interval, it checks all shards for the current
+ * event time watermark, and possibly emits the next watermark.
+ */
+ private class PeriodicWatermarkEmitter implements ProcessingTimeCallback {
+
+ private final ProcessingTimeService timerService;
+ private final long interval;
+
+ PeriodicWatermarkEmitter(ProcessingTimeService timerService, long autoWatermarkInterval) {
+ this.timerService = checkNotNull(timerService);
+ this.interval = autoWatermarkInterval;
+ }
+
+ public void start() {
+ LOG.debug("registering periodic watermark timer with interval {}", interval);
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+
+ @Override
+ public void onProcessingTime(long timestamp) {
+ emitWatermark();
+ // schedule the next watermark
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+ }
+
+ /** Timer task to update shared watermark state. */
+ private class WatermarkSyncCallback implements ProcessingTimeCallback {
+
+ private static final long LOG_INTERVAL_MILLIS = 60_000;
+
+ private final ProcessingTimeService timerService;
+ private final long interval;
+ private long lastGlobalWatermark = Long.MIN_VALUE;
+ private long propagatedLocalWatermark = Long.MIN_VALUE;
+ private int stalledWatermarkIntervalCount = 0;
+ private long lastLogged;
+
+ WatermarkSyncCallback(ProcessingTimeService timerService, long interval) {
+ this.timerService = checkNotNull(timerService);
+ this.interval = interval;
+ MetricGroup shardMetricsGroup =
+ consumerMetricGroup.addGroup(
+ "subtaskId", String.valueOf(indexOfThisConsumerSubtask));
+ shardMetricsGroup.gauge("localWatermark", () -> nextWatermark);
+ shardMetricsGroup.gauge("globalWatermark", () -> lastGlobalWatermark);
+ }
+
+ public void start() {
+ LOG.info("Registering watermark tracker with interval {}", interval);
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+
+ @Override
+ public void onProcessingTime(long timestamp) {
+ if (nextWatermark != Long.MIN_VALUE) {
+ long globalWatermark = lastGlobalWatermark;
+ // TODO: refresh watermark while idle
+ if (!(isIdle && nextWatermark == propagatedLocalWatermark)) {
+ globalWatermark = watermarkTracker.updateWatermark(nextWatermark);
+ propagatedLocalWatermark = nextWatermark;
+ } else {
+ LOG.info(
+ "WatermarkSyncCallback subtask: {} is idle",
+ indexOfThisConsumerSubtask);
+ }
+
+ if (timestamp - lastLogged > LOG_INTERVAL_MILLIS) {
+ lastLogged = System.currentTimeMillis();
+ LOG.info(
+ "WatermarkSyncCallback subtask: {} local watermark: {}"
+ + ", global watermark: {}, delta: {} timeouts: {}, emitter: {}",
+ indexOfThisConsumerSubtask,
+ nextWatermark,
+ globalWatermark,
+ nextWatermark - globalWatermark,
+ watermarkTracker.getUpdateTimeoutCount(),
+ recordEmitter.printInfo());
+
+ // Following is for debugging non-reproducible issue with stalled watermark
+ if (globalWatermark == nextWatermark
+ && globalWatermark == lastGlobalWatermark
+ && stalledWatermarkIntervalCount++ > 5) {
+ // subtask blocks watermark, log to aid troubleshooting
+ stalledWatermarkIntervalCount = 0;
+ for (Map.Entry<Integer, ShardWatermarkState> e :
+ shardWatermarks.entrySet()) {
+ RecordEmitter.RecordQueue<RecordWrapper<T>> q = e.getValue().emitQueue;
+ RecordWrapper<T> nextRecord = q.peek();
+ if (nextRecord != null) {
+ LOG.info(
+ "stalled watermark {} key {} next watermark {} next timestamp {}",
+ nextWatermark,
+ e.getKey(),
+ nextRecord.watermark,
+ nextRecord.timestamp);
+ }
+ }
+ }
+ }
+
+ lastGlobalWatermark = globalWatermark;
+ recordEmitter.setCurrentWatermark(globalWatermark);
+ }
+ // schedule next callback
+ timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this);
+ }
+ }
+
+ /**
+ * Registers a metric group associated with the shard id of the provided {@link
+ * KinesisStreamShardState shardState}.
+ *
+ * @return a {@link MetricGroup} that can be used to update metric values
+ */
+ private MetricGroup registerShardMetricGroup(
+ final MetricGroup metricGroup, final KinesisStreamShardState shardState) {
+ return metricGroup
+ .addGroup(
+ KinesisConsumerMetricConstants.STREAM_METRICS_GROUP,
+ shardState.getStreamShardHandle().getStreamName())
+ .addGroup(
+ KinesisConsumerMetricConstants.SHARD_METRICS_GROUP,
+ shardState.getStreamShardHandle().getShard().getShardId());
+ }
+
+ // ------------------------------------------------------------------------
+ // Miscellaneous utility functions
+ // ------------------------------------------------------------------------
+
+ /**
+ * Utility function to determine whether a shard should be subscribed by this consumer subtask.
+ *
+ * @param shardHash hash code for the shard
+ * @param totalNumberOfConsumerSubtasks total number of consumer subtasks
+ * @param indexOfThisConsumerSubtask index of this consumer subtask
+ */
+ public static boolean isThisSubtaskShouldSubscribeTo(
+ int shardHash, int totalNumberOfConsumerSubtasks, int indexOfThisConsumerSubtask) {
+ return (Math.abs(shardHash % totalNumberOfConsumerSubtasks)) == indexOfThisConsumerSubtask;
+ }
+
+ @VisibleForTesting
+ protected ExecutorService createShardConsumersThreadPool(final String subtaskName) {
+ return Executors.newCachedThreadPool(
+ new ThreadFactory() {
+ private final AtomicLong threadCount = new AtomicLong(0);
+
+ @Override
+ public Thread newThread(Runnable runnable) {
+ Thread thread = new Thread(runnable);
+ thread.setName(
+ "shardConsumers-"
+ + subtaskName
+ + "-thread-"
+ + threadCount.getAndIncrement());
+ thread.setDaemon(true);
+ return thread;
+ }
+ });
+ }
+
+ @VisibleForTesting
+ public List<KinesisStreamShardState> getSubscribedShardsState() {
+ return subscribedShardsState;
+ }
+
+ /**
+ * Utility function to create an initial map of the last discovered shard id of each subscribed
+ * stream, set to null; This is called in the constructor; correct values will be set later on
+ * by calling advanceLastDiscoveredShardOfStream().
+ *
+ * @param streams the list of subscribed streams
+ * @return the initial map for subscribedStreamsToLastDiscoveredShardIds
+ */
+ protected static HashMap<String, String>
+ createInitialSubscribedStreamsToLastDiscoveredShardsState(List<String> streams) {
+ HashMap<String, String> initial = new HashMap<>();
+ for (String stream : streams) {
+ initial.put(stream, null);
+ }
+ return initial;
+ }
+
+ /**
+ * Utility function to convert {@link StreamShardHandle} into {@link StreamShardMetadata}.
+ *
+ * @param streamShardHandle the {@link StreamShardHandle} to be converted
+ * @return a {@link StreamShardMetadata} object
+ */
+ public static StreamShardMetadata convertToStreamShardMetadata(
+ StreamShardHandle streamShardHandle) {
+ StreamShardMetadata streamShardMetadata = new StreamShardMetadata();
+
+ streamShardMetadata.setStreamName(streamShardHandle.getStreamName());
+ streamShardMetadata.setShardId(streamShardHandle.getShard().getShardId());
+ streamShardMetadata.setParentShardId(streamShardHandle.getShard().getParentShardId());
+ streamShardMetadata.setAdjacentParentShardId(
+ streamShardHandle.getShard().getAdjacentParentShardId());
+
+ if (streamShardHandle.getShard().getHashKeyRange() != null) {
+ streamShardMetadata.setStartingHashKey(
+ streamShardHandle.getShard().getHashKeyRange().getStartingHashKey());
+ streamShardMetadata.setEndingHashKey(
+ streamShardHandle.getShard().getHashKeyRange().getEndingHashKey());
+ }
+
+ if (streamShardHandle.getShard().getSequenceNumberRange() != null) {
+ streamShardMetadata.setStartingSequenceNumber(
+ streamShardHandle
+ .getShard()
+ .getSequenceNumberRange()
+ .getStartingSequenceNumber());
+ streamShardMetadata.setEndingSequenceNumber(
+ streamShardHandle
+ .getShard()
+ .getSequenceNumberRange()
+ .getEndingSequenceNumber());
+ }
+
+ return streamShardMetadata;
+ }
+
+ /**
+ * Utility function to convert {@link StreamShardMetadata} into {@link StreamShardHandle}.
+ *
+ * @param streamShardMetadata the {@link StreamShardMetadata} to be converted
+ * @return a {@link StreamShardHandle} object
+ */
+ public static StreamShardHandle convertToStreamShardHandle(
+ StreamShardMetadata streamShardMetadata) {
+ Shard shard = new Shard();
+ shard.withShardId(streamShardMetadata.getShardId());
+ shard.withParentShardId(streamShardMetadata.getParentShardId());
+ shard.withAdjacentParentShardId(streamShardMetadata.getAdjacentParentShardId());
+
+ HashKeyRange hashKeyRange = new HashKeyRange();
+ hashKeyRange.withStartingHashKey(streamShardMetadata.getStartingHashKey());
+ hashKeyRange.withEndingHashKey(streamShardMetadata.getEndingHashKey());
+ shard.withHashKeyRange(hashKeyRange);
+
+ SequenceNumberRange sequenceNumberRange = new SequenceNumberRange();
+ sequenceNumberRange.withStartingSequenceNumber(
+ streamShardMetadata.getStartingSequenceNumber());
+ sequenceNumberRange.withEndingSequenceNumber(streamShardMetadata.getEndingSequenceNumber());
+ shard.withSequenceNumberRange(sequenceNumberRange);
+
+ return new StreamShardHandle(streamShardMetadata.getStreamName(), shard);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
new file mode 100644
index 0000000..85dc8f4
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult;
+import org.apache.flink.streaming.connectors.kinesis.metrics.ShardConsumerMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static java.util.Optional.ofNullable;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.CANCELLED;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Thread that subscribes to the given {@link RecordPublisher}. Each thread is in charge of one
+ * Kinesis shard only.
+ *
+ * <p>A {@link ShardConsumer} is responsible for:
+ *
+ * <ul>
+ * <li>Running the {@link RecordPublisher} to consume all records from the subscribed shard
+ * <li>Deserializing and deaggregating incoming records from Kinesis
+ * <li>Logging metrics
+ * <li>Passing the records up to the {@link KinesisDataFetcher}
+ * </ul>
+ */
+@Internal
+public class ShardConsumer<T> implements Runnable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
+
+ private final KinesisDeserializationSchema<T> deserializer;
+
+ private final int subscribedShardStateIndex;
+
+ private final KinesisDataFetcher<T> fetcherRef;
+
+ private final StreamShardHandle subscribedShard;
+
+ private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
+
+ private SequenceNumber lastSequenceNum;
+
+ private final RecordPublisher recordPublisher;
+
+ /**
+ * Creates a shard consumer.
+ *
+ * @param fetcherRef reference to the owning fetcher
+ * @param recordPublisher the record publisher used to read records from kinesis
+ * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
+ * @param subscribedShard the shard this consumer is subscribed to
+ * @param lastSequenceNum the sequence number in the shard to start consuming
+ * @param shardConsumerMetricsReporter the reporter to report metrics to
+ * @param shardDeserializer used to deserialize incoming records
+ */
+ public ShardConsumer(
+ KinesisDataFetcher<T> fetcherRef,
+ RecordPublisher recordPublisher,
+ Integer subscribedShardStateIndex,
+ StreamShardHandle subscribedShard,
+ SequenceNumber lastSequenceNum,
+ ShardConsumerMetricsReporter shardConsumerMetricsReporter,
+ KinesisDeserializationSchema<T> shardDeserializer) {
+ this.fetcherRef = checkNotNull(fetcherRef);
+ this.recordPublisher = checkNotNull(recordPublisher);
+ this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
+ this.subscribedShard = checkNotNull(subscribedShard);
+ this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
+ this.lastSequenceNum = checkNotNull(lastSequenceNum);
+
+ checkArgument(
+ !lastSequenceNum.equals(
+ SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
+ "Should not start a ShardConsumer if the shard has already been completely read.");
+
+ this.deserializer = shardDeserializer;
+ }
+
+ @Override
+ public void run() {
+ try {
+ while (isRunning()) {
+ final RecordPublisherRunResult result =
+ recordPublisher.run(
+ batch -> {
+ if (!batch.getDeaggregatedRecords().isEmpty()) {
+ LOG.debug(
+ "stream: {}, shard: {}, millis behind latest: {}, batch size: {}",
+ subscribedShard.getStreamName(),
+ subscribedShard.getShard().getShardId(),
+ batch.getMillisBehindLatest(),
+ batch.getDeaggregatedRecordSize());
+ }
+ for (UserRecord userRecord : batch.getDeaggregatedRecords()) {
+ if (filterDeaggregatedRecord(userRecord)) {
+ deserializeRecordForCollectionAndUpdateState(
+ userRecord);
+ }
+ }
+
+ shardConsumerMetricsReporter.setAverageRecordSizeBytes(
+ batch.getAverageRecordSizeBytes());
+ shardConsumerMetricsReporter.setNumberOfAggregatedRecords(
+ batch.getAggregatedRecordSize());
+ shardConsumerMetricsReporter.setNumberOfDeaggregatedRecords(
+ batch.getDeaggregatedRecordSize());
+ ofNullable(batch.getMillisBehindLatest())
+ .ifPresent(
+ shardConsumerMetricsReporter
+ ::setMillisBehindLatest);
+
+ return lastSequenceNum;
+ });
+
+ if (result == COMPLETE) {
+ fetcherRef.updateState(
+ subscribedShardStateIndex,
+ SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get());
+ // we can close this consumer thread once we've reached the end of the
+ // subscribed shard
+ break;
+ } else if (result == CANCELLED) {
+ final String errorMessage =
+ "Shard consumer cancelled: " + subscribedShard.getShard().getShardId();
+ LOG.info(errorMessage);
+ throw new ShardConsumerCancelledException(errorMessage);
+ }
+ }
+ } catch (Throwable t) {
+ fetcherRef.stopWithError(t);
+ } finally {
+ this.shardConsumerMetricsReporter.unregister();
+ }
+ }
+
+ /**
+ * The loop in run() checks this before fetching next batch of records. Since this runnable will
+ * be executed by the ExecutorService {@code KinesisDataFetcher#shardConsumersExecutor}, this
+ * thread would be closed down by calling shutdownNow() on {@code
+ * KinesisDataFetcher#shardConsumersExecutor} and let the executor service interrupt all
+ * currently running {@link ShardConsumer}s. The AWS SDK resources must be shutdown prior to
+ * this thread in order to preserve classpath for teardown, therefore also check to see if the
+ * fetcher is still running.
+ */
+ private boolean isRunning() {
+ return !Thread.interrupted() && fetcherRef.isRunning();
+ }
+
+ /**
+ * Deserializes a record for collection, and accordingly updates the shard state in the fetcher.
+ * The last successfully collected sequence number in this shard consumer is also updated so
+ * that a {@link RecordPublisher} may be able to use the correct sequence number to refresh
+ * shard iterators if necessary.
+ *
+ * <p>Note that the server-side Kinesis timestamp is attached to the record when collected. When
+ * the user programs uses {@link TimeCharacteristic#EventTime}, this timestamp will be used by
+ * default.
+ *
+ * @param record record to deserialize and collect
+ */
+ private void deserializeRecordForCollectionAndUpdateState(final UserRecord record) {
+ ByteBuffer recordData = record.getData();
+
+ byte[] dataBytes = new byte[recordData.remaining()];
+ recordData.get(dataBytes);
+
+ final long approxArrivalTimestamp = record.getApproximateArrivalTimestamp().getTime();
+
+ final T value;
+ try {
+ value =
+ deserializer.deserialize(
+ dataBytes,
+ record.getPartitionKey(),
+ record.getSequenceNumber(),
+ approxArrivalTimestamp,
+ subscribedShard.getStreamName(),
+ subscribedShard.getShard().getShardId());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ SequenceNumber collectedSequenceNumber =
+ (record.isAggregated())
+ ? new SequenceNumber(
+ record.getSequenceNumber(), record.getSubSequenceNumber())
+ : new SequenceNumber(record.getSequenceNumber());
+
+ fetcherRef.emitRecordAndUpdateState(
+ value, approxArrivalTimestamp, subscribedShardStateIndex, collectedSequenceNumber);
+
+ this.lastSequenceNum = collectedSequenceNumber;
+ }
+
+ /**
+ * Filters out aggregated records that have previously been processed. This method is to support
+ * restarting from a partially consumed aggregated sequence number.
+ *
+ * @param record the record to filter
+ * @return true if the record should be retained
+ */
+ private boolean filterDeaggregatedRecord(final UserRecord record) {
+ if (!lastSequenceNum.isAggregated()) {
+ return true;
+ }
+
+ return !record.getSequenceNumber().equals(lastSequenceNum.getSequenceNumber())
+ || record.getSubSequenceNumber() > lastSequenceNum.getSubSequenceNumber();
+ }
+
+ /** An exception wrapper to indicate an error has been thrown from the shard consumer. */
+ abstract static class ShardConsumerException extends RuntimeException {
+ private static final long serialVersionUID = 7732343624482321663L;
+
+ public ShardConsumerException(final String message) {
+ super(message);
+ }
+ }
+
+ /** An exception to indicate the shard consumer has been cancelled. */
+ static class ShardConsumerCancelledException extends ShardConsumerException {
+ private static final long serialVersionUID = 2707399313569728649L;
+
+ public ShardConsumerCancelledException(final String message) {
+ super(message);
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordBatch.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordBatch.java
new file mode 100644
index 0000000..cca1880
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordBatch.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.util.Preconditions;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import com.amazonaws.services.kinesis.model.Record;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.List;
+
+/**
+ * A batch of UserRecords received from Kinesis. Input records are de-aggregated using KCL 1.x
+ * library. It is expected that AWS SDK v2.x messages are converted to KCL 1.x {@link UserRecord}.
+ */
+@Internal
+public class RecordBatch {
+
+ private final int aggregatedRecordSize;
+
+ private final List<UserRecord> deaggregatedRecords;
+
+ private final long totalSizeInBytes;
+
+ private final Long millisBehindLatest;
+
+ public RecordBatch(
+ final List<Record> records,
+ final StreamShardHandle subscribedShard,
+ @Nullable final Long millisBehindLatest) {
+ Preconditions.checkNotNull(subscribedShard);
+ this.aggregatedRecordSize = Preconditions.checkNotNull(records).size();
+ this.deaggregatedRecords = deaggregateRecords(records, subscribedShard);
+ this.totalSizeInBytes =
+ this.deaggregatedRecords.stream().mapToInt(r -> r.getData().remaining()).sum();
+ this.millisBehindLatest = millisBehindLatest;
+ }
+
+ public int getAggregatedRecordSize() {
+ return aggregatedRecordSize;
+ }
+
+ public int getDeaggregatedRecordSize() {
+ return deaggregatedRecords.size();
+ }
+
+ public List<UserRecord> getDeaggregatedRecords() {
+ return deaggregatedRecords;
+ }
+
+ public long getTotalSizeInBytes() {
+ return totalSizeInBytes;
+ }
+
+ public long getAverageRecordSizeBytes() {
+ return deaggregatedRecords.isEmpty()
+ ? 0
+ : getTotalSizeInBytes() / getDeaggregatedRecordSize();
+ }
+
+ @Nullable
+ public Long getMillisBehindLatest() {
+ return millisBehindLatest;
+ }
+
+ private List<UserRecord> deaggregateRecords(
+ final List<Record> records, final StreamShardHandle subscribedShard) {
+ BigInteger start =
+ new BigInteger(subscribedShard.getShard().getHashKeyRange().getStartingHashKey());
+ BigInteger end =
+ new BigInteger(subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
+
+ return UserRecord.deaggregate(records, start, end);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java
new file mode 100644
index 0000000..9da8794
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+
+/**
+ * A {@code RecordPublisher} will consume records from an external stream and deliver them to the
+ * registered subscriber.
+ */
+@Internal
+public interface RecordPublisher {
+
+ /**
+ * Run the record publisher. Records will be consumed from the stream and published to the
+ * consumer. The number of batches retrieved by a single invocation will vary based on
+ * implementation.
+ *
+ * @param recordBatchConsumer the record batch consumer in which to output records
+ * @return a status enum to represent whether a shard has been fully consumed
+ * @throws InterruptedException
+ */
+ RecordPublisherRunResult run(RecordBatchConsumer recordBatchConsumer)
+ throws InterruptedException;
+
+ /** A status enum to represent whether a shard has been fully consumed. */
+ enum RecordPublisherRunResult {
+ /** There are no more records to consume from this shard. */
+ COMPLETE,
+
+ /** There are more records to consume from this shard. */
+ INCOMPLETE,
+
+ /** The record publisher has been cancelled. */
+ CANCELLED
+ }
+
+ /**
+ * An interface used to collect record batches, and reply with the latest consumed sequence
+ * number.
+ */
+ interface RecordBatchConsumer {
+
+ SequenceNumber accept(RecordBatch recordBatch);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
new file mode 100644
index 0000000..a4a5848
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import java.util.Properties;
+
+/** A factory interface used to create instances of {@link RecordPublisher}. */
+@Internal
+public interface RecordPublisherFactory {
+
+ /**
+ * Create a {@link RecordPublisher}.
+ *
+ * @param startingPosition the position in the shard to start consuming records from
+ * @param consumerConfig the properties used to configure the {@link RecordPublisher}.
+ * @param metricGroup the {@link MetricGroup} used to report metrics to
+ * @param streamShardHandle the stream shard in which to consume from
+ * @return the constructed {@link RecordPublisher}
+ */
+ RecordPublisher create(
+ StartingPosition startingPosition,
+ Properties consumerConfig,
+ MetricGroup metricGroup,
+ StreamShardHandle streamShardHandle)
+ throws InterruptedException;
+
+ /** Destroy any open resources used by the factory. */
+ default void close() {
+ // Do nothing by default
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisher.java
new file mode 100644
index 0000000..9df68bf
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisher.java
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutShardSubscriber.FanOutSubscriberException;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutShardSubscriber.FanOutSubscriberInterruptedException;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutShardSubscriber.RecoverableFanOutSubscriberException;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.FullJitterBackoff;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.EncryptionType;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
+
+import javax.annotation.Nonnull;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.CANCELLED;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM;
+import static software.amazon.awssdk.services.kinesis.model.StartingPosition.builder;
+
+/**
+ * A {@link RecordPublisher} that will read and forward records from Kinesis using EFO, to the
+ * subscriber. Records are consumed via Enhanced Fan Out subscriptions using SubscribeToShard API.
+ */
+@Internal
+public class FanOutRecordPublisher implements RecordPublisher {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FanOutRecordPublisher.class);
+
+ private final FullJitterBackoff backoff;
+
+ private final String consumerArn;
+
+ private final KinesisProxyV2Interface kinesisProxy;
+
+ private final StreamShardHandle subscribedShard;
+
+ private final FanOutRecordPublisherConfiguration configuration;
+
+ /** The current attempt in the case of subsequent recoverable errors. */
+ private int attempt = 0;
+
+ private StartingPosition nextStartingPosition;
+
+ /**
+ * Instantiate a new FanOutRecordPublisher. Consumes data from KDS using EFO SubscribeToShard
+ * over AWS SDK V2.x
+ *
+ * @param startingPosition the position in the shard to start consuming from
+ * @param consumerArn the consumer ARN of the stream consumer
+ * @param subscribedShard the shard to consumer from
+ * @param kinesisProxy the proxy used to talk to Kinesis services
+ * @param configuration the record publisher configuration
+ */
+ public FanOutRecordPublisher(
+ final StartingPosition startingPosition,
+ final String consumerArn,
+ final StreamShardHandle subscribedShard,
+ final KinesisProxyV2Interface kinesisProxy,
+ final FanOutRecordPublisherConfiguration configuration,
+ final FullJitterBackoff backoff) {
+ this.nextStartingPosition = Preconditions.checkNotNull(startingPosition);
+ this.consumerArn = Preconditions.checkNotNull(consumerArn);
+ this.subscribedShard = Preconditions.checkNotNull(subscribedShard);
+ this.kinesisProxy = Preconditions.checkNotNull(kinesisProxy);
+ this.configuration = Preconditions.checkNotNull(configuration);
+ this.backoff = Preconditions.checkNotNull(backoff);
+ }
+
+ @Override
+ public RecordPublisherRunResult run(final RecordBatchConsumer recordConsumer)
+ throws InterruptedException {
+ LOG.info(
+ "Running fan out record publisher on {}::{} from {} - {}",
+ subscribedShard.getStreamName(),
+ subscribedShard.getShard().getShardId(),
+ nextStartingPosition.getShardIteratorType(),
+ nextStartingPosition.getStartingMarker());
+
+ Consumer<SubscribeToShardEvent> eventConsumer =
+ event -> {
+ RecordBatch recordBatch =
+ new RecordBatch(
+ toSdkV1Records(event.records()),
+ subscribedShard,
+ event.millisBehindLatest());
+ SequenceNumber sequenceNumber = recordConsumer.accept(recordBatch);
+ nextStartingPosition = getNextStartingPosition(sequenceNumber);
+ };
+
+ RecordPublisherRunResult result = runWithBackoff(eventConsumer);
+
+ LOG.info(
+ "Subscription expired {}::{}, with status {}",
+ subscribedShard.getStreamName(),
+ subscribedShard.getShard().getShardId(),
+ result);
+
+ return result;
+ }
+
+ private StartingPosition getNextStartingPosition(final SequenceNumber latestSequenceNumber) {
+ // When consuming from a timestamp sentinel/AT_TIMESTAMP ShardIteratorType.
+ // If the first RecordBatch has no deaggregated records, then the latestSequenceNumber would
+ // be the timestamp sentinel.
+ // This is because we have not yet received any real sequence numbers on this shard.
+ // In this condition we should retry from the previous starting position (AT_TIMESTAMP).
+ if (SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get().equals(latestSequenceNumber)) {
+ Preconditions.checkState(nextStartingPosition.getShardIteratorType() == AT_TIMESTAMP);
+ return nextStartingPosition;
+ } else {
+ return StartingPosition.continueFromSequenceNumber(latestSequenceNumber);
+ }
+ }
+
+ /**
+ * Runs the record publisher, will sleep for configuration computed jitter period in the case of
+ * certain exceptions. Unrecoverable exceptions are thrown to terminate the application.
+ *
+ * @param eventConsumer the consumer to pass events to
+ * @return {@code COMPLETE} if the shard is complete and this shard consumer should exit
+ * @throws InterruptedException
+ */
+ private RecordPublisherRunResult runWithBackoff(
+ final Consumer<SubscribeToShardEvent> eventConsumer) throws InterruptedException {
+ FanOutShardSubscriber fanOutShardSubscriber =
+ new FanOutShardSubscriber(
+ consumerArn,
+ subscribedShard.getShard().getShardId(),
+ kinesisProxy,
+ configuration.getSubscribeToShardTimeout());
+ boolean complete;
+
+ try {
+ complete =
+ fanOutShardSubscriber.subscribeToShardAndConsumeRecords(
+ toSdkV2StartingPosition(nextStartingPosition), eventConsumer);
+ attempt = 0;
+ } catch (FanOutSubscriberInterruptedException ex) {
+ LOG.info(
+ "Thread interrupted, closing record publisher for shard {}.",
+ subscribedShard.getShard().getShardId(),
+ ex);
+ return CANCELLED;
+ } catch (RecoverableFanOutSubscriberException ex) {
+ // Recoverable errors should be reattempted without contributing to the retry policy
+ // A recoverable error would not result in the Flink job being cancelled
+ backoff(ex);
+ return INCOMPLETE;
+ } catch (FanOutSubscriberException ex) {
+ // We have received an error from the network layer
+ // This can be due to limits being exceeded, network timeouts, etc
+ // We should backoff, reacquire a subscription and try again
+ if (ex.getCause() instanceof ResourceNotFoundException) {
+ LOG.warn(
+ "Received ResourceNotFoundException. Either the shard does not exist, or the stream subscriber has been deregistered."
+ + "Marking this shard as complete {} ({})",
+ subscribedShard.getShard().getShardId(),
+ consumerArn);
+
+ return COMPLETE;
+ }
+
+ if (attempt == configuration.getSubscribeToShardMaxRetries()) {
+ final String errorMessage =
+ "Maximum retries exceeded for SubscribeToShard. "
+ + "Failed "
+ + configuration.getSubscribeToShardMaxRetries()
+ + " times.";
+ LOG.error(errorMessage, ex.getCause());
+ throw new RuntimeException(errorMessage, ex.getCause());
+ }
+
+ attempt++;
+ backoff(ex);
+ return INCOMPLETE;
+ }
+
+ return complete ? COMPLETE : INCOMPLETE;
+ }
+
+ private void backoff(final Throwable ex) throws InterruptedException {
+ long backoffMillis =
+ backoff.calculateFullJitterBackoff(
+ configuration.getSubscribeToShardBaseBackoffMillis(),
+ configuration.getSubscribeToShardMaxBackoffMillis(),
+ configuration.getSubscribeToShardExpConstant(),
+ attempt);
+
+ LOG.warn(
+ "Encountered recoverable error {}. Backing off for {} millis {} ({})",
+ ex.getCause().getClass().getSimpleName(),
+ backoffMillis,
+ subscribedShard.getShard().getShardId(),
+ consumerArn,
+ ex);
+
+ backoff.sleep(backoffMillis);
+ }
+
+ /**
+ * Records that come from KPL may be aggregated. Records must be deaggregated before they are
+ * processed by the application. Deaggregation is performed by KCL. In order to prevent having
+ * to import KCL 1.x and 2.x we convert the records to v1 format and use KCL v1.
+ *
+ * @param records the SDK v2 records
+ * @return records converted to SDK v1 format
+ */
+ private List<com.amazonaws.services.kinesis.model.Record> toSdkV1Records(
+ final List<Record> records) {
+ final List<com.amazonaws.services.kinesis.model.Record> sdkV1Records = new ArrayList<>();
+
+ for (Record record : records) {
+ sdkV1Records.add(toSdkV1Record(record));
+ }
+
+ return sdkV1Records;
+ }
+
+ private com.amazonaws.services.kinesis.model.Record toSdkV1Record(
+ @Nonnull final Record record) {
+ final com.amazonaws.services.kinesis.model.Record recordV1 =
+ new com.amazonaws.services.kinesis.model.Record()
+ .withData(record.data().asByteBuffer())
+ .withSequenceNumber(record.sequenceNumber())
+ .withPartitionKey(record.partitionKey())
+ .withApproximateArrivalTimestamp(
+ new Date(record.approximateArrivalTimestamp().toEpochMilli()));
+
+ EncryptionType encryptionType = record.encryptionType();
+ if (encryptionType != null) {
+ recordV1.withEncryptionType(encryptionType.name());
+ }
+
+ return recordV1;
+ }
+
+ /**
+ * Converts a local {@link StartingPosition} to an AWS SDK V2 object representation.
+ *
+ * @param startingPosition the local {@link StartingPosition}
+ * @return an AWS SDK V2 representation
+ */
+ private software.amazon.awssdk.services.kinesis.model.StartingPosition toSdkV2StartingPosition(
+ StartingPosition startingPosition) {
+ software.amazon.awssdk.services.kinesis.model.StartingPosition.Builder builder =
+ builder().type(startingPosition.getShardIteratorType().toString());
+
+ Object marker = startingPosition.getStartingMarker();
+
+ switch (startingPosition.getShardIteratorType()) {
+ case AT_TIMESTAMP:
+ {
+ Preconditions.checkNotNull(
+ marker, "StartingPosition AT_TIMESTAMP date marker is null.");
+ builder.timestamp(((Date) marker).toInstant());
+ break;
+ }
+ case AT_SEQUENCE_NUMBER:
+ case AFTER_SEQUENCE_NUMBER:
+ {
+ Preconditions.checkNotNull(
+ marker, "StartingPosition *_SEQUENCE_NUMBER position is null.");
+ builder.sequenceNumber(marker.toString());
+ break;
+ }
+ }
+
+ return builder.build();
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherConfiguration.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherConfiguration.java
new file mode 100644
index 0000000..cd46876
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherConfiguration.java
@@ -0,0 +1,475 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.fanout;
+
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.EFORegistrationType;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.RecordPublisherType;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+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.kinesis.config.ConsumerConfigConstants.efoConsumerArn;
+
+/** This is a configuration class for enhanced fan-out components. */
+public class FanOutRecordPublisherConfiguration {
+
+ /** The efo registration type for de-/registration of streams. */
+ private final EFORegistrationType efoRegistrationType;
+
+ /**
+ * The efo stream consumer name. Should not be Null if the efoRegistrationType is either LAZY or
+ * EAGER.
+ */
+ @Nullable private String consumerName;
+
+ /** A map of stream to stream consumer ARN for EFO subscriptions. */
+ private final Map<String, String> streamConsumerArns = new HashMap<>();
+
+ /** Base backoff millis for the deregister stream operation. */
+ private final int subscribeToShardMaxRetries;
+
+ /** A timeout when waiting for a shard subscription to be established. */
+ private final Duration subscribeToShardTimeout;
+
+ /** Maximum backoff millis for the subscribe to shard operation. */
+ private final long subscribeToShardMaxBackoffMillis;
+
+ /** Base backoff millis for the subscribe to shard operation. */
+ private final long subscribeToShardBaseBackoffMillis;
+
+ /** Exponential backoff power constant for the subscribe to shard operation. */
+ private final double subscribeToShardExpConstant;
+
+ /** Base backoff millis for the register stream operation. */
+ private final long registerStreamBaseBackoffMillis;
+
+ /** Maximum backoff millis for the register stream operation. */
+ private final long registerStreamMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the register stream operation. */
+ private final double registerStreamExpConstant;
+
+ /** Maximum retry attempts for the register stream operation. */
+ private final int registerStreamMaxRetries;
+
+ /** Maximum time to wait for a stream consumer to become active before giving up. */
+ private final Duration registerStreamConsumerTimeout;
+
+ /** Base backoff millis for the deregister stream operation. */
+ private final long deregisterStreamBaseBackoffMillis;
+
+ /** Maximum backoff millis for the deregister stream operation. */
+ private final long deregisterStreamMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the deregister stream operation. */
+ private final double deregisterStreamExpConstant;
+
+ /** Maximum retry attempts for the deregister stream operation. */
+ private final int deregisterStreamMaxRetries;
+
+ /** Maximum time to wait for a stream consumer to deregister before giving up. */
+ private final Duration deregisterStreamConsumerTimeout;
+
+ /** Max retries for the describe stream operation. */
+ private final int describeStreamMaxRetries;
+
+ /** Backoff millis for the describe stream operation. */
+ private final long describeStreamBaseBackoffMillis;
+
+ /** Maximum backoff millis for the describe stream operation. */
+ private final long describeStreamMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the describe stream operation. */
+ private final double describeStreamExpConstant;
+
+ /** Max retries for the describe stream consumer operation. */
+ private final int describeStreamConsumerMaxRetries;
+
+ /** Backoff millis for the describe stream consumer operation. */
+ private final long describeStreamConsumerBaseBackoffMillis;
+
+ /** Maximum backoff millis for the describe stream consumer operation. */
+ private final long describeStreamConsumerMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the describe stream consumer operation. */
+ private final double describeStreamConsumerExpConstant;
+
+ /**
+ * Creates a FanOutRecordPublisherConfiguration.
+ *
+ * @param configProps the configuration properties from config file.
+ * @param streams the streams which is sent to match the EFO consumer arn if the EFO
+ * registration mode is set to `NONE`.
+ */
+ public FanOutRecordPublisherConfiguration(
+ final Properties configProps, final List<String> streams) {
+ Preconditions.checkArgument(
+ configProps
+ .getProperty(ConsumerConfigConstants.RECORD_PUBLISHER_TYPE)
+ .equals(RecordPublisherType.EFO.toString()),
+ "Only efo record publisher can register a FanOutProperties.");
+ KinesisConfigUtil.validateEfoConfiguration(configProps, streams);
+
+ efoRegistrationType =
+ EFORegistrationType.valueOf(
+ configProps.getProperty(
+ ConsumerConfigConstants.EFO_REGISTRATION_TYPE,
+ EFORegistrationType.EAGER.toString()));
+ // if efo registration type is EAGER|LAZY, then user should explicitly provide a consumer
+ // name for each stream.
+ if (efoRegistrationType == EFORegistrationType.EAGER
+ || efoRegistrationType == EFORegistrationType.LAZY) {
+ consumerName = configProps.getProperty(ConsumerConfigConstants.EFO_CONSUMER_NAME);
+ }
+
+ for (String stream : streams) {
+ String key = efoConsumerArn(stream);
+ if (configProps.containsKey(key)) {
+ streamConsumerArns.put(stream, configProps.getProperty(key));
+ }
+ }
+
+ this.subscribeToShardMaxRetries =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.SUBSCRIBE_TO_SHARD_RETRIES))
+ .map(Integer::parseInt)
+ .orElse(ConsumerConfigConstants.DEFAULT_SUBSCRIBE_TO_SHARD_RETRIES);
+ this.subscribeToShardTimeout =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.SUBSCRIBE_TO_SHARD_TIMEOUT_SECONDS))
+ .map(Integer::parseInt)
+ .map(Duration::ofSeconds)
+ .orElse(ConsumerConfigConstants.DEFAULT_SUBSCRIBE_TO_SHARD_TIMEOUT);
+ this.subscribeToShardBaseBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.SUBSCRIBE_TO_SHARD_BACKOFF_BASE))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_BASE);
+ this.subscribeToShardMaxBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.SUBSCRIBE_TO_SHARD_BACKOFF_MAX))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_MAX);
+ this.subscribeToShardExpConstant =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .SUBSCRIBE_TO_SHARD_BACKOFF_EXPONENTIAL_CONSTANT))
+ .map(Double::parseDouble)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_SUBSCRIBE_TO_SHARD_BACKOFF_EXPONENTIAL_CONSTANT);
+
+ this.registerStreamBaseBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.REGISTER_STREAM_BACKOFF_BASE))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_REGISTER_STREAM_BACKOFF_BASE);
+ this.registerStreamMaxBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.REGISTER_STREAM_BACKOFF_MAX))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_REGISTER_STREAM_BACKOFF_MAX);
+ this.registerStreamExpConstant =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .REGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT))
+ .map(Double::parseDouble)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_REGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT);
+ this.registerStreamMaxRetries =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.REGISTER_STREAM_RETRIES))
+ .map(Integer::parseInt)
+ .orElse(ConsumerConfigConstants.DEFAULT_REGISTER_STREAM_RETRIES);
+ this.registerStreamConsumerTimeout =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.REGISTER_STREAM_TIMEOUT_SECONDS))
+ .map(Integer::parseInt)
+ .map(Duration::ofSeconds)
+ .orElse(ConsumerConfigConstants.DEFAULT_REGISTER_STREAM_TIMEOUT);
+
+ this.deregisterStreamBaseBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.DEREGISTER_STREAM_BACKOFF_BASE))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_DEREGISTER_STREAM_BACKOFF_BASE);
+ this.deregisterStreamMaxBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.DEREGISTER_STREAM_BACKOFF_MAX))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_DEREGISTER_STREAM_BACKOFF_MAX);
+ this.deregisterStreamExpConstant =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .DEREGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT))
+ .map(Double::parseDouble)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_DEREGISTER_STREAM_BACKOFF_EXPONENTIAL_CONSTANT);
+ this.deregisterStreamMaxRetries =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.DEREGISTER_STREAM_RETRIES))
+ .map(Integer::parseInt)
+ .orElse(ConsumerConfigConstants.DEFAULT_DEREGISTER_STREAM_RETRIES);
+ this.deregisterStreamConsumerTimeout =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.DEREGISTER_STREAM_TIMEOUT_SECONDS))
+ .map(Integer::parseInt)
+ .map(Duration::ofSeconds)
+ .orElse(ConsumerConfigConstants.DEFAULT_DEREGISTER_STREAM_TIMEOUT);
+
+ this.describeStreamMaxRetries =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_DESCRIBE_RETRIES))
+ .map(Integer::parseInt)
+ .orElse(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRIES);
+ this.describeStreamBaseBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE);
+ this.describeStreamMaxBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX))
+ .map(Long::parseLong)
+ .orElse(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX);
+ this.describeStreamExpConstant =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))
+ .map(Double::parseDouble)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT);
+ this.describeStreamConsumerMaxRetries =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants.DESCRIBE_STREAM_CONSUMER_RETRIES))
+ .map(Integer::parseInt)
+ .orElse(ConsumerConfigConstants.DEFAULT_DESCRIBE_STREAM_CONSUMER_RETRIES);
+ this.describeStreamConsumerBaseBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .DESCRIBE_STREAM_CONSUMER_BACKOFF_BASE))
+ .map(Long::parseLong)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_BASE);
+ this.describeStreamConsumerMaxBackoffMillis =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .DESCRIBE_STREAM_CONSUMER_BACKOFF_MAX))
+ .map(Long::parseLong)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_MAX);
+ this.describeStreamConsumerExpConstant =
+ Optional.ofNullable(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .DESCRIBE_STREAM_CONSUMER_BACKOFF_EXPONENTIAL_CONSTANT))
+ .map(Double::parseDouble)
+ .orElse(
+ ConsumerConfigConstants
+ .DEFAULT_DESCRIBE_STREAM_CONSUMER_BACKOFF_EXPONENTIAL_CONSTANT);
+ }
+
+ // ------------------------------------------------------------------------
+ // subscribeToShard() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Get maximum retry attempts for the subscribe to shard operation. */
+ public int getSubscribeToShardMaxRetries() {
+ return subscribeToShardMaxRetries;
+ }
+
+ /** Get timeout when waiting for a shard subscription to be established. */
+ public Duration getSubscribeToShardTimeout() {
+ return subscribeToShardTimeout;
+ }
+
+ /** Get maximum backoff millis for the subscribe to shard operation. */
+ public long getSubscribeToShardMaxBackoffMillis() {
+ return subscribeToShardMaxBackoffMillis;
+ }
+
+ /** Get base backoff millis for the subscribe to shard operation. */
+ public long getSubscribeToShardBaseBackoffMillis() {
+ return subscribeToShardBaseBackoffMillis;
+ }
+
+ /** Get exponential backoff power constant for the subscribe to shard operation. */
+ public double getSubscribeToShardExpConstant() {
+ return subscribeToShardExpConstant;
+ }
+
+ // ------------------------------------------------------------------------
+ // registerStream() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Get base backoff millis for the register stream operation. */
+ public long getRegisterStreamBaseBackoffMillis() {
+ return registerStreamBaseBackoffMillis;
+ }
+
+ /** Get maximum backoff millis for the register stream operation. */
+ public long getRegisterStreamMaxBackoffMillis() {
+ return registerStreamMaxBackoffMillis;
+ }
+
+ /** Get exponential backoff power constant for the register stream operation. */
+ public double getRegisterStreamExpConstant() {
+ return registerStreamExpConstant;
+ }
+
+ /** Get maximum retry attempts for the register stream operation. */
+ public int getRegisterStreamMaxRetries() {
+ return registerStreamMaxRetries;
+ }
+
+ /** Get maximum duration to wait for a stream consumer to become active before giving up. */
+ public Duration getRegisterStreamConsumerTimeout() {
+ return registerStreamConsumerTimeout;
+ }
+
+ // ------------------------------------------------------------------------
+ // deregisterStream() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Get base backoff millis for the deregister stream operation. */
+ public long getDeregisterStreamBaseBackoffMillis() {
+ return deregisterStreamBaseBackoffMillis;
+ }
+
+ /** Get maximum backoff millis for the deregister stream operation. */
+ public long getDeregisterStreamMaxBackoffMillis() {
+ return deregisterStreamMaxBackoffMillis;
+ }
+
+ /** Get exponential backoff power constant for the deregister stream operation. */
+ public double getDeregisterStreamExpConstant() {
+ return deregisterStreamExpConstant;
+ }
+
+ /** Get maximum retry attempts for the register stream operation. */
+ public int getDeregisterStreamMaxRetries() {
+ return deregisterStreamMaxRetries;
+ }
+
+ /** Get maximum duration to wait for a stream consumer to deregister before giving up. */
+ public Duration getDeregisterStreamConsumerTimeout() {
+ return deregisterStreamConsumerTimeout;
+ }
+
+ // ------------------------------------------------------------------------
+ // describeStream() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Get maximum retry attempts for the describe stream operation. */
+ public int getDescribeStreamMaxRetries() {
+ return describeStreamMaxRetries;
+ }
+
+ /** Get base backoff millis for the describe stream operation. */
+ public long getDescribeStreamBaseBackoffMillis() {
+ return describeStreamBaseBackoffMillis;
+ }
+
+ /** Get maximum backoff millis for the describe stream operation. */
+ public long getDescribeStreamMaxBackoffMillis() {
+ return describeStreamMaxBackoffMillis;
+ }
+
+ /** Get exponential backoff power constant for the describe stream operation. */
+ public double getDescribeStreamExpConstant() {
+ return describeStreamExpConstant;
+ }
+
+ // ------------------------------------------------------------------------
+ // describeStreamConsumer() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Get maximum retry attempts for the describe stream operation. */
+ public int getDescribeStreamConsumerMaxRetries() {
+ return describeStreamConsumerMaxRetries;
+ }
+
+ /** Get base backoff millis for the describe stream operation. */
+ public long getDescribeStreamConsumerBaseBackoffMillis() {
+ return describeStreamConsumerBaseBackoffMillis;
+ }
+
+ /** Get maximum backoff millis for the describe stream operation. */
+ public long getDescribeStreamConsumerMaxBackoffMillis() {
+ return describeStreamConsumerMaxBackoffMillis;
+ }
+
+ /** Get exponential backoff power constant for the describe stream operation. */
+ public double getDescribeStreamConsumerExpConstant() {
+ return describeStreamConsumerExpConstant;
+ }
+
+ /** Get efo registration type. */
+ public EFORegistrationType getEfoRegistrationType() {
+ return efoRegistrationType;
+ }
+
+ /** Get consumer name, will be null if efo registration type is 'NONE'. */
+ public Optional<String> getConsumerName() {
+ return Optional.ofNullable(consumerName);
+ }
+
+ /**
+ * Get the according consumer arn to the stream, will be null if efo registration type is 'LAZY'
+ * or 'EAGER'.
+ */
+ public Optional<String> getStreamConsumerArn(String stream) {
+ return Optional.ofNullable(streamConsumerArns.get(stream));
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherFactory.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherFactory.java
new file mode 100644
index 0000000..5752822
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisherFactory.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.FullJitterBackoff;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.util.Collections.singletonList;
+
+/** A {@link RecordPublisher} factory used to create instances of {@link FanOutRecordPublisher}. */
+@Internal
+public class FanOutRecordPublisherFactory implements RecordPublisherFactory {
+
+ private static final FullJitterBackoff BACKOFF = new FullJitterBackoff();
+
+ /**
+ * A singleton {@link KinesisProxyV2} is used per Flink task. The {@link KinesisAsyncClient}
+ * uses an internal thread pool; using a single client reduces overhead.
+ */
+ private final KinesisProxyV2Interface kinesisProxy;
+
+ /**
+ * Instantiate a factory responsible for creating {@link FanOutRecordPublisher}.
+ *
+ * @param kinesisProxy the singleton proxy used by all record publishers created by this factory
+ */
+ public FanOutRecordPublisherFactory(final KinesisProxyV2Interface kinesisProxy) {
+ this.kinesisProxy = kinesisProxy;
+ }
+
+ /**
+ * Create a {@link FanOutRecordPublisher}.
+ *
+ * @param startingPosition the starting position in the shard to start consuming from
+ * @param consumerConfig the consumer configuration properties
+ * @param metricGroup the metric group to report metrics to
+ * @param streamShardHandle the shard this consumer is subscribed to
+ * @return a {@link FanOutRecordPublisher}
+ */
+ @Override
+ public FanOutRecordPublisher create(
+ final StartingPosition startingPosition,
+ final Properties consumerConfig,
+ final MetricGroup metricGroup,
+ final StreamShardHandle streamShardHandle) {
+ Preconditions.checkNotNull(startingPosition);
+ Preconditions.checkNotNull(consumerConfig);
+ Preconditions.checkNotNull(metricGroup);
+ Preconditions.checkNotNull(streamShardHandle);
+
+ String stream = streamShardHandle.getStreamName();
+ FanOutRecordPublisherConfiguration configuration =
+ new FanOutRecordPublisherConfiguration(consumerConfig, singletonList(stream));
+
+ Optional<String> streamConsumerArn = configuration.getStreamConsumerArn(stream);
+ Preconditions.checkState(streamConsumerArn.isPresent());
+
+ return new FanOutRecordPublisher(
+ startingPosition,
+ streamConsumerArn.get(),
+ streamShardHandle,
+ kinesisProxy,
+ configuration,
+ BACKOFF);
+ }
+
+ @Override
+ public void close() {
+ kinesisProxy.close();
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutShardSubscriber.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutShardSubscriber.java
new file mode 100644
index 0000000..a280a8f
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutShardSubscriber.java
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import io.netty.handler.timeout.ReadTimeoutException;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.kinesis.model.StartingPosition;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
+
+import java.time.Duration;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * This class is responsible for acquiring an Enhanced Fan Out subscription and consuming records
+ * from a shard. A queue is used to buffer records between the Kinesis Proxy and Flink application.
+ * This allows processing to be separated from consumption; errors thrown in the consumption layer
+ * do not propagate up to application.
+ *
+ * <pre>{@code [
+ * | ----------- Source Connector Thread ----------- | | --- KinesisAsyncClient Thread(s) -- |
+ * | FanOutRecordPublisher | FanOutShardSubscription | == blocking queue == | KinesisProxyV2 | KinesisAsyncClient |
+ * ]}</pre>
+ *
+ * <p>Three types of message are passed over the queue for inter-thread communication:
+ *
+ * <ul>
+ * <li>{@link SubscriptionNextEvent} - passes data from the network to the consumer
+ * <li>{@link SubscriptionCompleteEvent} - indicates a subscription has expired
+ * <li>{@link SubscriptionErrorEvent} - passes an exception from the network to the consumer
+ * </ul>
+ *
+ * <p>The blocking queue has a maximum capacity of two. One slot is used for a record batch, the
+ * remaining slot is reserved to completion events. At maximum capacity we will have two {@link
+ * SubscribeToShardEvent} in memory (per instance of this class):
+ *
+ * <ul>
+ * <li>1 event being processed by the consumer
+ * <li>1 event enqueued in the blocking queue
+ * </ul>
+ */
+@Internal
+public class FanOutShardSubscriber {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FanOutShardSubscriber.class);
+
+ /**
+ * The maximum capacity of the queue between the network and consumption thread. The queue is
+ * mainly used to isolate networking from consumption such that errors do not bubble up. This
+ * queue also acts as a buffer resulting in a record prefetch and reduced latency. Capacity is 2
+ * to allow 1 pending record batch and leave room for a completion event to avoid any writer
+ * thread blocking on the queue.
+ */
+ private static final int QUEUE_CAPACITY = 2;
+
+ /**
+ * Read timeout will occur after 30 seconds, a sanity timeout to prevent lockup in unexpected
+ * error states. If the consumer does not receive a new event within the QUEUE_TIMEOUT_SECONDS
+ * it will backoff and resubscribe.
+ */
+ private static final Duration DEFAULT_QUEUE_TIMEOUT = Duration.ofSeconds(35);
+
+ private final BlockingQueue<FanOutSubscriptionEvent> queue =
+ new LinkedBlockingQueue<>(QUEUE_CAPACITY);
+
+ private final AtomicReference<FanOutSubscriptionEvent> subscriptionErrorEvent =
+ new AtomicReference<>();
+
+ private final KinesisProxyV2Interface kinesis;
+
+ private final String consumerArn;
+
+ private final String shardId;
+
+ private final Duration subscribeToShardTimeout;
+
+ private final Duration queueWaitTimeout;
+
+ /**
+ * Create a new Fan Out Shard subscriber.
+ *
+ * @param consumerArn the stream consumer ARN
+ * @param shardId the shard ID to subscribe to
+ * @param kinesis the Kinesis Proxy used to communicate via AWS SDK v2
+ * @param subscribeToShardTimeout A timeout when waiting for a shard subscription to be
+ * established
+ */
+ FanOutShardSubscriber(
+ final String consumerArn,
+ final String shardId,
+ final KinesisProxyV2Interface kinesis,
+ final Duration subscribeToShardTimeout) {
+ this(consumerArn, shardId, kinesis, subscribeToShardTimeout, DEFAULT_QUEUE_TIMEOUT);
+ }
+
+ /**
+ * Create a new Fan Out Shard Subscriber.
+ *
+ * @param consumerArn the stream consumer ARN
+ * @param shardId the shard ID to subscribe to
+ * @param kinesis the Kinesis Proxy used to communicate via AWS SDK v2
+ * @param subscribeToShardTimeout A timeout when waiting for a shard subscription to be
+ * established
+ * @param queueWaitTimeout A timeout when enqueuing/de-queueing
+ */
+ @VisibleForTesting
+ FanOutShardSubscriber(
+ final String consumerArn,
+ final String shardId,
+ final KinesisProxyV2Interface kinesis,
+ final Duration subscribeToShardTimeout,
+ final Duration queueWaitTimeout) {
+ this.kinesis = Preconditions.checkNotNull(kinesis);
+ this.consumerArn = Preconditions.checkNotNull(consumerArn);
+ this.shardId = Preconditions.checkNotNull(shardId);
+ this.subscribeToShardTimeout = subscribeToShardTimeout;
+ this.queueWaitTimeout = queueWaitTimeout;
+ }
+
+ /**
+ * Obtains a subscription to the shard from the specified {@code startingPosition}. {@link
+ * SubscribeToShardEvent} received from KDS are delivered to the given {@code eventConsumer}.
+ * Returns false if there are records left to consume from the shard.
+ *
+ * @param startingPosition the position in the stream in which to start receiving records
+ * @param eventConsumer the consumer to deliver received events to
+ * @return true if there are no more messages (complete), false if a subsequent subscription
+ * should be obtained
+ * @throws FanOutSubscriberException when an exception is propagated from the networking stack
+ * @throws InterruptedException when the thread is interrupted
+ */
+ boolean subscribeToShardAndConsumeRecords(
+ final StartingPosition startingPosition,
+ final Consumer<SubscribeToShardEvent> eventConsumer)
+ throws InterruptedException, FanOutSubscriberException {
+ LOG.debug("Subscribing to shard {} ({})", shardId, consumerArn);
+
+ final FanOutShardSubscription subscription;
+ try {
+ subscription = openSubscriptionToShard(startingPosition);
+ } catch (FanOutSubscriberException ex) {
+ // The only exception that should cause a failure is a ResourceNotFoundException
+ // Rethrow the exception to trigger the application to terminate
+ if (ex.getCause() instanceof ResourceNotFoundException) {
+ throw (ResourceNotFoundException) ex.getCause();
+ }
+
+ throw ex;
+ }
+
+ return consumeAllRecordsFromKinesisShard(eventConsumer, subscription);
+ }
+
+ /**
+ * Calls {@link KinesisProxyV2#subscribeToShard} and waits to acquire a subscription. In the
+ * event a non-recoverable error occurs this method will rethrow the exception. Once the
+ * subscription is acquired the client signals to the producer that we are ready to receive
+ * records.
+ *
+ * @param startingPosition the position in which to start consuming from
+ * @throws FanOutSubscriberException when an exception is propagated from the networking stack
+ */
+ private FanOutShardSubscription openSubscriptionToShard(final StartingPosition startingPosition)
+ throws FanOutSubscriberException, InterruptedException {
+ SubscribeToShardRequest request =
+ SubscribeToShardRequest.builder()
+ .consumerARN(consumerArn)
+ .shardId(shardId)
+ .startingPosition(startingPosition)
+ .build();
+
+ AtomicReference<Throwable> exception = new AtomicReference<>();
+ CountDownLatch waitForSubscriptionLatch = new CountDownLatch(1);
+ FanOutShardSubscription subscription =
+ new FanOutShardSubscription(waitForSubscriptionLatch);
+
+ SubscribeToShardResponseHandler responseHandler =
+ SubscribeToShardResponseHandler.builder()
+ .onError(
+ e -> {
+ // Errors that occur while trying to acquire a subscription are
+ // only thrown from here
+ // Errors that occur during the subscription are surfaced here
+ // and to the FanOutShardSubscription
+ // (errors are ignored here once the subscription is open)
+ if (waitForSubscriptionLatch.getCount() > 0) {
+ exception.set(e);
+ waitForSubscriptionLatch.countDown();
+ }
+ })
+ .subscriber(() -> subscription)
+ .build();
+
+ kinesis.subscribeToShard(request, responseHandler);
+
+ boolean subscriptionEstablished =
+ waitForSubscriptionLatch.await(
+ subscribeToShardTimeout.toMillis(), TimeUnit.MILLISECONDS);
+
+ if (!subscriptionEstablished) {
+ final String errorMessage =
+ "Timed out acquiring subscription - " + shardId + " (" + consumerArn + ")";
+ LOG.error(errorMessage);
+ subscription.cancelSubscription();
+ handleError(
+ new RecoverableFanOutSubscriberException(new TimeoutException(errorMessage)));
+ }
+
+ Throwable throwable = exception.get();
+ if (throwable != null) {
+ handleError(throwable);
+ }
+
+ LOG.debug("Acquired subscription - {} ({})", shardId, consumerArn);
+
+ // Request the first record to kick off consumption
+ // Following requests are made by the FanOutShardSubscription on the netty thread
+ subscription.requestRecord();
+
+ return subscription;
+ }
+
+ /**
+ * Update the reference to the latest networking error in this object. Parent caller can
+ * interrogate to decide how to handle error.
+ *
+ * @param throwable the exception that has occurred
+ */
+ private void handleError(final Throwable throwable) throws FanOutSubscriberException {
+ Throwable cause;
+ if (throwable instanceof CompletionException || throwable instanceof ExecutionException) {
+ cause = throwable.getCause();
+ } else {
+ cause = throwable;
+ }
+
+ LOG.warn(
+ "Error occurred on EFO subscription: {} - ({}). {} ({})",
+ throwable.getClass().getName(),
+ throwable.getMessage(),
+ shardId,
+ consumerArn,
+ cause);
+
+ if (isInterrupted(throwable)) {
+ throw new FanOutSubscriberInterruptedException(throwable);
+ } else if (cause instanceof FanOutSubscriberException) {
+ throw (FanOutSubscriberException) cause;
+ } else if (cause instanceof ReadTimeoutException) {
+ // ReadTimeoutException occurs naturally under backpressure scenarios when full batches
+ // take longer to
+ // process than standard read timeout (default 30s). Recoverable exceptions are intended
+ // to be retried
+ // indefinitely to avoid system degradation under backpressure. The EFO connection
+ // (subscription) to Kinesis
+ // is closed, and reacquired once the queue of records has been processed.
+ throw new RecoverableFanOutSubscriberException(cause);
+ } else {
+ throw new RetryableFanOutSubscriberException(cause);
+ }
+ }
+
+ private boolean isInterrupted(final Throwable throwable) {
+ Throwable cause = throwable;
+ while (cause != null) {
+ if (cause instanceof InterruptedException) {
+ return true;
+ }
+
+ cause = cause.getCause();
+ }
+
+ return false;
+ }
+
+ /**
+ * Once the subscription is open, records will be delivered to the {@link BlockingQueue}. Queue
+ * capacity is hardcoded to 1 record, the queue is used solely to separate consumption and
+ * processing. However, this buffer will result in latency reduction as records are pre-fetched
+ * as a result. This method will poll the queue and exit under any of these conditions: - {@code
+ * continuationSequenceNumber} is {@code null}, indicating the shard is complete - The
+ * subscription expires, indicated by a {@link SubscriptionCompleteEvent} - There is an error
+ * while consuming records, indicated by a {@link SubscriptionErrorEvent}
+ *
+ * @param eventConsumer the event consumer to deliver records to
+ * @param subscription the subscription we are subscribed to
+ * @return true if there are no more messages (complete), false if a subsequent subscription
+ * should be obtained
+ * @throws FanOutSubscriberException when an exception is propagated from the networking stack
+ * @throws InterruptedException when the thread is interrupted
+ */
+ private boolean consumeAllRecordsFromKinesisShard(
+ final Consumer<SubscribeToShardEvent> eventConsumer,
+ final FanOutShardSubscription subscription)
+ throws InterruptedException, FanOutSubscriberException {
+ String continuationSequenceNumber;
+ boolean result = true;
+
+ do {
+ FanOutSubscriptionEvent subscriptionEvent;
+ if (subscriptionErrorEvent.get() != null) {
+ subscriptionEvent = subscriptionErrorEvent.get();
+ } else {
+ // Read timeout occurs after 30 seconds, add a sanity timeout to prevent lockup
+ subscriptionEvent = queue.poll(queueWaitTimeout.toMillis(), MILLISECONDS);
+ }
+
+ if (subscriptionEvent == null) {
+ LOG.info(
+ "Timed out polling events from network, reacquiring subscription - {} ({})",
+ shardId,
+ consumerArn);
+ result = false;
+ break;
+ } else if (subscriptionEvent.isSubscribeToShardEvent()) {
+ // Request for KDS to send the next record batch
+ subscription.requestRecord();
+
+ SubscribeToShardEvent event = subscriptionEvent.getSubscribeToShardEvent();
+ continuationSequenceNumber = event.continuationSequenceNumber();
+ if (!event.records().isEmpty()) {
+ eventConsumer.accept(event);
+ }
+ } else if (subscriptionEvent.isSubscriptionComplete()) {
+ // The subscription is complete, but the shard might not be, so we return incomplete
+ return false;
+ } else {
+ handleError(subscriptionEvent.getThrowable());
+ result = false;
+ break;
+ }
+ } while (continuationSequenceNumber != null);
+
+ subscription.cancelSubscription();
+ return result;
+ }
+
+ /**
+ * The {@link FanOutShardSubscription} subscribes to the events coming from KDS and adds them to
+ * the {@link BlockingQueue}. Backpressure is applied based on the maximum capacity of the
+ * queue. The {@link Subscriber} methods of this class are invoked by a thread from the {@link
+ * KinesisAsyncClient}.
+ */
+ private class FanOutShardSubscription implements Subscriber<SubscribeToShardEventStream> {
+
+ private Subscription subscription;
+
+ private volatile boolean cancelled = false;
+
+ private final CountDownLatch waitForSubscriptionLatch;
+
+ private FanOutShardSubscription(final CountDownLatch waitForSubscriptionLatch) {
+ this.waitForSubscriptionLatch = waitForSubscriptionLatch;
+ }
+
+ /** Flag to the producer that we are ready to receive more events. */
+ void requestRecord() {
+ if (!cancelled) {
+ LOG.debug(
+ "Requesting more records from EFO subscription - {} ({})",
+ shardId,
+ consumerArn);
+ subscription.request(1);
+ }
+ }
+
+ @Override
+ public void onSubscribe(Subscription subscription) {
+ this.subscription = subscription;
+ waitForSubscriptionLatch.countDown();
+ }
+
+ @Override
+ public void onNext(SubscribeToShardEventStream subscribeToShardEventStream) {
+ subscribeToShardEventStream.accept(
+ new SubscribeToShardResponseHandler.Visitor() {
+ @Override
+ public void visit(SubscribeToShardEvent event) {
+ enqueueEvent(new SubscriptionNextEvent(event));
+ }
+ });
+ }
+
+ @Override
+ public void onError(Throwable throwable) {
+ LOG.debug(
+ "Error occurred on EFO subscription: {} - ({}). {} ({})",
+ throwable.getClass().getName(),
+ throwable.getMessage(),
+ shardId,
+ consumerArn,
+ throwable);
+
+ SubscriptionErrorEvent subscriptionErrorEvent = new SubscriptionErrorEvent(throwable);
+ if (FanOutShardSubscriber.this.subscriptionErrorEvent.get() == null) {
+ FanOutShardSubscriber.this.subscriptionErrorEvent.set(subscriptionErrorEvent);
+ } else {
+ LOG.warn("Error already queued. Ignoring subsequent exception.", throwable);
+ }
+
+ // Cancel the subscription to signal the onNext to stop requesting data
+ cancelSubscription();
+
+ // If there is space in the queue, insert the error to wake up blocked thread
+ if (queue.isEmpty()) {
+ queue.offer(subscriptionErrorEvent);
+ }
+ }
+
+ @Override
+ public void onComplete() {
+ LOG.debug("EFO subscription complete - {} ({})", shardId, consumerArn);
+ enqueueEvent(new SubscriptionCompleteEvent());
+ }
+
+ private void cancelSubscription() {
+ if (cancelled) {
+ return;
+ }
+ cancelled = true;
+
+ if (subscription != null) {
+ subscription.cancel();
+ }
+ }
+
+ /**
+ * Adds the event to the queue blocking until complete.
+ *
+ * @param event the event to enqueue
+ */
+ private void enqueueEvent(final FanOutSubscriptionEvent event) {
+ if (cancelled) {
+ return;
+ }
+
+ try {
+ if (!queue.offer(event, queueWaitTimeout.toMillis(), TimeUnit.MILLISECONDS)) {
+ final String errorMessage =
+ "Timed out enqueuing event "
+ + event.getClass().getSimpleName()
+ + " - "
+ + shardId
+ + " ("
+ + consumerArn
+ + ")";
+ LOG.error(errorMessage);
+ onError(
+ new RecoverableFanOutSubscriberException(
+ new TimeoutException(errorMessage)));
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ /** An exception wrapper to indicate an error has been thrown from the networking stack. */
+ abstract static class FanOutSubscriberException extends Exception {
+
+ private static final long serialVersionUID = -3899472233945299730L;
+
+ public FanOutSubscriberException(Throwable cause) {
+ super(cause);
+ }
+ }
+
+ /**
+ * An exception wrapper to indicate a retryable error has been thrown from the networking stack.
+ * Retryable errors are subject to the Subscribe to Shard retry policy. If the configured number
+ * of retries are exceeded the application will terminate.
+ */
+ static class RetryableFanOutSubscriberException extends FanOutSubscriberException {
+
+ private static final long serialVersionUID = -2967281117554404883L;
+
+ public RetryableFanOutSubscriberException(Throwable cause) {
+ super(cause);
+ }
+ }
+
+ /**
+ * An exception wrapper to indicate a recoverable error has been thrown from the networking
+ * stack. Recoverable errors are not counted in the retry policy.
+ */
+ static class RecoverableFanOutSubscriberException extends FanOutSubscriberException {
+
+ private static final long serialVersionUID = -3223347557038294482L;
+
+ public RecoverableFanOutSubscriberException(Throwable cause) {
+ super(cause);
+ }
+ }
+
+ /** An exception wrapper to indicate the subscriber has been interrupted. */
+ static class FanOutSubscriberInterruptedException extends FanOutSubscriberException {
+
+ private static final long serialVersionUID = -2783477408630427189L;
+
+ public FanOutSubscriberInterruptedException(Throwable cause) {
+ super(cause);
+ }
+ }
+
+ /**
+ * An interface used to pass messages between {@link FanOutShardSubscription} and {@link
+ * FanOutShardSubscriber} via the {@link BlockingQueue}.
+ */
+ private interface FanOutSubscriptionEvent {
+
+ default boolean isSubscribeToShardEvent() {
+ return false;
+ }
+
+ default boolean isSubscriptionComplete() {
+ return false;
+ }
+
+ default SubscribeToShardEvent getSubscribeToShardEvent() {
+ throw new UnsupportedOperationException(
+ "This event does not support getSubscribeToShardEvent()");
+ }
+
+ default Throwable getThrowable() {
+ throw new UnsupportedOperationException("This event does not support getThrowable()");
+ }
+ }
+
+ /** Indicates that an EFO subscription has completed/expired. */
+ private static class SubscriptionCompleteEvent implements FanOutSubscriptionEvent {
+
+ @Override
+ public boolean isSubscriptionComplete() {
+ return true;
+ }
+ }
+
+ /** Poison pill, indicates that an error occurred while consuming from KDS. */
+ private static class SubscriptionErrorEvent implements FanOutSubscriptionEvent {
+ private final Throwable throwable;
+
+ private SubscriptionErrorEvent(Throwable throwable) {
+ this.throwable = throwable;
+ }
+
+ @Override
+ public Throwable getThrowable() {
+ return throwable;
+ }
+ }
+
+ /** A wrapper to pass the next {@link SubscribeToShardEvent} between threads. */
+ private static class SubscriptionNextEvent implements FanOutSubscriptionEvent {
+ private final SubscribeToShardEvent subscribeToShardEvent;
+
+ private SubscriptionNextEvent(SubscribeToShardEvent subscribeToShardEvent) {
+ this.subscribeToShardEvent = subscribeToShardEvent;
+ }
+
+ @Override
+ public boolean isSubscribeToShardEvent() {
+ return true;
+ }
+
+ @Override
+ public SubscribeToShardEvent getSubscribeToShardEvent() {
+ return subscribeToShardEvent;
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/StreamConsumerRegistrar.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/StreamConsumerRegistrar.java
new file mode 100644
index 0000000..9bf5e18
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/StreamConsumerRegistrar.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.fanout;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisException.FlinkKinesisTimeoutException;
+import org.apache.flink.streaming.connectors.kinesis.proxy.FullJitterBackoff;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyV2Interface;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse;
+import software.amazon.awssdk.services.kinesis.model.ResourceInUseException;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.EFORegistrationType.LAZY;
+import static software.amazon.awssdk.services.kinesis.model.ConsumerStatus.ACTIVE;
+import static software.amazon.awssdk.services.kinesis.model.ConsumerStatus.DELETING;
+
+/**
+ * Responsible for registering and deregistering EFO stream consumers. Will block until consumers
+ * are ready.
+ */
+@Internal
+public class StreamConsumerRegistrar {
+
+ private static final Logger LOG = LoggerFactory.getLogger(StreamConsumerRegistrar.class);
+
+ private final KinesisProxyV2Interface kinesisProxyV2Interface;
+
+ private final FanOutRecordPublisherConfiguration configuration;
+
+ private final FullJitterBackoff backoff;
+
+ public StreamConsumerRegistrar(
+ final KinesisProxyV2Interface kinesisProxyV2Interface,
+ final FanOutRecordPublisherConfiguration configuration,
+ final FullJitterBackoff backoff) {
+ this.kinesisProxyV2Interface = Preconditions.checkNotNull(kinesisProxyV2Interface);
+ this.configuration = Preconditions.checkNotNull(configuration);
+ this.backoff = Preconditions.checkNotNull(backoff);
+ }
+
+ /**
+ * Register a stream consumer with the given name against the given stream. Blocks until the
+ * consumer becomes active. If the stream consumer already exists, the ARN is returned.
+ *
+ * @param stream the stream to register the stream consumer against
+ * @param streamConsumerName the name of the new stream consumer
+ * @return the stream consumer ARN
+ * @throws ExecutionException
+ * @throws InterruptedException
+ */
+ public String registerStreamConsumer(final String stream, final String streamConsumerName)
+ throws ExecutionException, InterruptedException {
+ LOG.debug("Registering stream consumer - {}::{}", stream, streamConsumerName);
+
+ int attempt = 1;
+
+ if (configuration.getEfoRegistrationType() == LAZY) {
+ registrationBackoff(configuration, backoff, attempt++);
+ }
+
+ DescribeStreamSummaryResponse describeStreamSummaryResponse =
+ kinesisProxyV2Interface.describeStreamSummary(stream);
+ String streamArn = describeStreamSummaryResponse.streamDescriptionSummary().streamARN();
+
+ LOG.debug("Found stream ARN - {}", streamArn);
+
+ Optional<DescribeStreamConsumerResponse> describeStreamConsumerResponse =
+ describeStreamConsumer(streamArn, streamConsumerName);
+
+ if (!describeStreamConsumerResponse.isPresent()) {
+ invokeIgnoringResourceInUse(
+ () ->
+ kinesisProxyV2Interface.registerStreamConsumer(
+ streamArn, streamConsumerName));
+ }
+
+ String streamConsumerArn =
+ waitForConsumerToBecomeActive(
+ describeStreamConsumerResponse.orElse(null),
+ streamArn,
+ streamConsumerName,
+ attempt);
+
+ LOG.debug("Using stream consumer - {}", streamConsumerArn);
+
+ return streamConsumerArn;
+ }
+
+ /**
+ * Deregister the stream consumer with the given ARN. Blocks until the consumer is deleted.
+ *
+ * @param stream the stream in which to deregister the consumer
+ * @throws ExecutionException
+ * @throws InterruptedException
+ */
+ public void deregisterStreamConsumer(final String stream)
+ throws InterruptedException, ExecutionException {
+ LOG.debug("Deregistering stream consumer - {}", stream);
+
+ int attempt = 1;
+ String streamConsumerArn = getStreamConsumerArn(stream);
+
+ deregistrationBackoff(configuration, backoff, attempt++);
+
+ Optional<DescribeStreamConsumerResponse> response =
+ describeStreamConsumer(streamConsumerArn);
+ if (response.isPresent()
+ && response.get().consumerDescription().consumerStatus() != DELETING) {
+ invokeIgnoringResourceInUse(
+ () -> kinesisProxyV2Interface.deregisterStreamConsumer(streamConsumerArn));
+ }
+
+ waitForConsumerToDeregister(response.orElse(null), streamConsumerArn, attempt);
+
+ LOG.debug("Deregistered stream consumer - {}", streamConsumerArn);
+ }
+
+ /** Destroy any open resources used by the factory. */
+ public void close() {
+ kinesisProxyV2Interface.close();
+ }
+
+ @VisibleForTesting
+ void registrationBackoff(
+ final FanOutRecordPublisherConfiguration configuration,
+ final FullJitterBackoff backoff,
+ int attempt)
+ throws InterruptedException {
+ long backoffMillis =
+ backoff.calculateFullJitterBackoff(
+ configuration.getRegisterStreamBaseBackoffMillis(),
+ configuration.getRegisterStreamMaxBackoffMillis(),
+ configuration.getRegisterStreamExpConstant(),
+ attempt);
+
+ backoff.sleep(backoffMillis);
+ }
+
+ @VisibleForTesting
+ void deregistrationBackoff(
+ final FanOutRecordPublisherConfiguration configuration,
+ final FullJitterBackoff backoff,
+ int attempt)
+ throws InterruptedException {
+ long backoffMillis =
+ backoff.calculateFullJitterBackoff(
+ configuration.getDeregisterStreamBaseBackoffMillis(),
+ configuration.getDeregisterStreamMaxBackoffMillis(),
+ configuration.getDeregisterStreamExpConstant(),
+ attempt);
+
+ backoff.sleep(backoffMillis);
+ }
+
+ private String waitForConsumerToBecomeActive(
+ @Nullable final DescribeStreamConsumerResponse describeStreamConsumerResponse,
+ final String streamArn,
+ final String streamConsumerName,
+ final int initialAttempt)
+ throws InterruptedException, ExecutionException {
+ int attempt = initialAttempt;
+
+ Instant start = Instant.now();
+ Duration timeout = configuration.getRegisterStreamConsumerTimeout();
+
+ DescribeStreamConsumerResponse response = describeStreamConsumerResponse;
+ while (response == null || response.consumerDescription().consumerStatus() != ACTIVE) {
+ LOG.debug(
+ "Waiting for stream consumer to become active, attempt {} - {} on {}",
+ attempt,
+ streamConsumerName,
+ streamArn);
+ registrationBackoff(configuration, backoff, attempt++);
+ response =
+ kinesisProxyV2Interface.describeStreamConsumer(streamArn, streamConsumerName);
+
+ if (Duration.between(start, Instant.now()).compareTo(timeout) > 0) {
+ throw new FlinkKinesisTimeoutException(
+ "Timeout waiting for stream consumer to become active: "
+ + streamConsumerName
+ + " on "
+ + streamArn);
+ }
+ }
+
+ return response.consumerDescription().consumerARN();
+ }
+
+ private void waitForConsumerToDeregister(
+ @Nullable final DescribeStreamConsumerResponse describeStreamConsumerResponse,
+ final String streamConsumerArn,
+ final int initialAttempt)
+ throws InterruptedException, ExecutionException {
+ int attempt = initialAttempt;
+
+ Instant start = Instant.now();
+ Duration timeout = configuration.getDeregisterStreamConsumerTimeout();
+
+ Optional<DescribeStreamConsumerResponse> response =
+ Optional.ofNullable(describeStreamConsumerResponse);
+ while (response.isPresent()
+ && response.get().consumerDescription().consumerStatus() != DELETING) {
+ LOG.debug(
+ "Waiting for stream consumer to deregister, attempt {} - {}",
+ attempt,
+ streamConsumerArn);
+ deregistrationBackoff(configuration, backoff, attempt++);
+ response = describeStreamConsumer(streamConsumerArn);
+
+ if (Duration.between(start, Instant.now()).compareTo(timeout) > 0) {
+ throw new FlinkKinesisTimeoutException(
+ "Timeout waiting for stream consumer to deregister: " + streamConsumerArn);
+ }
+ }
+ }
+
+ private Optional<DescribeStreamConsumerResponse> describeStreamConsumer(
+ final String streamArn, final String streamConsumerName)
+ throws InterruptedException, ExecutionException {
+ return describeStreamConsumer(
+ () ->
+ kinesisProxyV2Interface.describeStreamConsumer(
+ streamArn, streamConsumerName));
+ }
+
+ private Optional<DescribeStreamConsumerResponse> describeStreamConsumer(
+ final String streamConsumerArn) throws InterruptedException, ExecutionException {
+ return describeStreamConsumer(
+ () -> kinesisProxyV2Interface.describeStreamConsumer(streamConsumerArn));
+ }
+
+ private Optional<DescribeStreamConsumerResponse> describeStreamConsumer(
+ final ResponseSupplier<DescribeStreamConsumerResponse> responseSupplier)
+ throws InterruptedException, ExecutionException {
+ DescribeStreamConsumerResponse response;
+
+ try {
+ response = responseSupplier.get();
+ } catch (ExecutionException ex) {
+ if (isResourceNotFound(ex)) {
+ return Optional.empty();
+ }
+
+ throw ex;
+ }
+
+ return Optional.ofNullable(response);
+ }
+
+ private <T> void invokeIgnoringResourceInUse(final ResponseSupplier<T> responseSupplier)
+ throws InterruptedException, ExecutionException {
+ try {
+ responseSupplier.get();
+ } catch (ExecutionException ex) {
+ if (isResourceInUse(ex)) {
+ // The stream consumer may have been created since we performed the describe
+ return;
+ }
+
+ throw ex;
+ }
+ }
+
+ private boolean isResourceNotFound(final ExecutionException ex) {
+ return ex.getCause() instanceof ResourceNotFoundException;
+ }
+
+ private boolean isResourceInUse(final ExecutionException ex) {
+ return ex.getCause() instanceof ResourceInUseException;
+ }
+
+ private String getStreamConsumerArn(final String stream) {
+ Optional<String> streamConsumerArn = configuration.getStreamConsumerArn(stream);
+ if (!streamConsumerArn.isPresent()) {
+ throw new IllegalArgumentException(
+ "Stream consumer ARN not found for stream: " + stream);
+ }
+
+ return streamConsumerArn.get();
+ }
+
+ private interface ResponseSupplier<T> {
+ T get() throws ExecutionException, InterruptedException;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
new file mode 100644
index 0000000..2de4655
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+/**
+ * An adaptive record publisher to add a dynamic batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts
+ * to balance quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {
+ // AWS Kinesis has a read limit of 2 MB/sec
+ // https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
+ private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
+
+ private int lastRecordBatchSize = 0;
+
+ private long lastRecordBatchSizeInBytes = 0;
+
+ private long processingStartTimeNanos = System.nanoTime();
+
+ private int maxNumberOfRecordsPerFetch;
+
+ private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+ AdaptivePollingRecordPublisher(
+ final StartingPosition startingPosition,
+ final StreamShardHandle subscribedShard,
+ final PollingRecordPublisherMetricsReporter metricsReporter,
+ final KinesisProxyInterface kinesisProxy,
+ final int maxNumberOfRecordsPerFetch,
+ final long fetchIntervalMillis)
+ throws InterruptedException {
+ super(
+ startingPosition,
+ subscribedShard,
+ metricsReporter,
+ kinesisProxy,
+ maxNumberOfRecordsPerFetch,
+ fetchIntervalMillis);
+ this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+ this.metricsReporter = metricsReporter;
+ }
+
+ @Override
+ public RecordPublisherRunResult run(final RecordBatchConsumer consumer)
+ throws InterruptedException {
+ final RecordPublisherRunResult result =
+ super.run(
+ batch -> {
+ SequenceNumber latestSequenceNumber = consumer.accept(batch);
+ lastRecordBatchSize = batch.getDeaggregatedRecordSize();
+ lastRecordBatchSizeInBytes = batch.getTotalSizeInBytes();
+ return latestSequenceNumber;
+ },
+ maxNumberOfRecordsPerFetch);
+
+ long endTimeNanos = System.nanoTime();
+ long runLoopTimeNanos = endTimeNanos - processingStartTimeNanos;
+
+ maxNumberOfRecordsPerFetch =
+ adaptRecordsToRead(
+ runLoopTimeNanos,
+ lastRecordBatchSize,
+ lastRecordBatchSizeInBytes,
+ maxNumberOfRecordsPerFetch);
+
+ processingStartTimeNanos = endTimeNanos;
+
+ return result;
+ }
+
+ /**
+ * Calculates how many records to read each time through the loop based on a target throughput
+ * and the measured frequenecy of the loop.
+ *
+ * @param runLoopTimeNanos The total time of one pass through the loop
+ * @param numRecords The number of records of the last read operation
+ * @param recordBatchSizeBytes The total batch size of the last read operation
+ * @param maxNumberOfRecordsPerFetch The current maxNumberOfRecordsPerFetch
+ */
+ private int adaptRecordsToRead(
+ long runLoopTimeNanos,
+ int numRecords,
+ long recordBatchSizeBytes,
+ int maxNumberOfRecordsPerFetch) {
+ if (numRecords != 0 && runLoopTimeNanos != 0) {
+ long averageRecordSizeBytes = recordBatchSizeBytes / numRecords;
+ // Adjust number of records to fetch from the shard depending on current average record
+ // size
+ // to optimize 2 Mb / sec read limits
+ double loopFrequencyHz = 1000000000.0d / runLoopTimeNanos;
+ double bytesPerRead = KINESIS_SHARD_BYTES_PER_SECOND_LIMIT / loopFrequencyHz;
+ maxNumberOfRecordsPerFetch = (int) (bytesPerRead / averageRecordSizeBytes);
+ // Ensure the value is greater than 0 and not more than 10000L
+ maxNumberOfRecordsPerFetch =
+ Math.max(
+ 1,
+ Math.min(
+ maxNumberOfRecordsPerFetch,
+ ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX));
+
+ // Set metrics
+ metricsReporter.setLoopFrequencyHz(loopFrequencyHz);
+ metricsReporter.setBytesPerRead(bytesPerRead);
+ }
+ return maxNumberOfRecordsPerFetch;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
new file mode 100644
index 0000000..70e48a4
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.util.Preconditions;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+ private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+ private final KinesisProxyInterface kinesisProxy;
+
+ private final StreamShardHandle subscribedShard;
+
+ private String nextShardItr;
+
+ private StartingPosition nextStartingPosition;
+
+ private final int maxNumberOfRecordsPerFetch;
+
+ private final long fetchIntervalMillis;
+
+ private long processingStartTimeNanos = System.nanoTime();
+
+ /**
+ * A Polling implementation of {@link RecordPublisher} that polls kinesis for records. The
+ * following KDS services are used: GetRecords and GetShardIterator.
+ *
+ * @param startingPosition the position in the stream to start consuming from
+ * @param subscribedShard the shard in which to consume from
+ * @param metricsReporter a metric reporter used to output metrics
+ * @param kinesisProxy the proxy used to communicate with kinesis
+ * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+ * @param fetchIntervalMillis the target interval between each GetRecords invocation
+ */
+ PollingRecordPublisher(
+ final StartingPosition startingPosition,
+ final StreamShardHandle subscribedShard,
+ final PollingRecordPublisherMetricsReporter metricsReporter,
+ final KinesisProxyInterface kinesisProxy,
+ final int maxNumberOfRecordsPerFetch,
+ final long fetchIntervalMillis)
+ throws InterruptedException {
+ this.nextStartingPosition = Preconditions.checkNotNull(startingPosition);
+ this.subscribedShard = Preconditions.checkNotNull(subscribedShard);
+ this.metricsReporter = Preconditions.checkNotNull(metricsReporter);
+ this.kinesisProxy = Preconditions.checkNotNull(kinesisProxy);
+ this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+ this.fetchIntervalMillis = fetchIntervalMillis;
+
+ Preconditions.checkArgument(fetchIntervalMillis >= 0);
+ Preconditions.checkArgument(maxNumberOfRecordsPerFetch > 0);
+
+ this.nextShardItr = getShardIterator();
+ }
+
+ @Override
+ public RecordPublisherRunResult run(final RecordBatchConsumer consumer)
+ throws InterruptedException {
+ return run(consumer, maxNumberOfRecordsPerFetch);
+ }
+
+ public RecordPublisherRunResult run(final RecordBatchConsumer consumer, int maxNumberOfRecords)
+ throws InterruptedException {
+ if (nextShardItr == null) {
+ return COMPLETE;
+ }
+
+ metricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecords);
+
+ GetRecordsResult result = getRecords(nextShardItr, maxNumberOfRecords);
+
+ RecordBatch recordBatch =
+ new RecordBatch(
+ result.getRecords(), subscribedShard, result.getMillisBehindLatest());
+ SequenceNumber latestSequenceNumber = consumer.accept(recordBatch);
+
+ nextStartingPosition = getNextStartingPosition(latestSequenceNumber);
+ nextShardItr = result.getNextShardIterator();
+
+ long adjustmentEndTimeNanos =
+ adjustRunLoopFrequency(processingStartTimeNanos, System.nanoTime());
+ long runLoopTimeNanos = adjustmentEndTimeNanos - processingStartTimeNanos;
+
+ processingStartTimeNanos = adjustmentEndTimeNanos;
+ metricsReporter.setRunLoopTimeNanos(runLoopTimeNanos);
+
+ return nextShardItr == null ? COMPLETE : INCOMPLETE;
+ }
+
+ private StartingPosition getNextStartingPosition(final SequenceNumber latestSequenceNumber) {
+ // When consuming from a timestamp sentinel/AT_TIMESTAMP ShardIteratorType.
+ // If the first RecordBatch is empty, then the latestSequenceNumber would be the timestamp
+ // sentinel.
+ // This is because we have not yet received any real sequence numbers on this shard.
+ // In this condition we should retry from the previous starting position (AT_TIMESTAMP).
+ if (SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get().equals(latestSequenceNumber)) {
+ Preconditions.checkState(nextStartingPosition.getShardIteratorType() == AT_TIMESTAMP);
+ return nextStartingPosition;
+ } else {
+ return StartingPosition.continueFromSequenceNumber(latestSequenceNumber);
+ }
+ }
+
+ /**
+ * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+ * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+ * such occasions. The returned shard iterator within the successful {@link GetRecordsResult}
+ * should be used for the next call to this method.
+ *
+ * <p>Note: it is important that this method is not called again before all the records from the
+ * last result have been fully collected with {@code
+ * ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise {@code
+ * ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated
+ * record, leading to incorrect shard iteration if the iterator had to be refreshed.
+ *
+ * @param shardItr shard iterator to use
+ * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+ * @return get records result
+ */
+ private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords)
+ throws InterruptedException {
+ GetRecordsResult getRecordsResult = null;
+ while (getRecordsResult == null) {
+ try {
+ getRecordsResult = kinesisProxy.getRecords(shardItr, maxNumberOfRecords);
+ } catch (ExpiredIteratorException | InterruptedException eiEx) {
+ LOG.warn(
+ "Encountered an unexpected expired iterator {} for shard {};"
+ + " refreshing the iterator ...",
+ shardItr,
+ subscribedShard);
+
+ shardItr = getShardIterator();
+
+ // sleep for the fetch interval before the next getRecords attempt with the
+ // refreshed iterator
+ if (fetchIntervalMillis != 0) {
+ Thread.sleep(fetchIntervalMillis);
+ }
+ }
+ }
+ return getRecordsResult;
+ }
+
+ /**
+ * Returns a shard iterator for the given {@link SequenceNumber}.
+ *
+ * @return shard iterator
+ */
+ @Nullable
+ private String getShardIterator() throws InterruptedException {
+ return kinesisProxy.getShardIterator(
+ subscribedShard,
+ nextStartingPosition.getShardIteratorType().toString(),
+ nextStartingPosition.getStartingMarker());
+ }
+
+ /**
+ * Adjusts loop timing to match target frequency if specified.
+ *
+ * @param processingStartTimeNanos The start time of the run loop "work"
+ * @param processingEndTimeNanos The end time of the run loop "work"
+ * @return The System.nanoTime() after the sleep (if any)
+ * @throws InterruptedException
+ */
+ private long adjustRunLoopFrequency(long processingStartTimeNanos, long processingEndTimeNanos)
+ throws InterruptedException {
+ long endTimeNanos = processingEndTimeNanos;
+ if (fetchIntervalMillis != 0) {
+ long processingTimeNanos = processingEndTimeNanos - processingStartTimeNanos;
+ long sleepTimeMillis = fetchIntervalMillis - (processingTimeNanos / 1_000_000);
+ if (sleepTimeMillis > 0) {
+ Thread.sleep(sleepTimeMillis);
+ endTimeNanos = System.nanoTime();
+ metricsReporter.setSleepTimeMillis(sleepTimeMillis);
+ }
+ }
+ return endTimeNanos;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherConfiguration.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherConfiguration.java
new file mode 100644
index 0000000..871aaa2
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherConfiguration.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+
+import java.util.Properties;
+
+/** A configuration class for {@link PollingRecordPublisher} instantiated from a properties map. */
+@Internal
+public class PollingRecordPublisherConfiguration {
+
+ private final boolean adaptiveReads;
+
+ private final int maxNumberOfRecordsPerFetch;
+
+ private final long fetchIntervalMillis;
+
+ public PollingRecordPublisherConfiguration(final Properties consumerConfig) {
+ this.maxNumberOfRecordsPerFetch =
+ Integer.parseInt(
+ consumerConfig.getProperty(
+ ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
+ Integer.toString(
+ ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
+
+ this.fetchIntervalMillis =
+ Long.parseLong(
+ consumerConfig.getProperty(
+ ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
+
+ this.adaptiveReads =
+ Boolean.parseBoolean(
+ consumerConfig.getProperty(
+ ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
+ Boolean.toString(
+ ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
+ }
+
+ public boolean isAdaptiveReads() {
+ return adaptiveReads;
+ }
+
+ public int getMaxNumberOfRecordsPerFetch() {
+ return maxNumberOfRecordsPerFetch;
+ }
+
+ public long getFetchIntervalMillis() {
+ return fetchIntervalMillis;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherFactory.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherFactory.java
new file mode 100644
index 0000000..4880d35
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisherFactory.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher.FlinkKinesisProxyFactory;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisherFactory;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Properties;
+
+/** A {@link RecordPublisher} factory used to create instances of {@link PollingRecordPublisher}. */
+@Internal
+public class PollingRecordPublisherFactory implements RecordPublisherFactory {
+
+ private final FlinkKinesisProxyFactory kinesisProxyFactory;
+
+ public PollingRecordPublisherFactory(final FlinkKinesisProxyFactory kinesisProxyFactory) {
+ this.kinesisProxyFactory = kinesisProxyFactory;
+ }
+
+ /**
+ * Create a {@link PollingRecordPublisher}. An {@link AdaptivePollingRecordPublisher} will be
+ * created should adaptive reads be enabled in the configuration.
+ *
+ * @param startingPosition the position in the shard to start consuming records from
+ * @param consumerConfig the consumer configuration properties
+ * @param metricGroup the metric group to report metrics to
+ * @param streamShardHandle the shard this consumer is subscribed to
+ * @return a {@link PollingRecordPublisher}
+ */
+ @Override
+ public PollingRecordPublisher create(
+ final StartingPosition startingPosition,
+ final Properties consumerConfig,
+ final MetricGroup metricGroup,
+ final StreamShardHandle streamShardHandle)
+ throws InterruptedException {
+ Preconditions.checkNotNull(startingPosition);
+ Preconditions.checkNotNull(consumerConfig);
+ Preconditions.checkNotNull(metricGroup);
+ Preconditions.checkNotNull(streamShardHandle);
+
+ final PollingRecordPublisherConfiguration configuration =
+ new PollingRecordPublisherConfiguration(consumerConfig);
+ final PollingRecordPublisherMetricsReporter metricsReporter =
+ new PollingRecordPublisherMetricsReporter(metricGroup);
+ final KinesisProxyInterface kinesisProxy = kinesisProxyFactory.create(consumerConfig);
+
+ if (configuration.isAdaptiveReads()) {
+ return new AdaptivePollingRecordPublisher(
+ startingPosition,
+ streamShardHandle,
+ metricsReporter,
+ kinesisProxy,
+ configuration.getMaxNumberOfRecordsPerFetch(),
+ configuration.getFetchIntervalMillis());
+ } else {
+ return new PollingRecordPublisher(
+ startingPosition,
+ streamShardHandle,
+ metricsReporter,
+ kinesisProxy,
+ configuration.getMaxNumberOfRecordsPerFetch(),
+ configuration.getFetchIntervalMillis());
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/KinesisConsumerMetricConstants.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/KinesisConsumerMetricConstants.java
new file mode 100644
index 0000000..5b7135a
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/KinesisConsumerMetricConstants.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.metrics;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A collection of consumer metric related constant names.
+ *
+ * <p>The names must not be changed, as that would break backwards compatibility for the consumer
+ * metrics.
+ */
+@Internal
+public class KinesisConsumerMetricConstants {
+
+ public static final String KINESIS_CONSUMER_METRICS_GROUP = "KinesisConsumer";
+
+ public static final String STREAM_METRICS_GROUP = "stream";
+ public static final String SHARD_METRICS_GROUP = "shardId";
+
+ public static final String MILLIS_BEHIND_LATEST_GAUGE = "millisBehindLatest";
+ public static final String SLEEP_TIME_MILLIS = "sleepTimeMillis";
+ public static final String MAX_RECORDS_PER_FETCH = "maxNumberOfRecordsPerFetch";
+ public static final String NUM_AGGREGATED_RECORDS_PER_FETCH =
+ "numberOfAggregatedRecordsPerFetch";
+ public static final String NUM_DEAGGREGATED_RECORDS_PER_FETCH =
+ "numberOfDeaggregatedRecordsPerFetch";
+ public static final String AVG_RECORD_SIZE_BYTES = "averageRecordSizeBytes";
+ public static final String RUNTIME_LOOP_NANOS = "runLoopTimeNanos";
+ public static final String LOOP_FREQUENCY_HZ = "loopFrequencyHz";
+ public static final String BYTES_PER_READ = "bytesRequestedPerFetch";
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/PollingRecordPublisherMetricsReporter.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/PollingRecordPublisherMetricsReporter.java
new file mode 100644
index 0000000..c03534b
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/PollingRecordPublisherMetricsReporter.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling.PollingRecordPublisher;
+
+/** A container for {@link PollingRecordPublisher}s to report metric values. */
+@Internal
+public class PollingRecordPublisherMetricsReporter {
+
+ private volatile double loopFrequencyHz = 0.0;
+ private volatile double bytesPerRead = 0.0;
+ private volatile long runLoopTimeNanos = 0L;
+ private volatile long sleepTimeMillis = 0L;
+ private volatile int maxNumberOfRecordsPerFetch = 0;
+
+ public PollingRecordPublisherMetricsReporter(final MetricGroup metricGroup) {
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.MAX_RECORDS_PER_FETCH,
+ this::getMaxNumberOfRecordsPerFetch);
+ metricGroup.gauge(KinesisConsumerMetricConstants.BYTES_PER_READ, this::getBytesPerRead);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.RUNTIME_LOOP_NANOS, this::getRunLoopTimeNanos);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.LOOP_FREQUENCY_HZ, this::getLoopFrequencyHz);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.SLEEP_TIME_MILLIS, this::getSleepTimeMillis);
+ }
+
+ public double getLoopFrequencyHz() {
+ return loopFrequencyHz;
+ }
+
+ public void setLoopFrequencyHz(double loopFrequencyHz) {
+ this.loopFrequencyHz = loopFrequencyHz;
+ }
+
+ public double getBytesPerRead() {
+ return bytesPerRead;
+ }
+
+ public void setBytesPerRead(double bytesPerRead) {
+ this.bytesPerRead = bytesPerRead;
+ }
+
+ public long getRunLoopTimeNanos() {
+ return runLoopTimeNanos;
+ }
+
+ public void setRunLoopTimeNanos(long runLoopTimeNanos) {
+ this.runLoopTimeNanos = runLoopTimeNanos;
+ }
+
+ public long getSleepTimeMillis() {
+ return sleepTimeMillis;
+ }
+
+ public void setSleepTimeMillis(long sleepTimeMillis) {
+ this.sleepTimeMillis = sleepTimeMillis;
+ }
+
+ public int getMaxNumberOfRecordsPerFetch() {
+ return maxNumberOfRecordsPerFetch;
+ }
+
+ public void setMaxNumberOfRecordsPerFetch(int maxNumberOfRecordsPerFetch) {
+ this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/ShardConsumerMetricsReporter.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/ShardConsumerMetricsReporter.java
new file mode 100644
index 0000000..d8fdf99
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/metrics/ShardConsumerMetricsReporter.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.metrics;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer;
+
+/** A container for {@link ShardConsumer}s to report metric values. */
+@Internal
+public class ShardConsumerMetricsReporter {
+
+ private final MetricGroup metricGroup;
+
+ private volatile long millisBehindLatest = -1;
+ private volatile long averageRecordSizeBytes = 0L;
+ private volatile int numberOfAggregatedRecords = 0;
+ private volatile int numberOfDeaggregatedRecords = 0;
+
+ public ShardConsumerMetricsReporter(final MetricGroup metricGroup) {
+ this.metricGroup = metricGroup;
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.MILLIS_BEHIND_LATEST_GAUGE,
+ this::getMillisBehindLatest);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.NUM_AGGREGATED_RECORDS_PER_FETCH,
+ this::getNumberOfAggregatedRecords);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.NUM_DEAGGREGATED_RECORDS_PER_FETCH,
+ this::getNumberOfDeaggregatedRecords);
+ metricGroup.gauge(
+ KinesisConsumerMetricConstants.AVG_RECORD_SIZE_BYTES,
+ this::getAverageRecordSizeBytes);
+ }
+
+ public long getMillisBehindLatest() {
+ return millisBehindLatest;
+ }
+
+ public void setMillisBehindLatest(long millisBehindLatest) {
+ this.millisBehindLatest = millisBehindLatest;
+ }
+
+ public long getAverageRecordSizeBytes() {
+ return averageRecordSizeBytes;
+ }
+
+ public void setAverageRecordSizeBytes(long averageRecordSizeBytes) {
+ this.averageRecordSizeBytes = averageRecordSizeBytes;
+ }
+
+ public int getNumberOfAggregatedRecords() {
+ return numberOfAggregatedRecords;
+ }
+
+ public void setNumberOfAggregatedRecords(int numberOfAggregatedRecords) {
+ this.numberOfAggregatedRecords = numberOfAggregatedRecords;
+ }
+
+ public int getNumberOfDeaggregatedRecords() {
+ return numberOfDeaggregatedRecords;
+ }
+
+ public void setNumberOfDeaggregatedRecords(int numberOfDeaggregatedRecords) {
+ this.numberOfDeaggregatedRecords = numberOfDeaggregatedRecords;
+ }
+
+ public void unregister() {
+ if (this.metricGroup instanceof AbstractMetricGroup) {
+ ((AbstractMetricGroup) this.metricGroup).close();
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java
new file mode 100644
index 0000000..ba6e99d
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import com.amazonaws.services.kinesis.model.Shard;
+
+/** DynamoDB streams shard handle format and utilities. */
+public class DynamoDBStreamsShardHandle extends StreamShardHandle {
+ public static final String SHARDID_PREFIX = "shardId-";
+ public static final int SHARDID_PREFIX_LEN = SHARDID_PREFIX.length();
+
+ public DynamoDBStreamsShardHandle(String streamName, Shard shard) {
+ super(streamName, shard);
+ }
+
+ public static int compareShardIds(String firstShardId, String secondShardId) {
+ if (!isValidShardId(firstShardId)) {
+ throw new IllegalArgumentException(
+ String.format("The first shard id %s has invalid format.", firstShardId));
+ } else if (!isValidShardId(secondShardId)) {
+ throw new IllegalArgumentException(
+ String.format("The second shard id %s has invalid format.", secondShardId));
+ }
+
+ return firstShardId
+ .substring(SHARDID_PREFIX_LEN)
+ .compareTo(secondShardId.substring(SHARDID_PREFIX_LEN));
+ }
+
+ /**
+ * Dynamodb streams shard ID is a char string ranging from 28 characters to 65 characters. (See
+ * https://docs.aws.amazon.com/amazondynamodb/latest/APIReference/API_streams_Shard.html)
+ *
+ * <p>The shardId observed usually takes the format of: "shardId-00000001536805703746-69688cb1",
+ * where "shardId-" is a prefix, followed by a 20-digit timestamp string and 0-36 or more
+ * characters, separated by '-'. Following this format, it is expected the child shards created
+ * during a re-sharding event have shardIds bigger than their parents.
+ *
+ * @param shardId shard Id
+ * @return boolean indicate if the given shard Id is valid
+ */
+ public static boolean isValidShardId(String shardId) {
+ return shardId == null ? false : shardId.matches("^shardId-\\d{20}-{0,1}\\w{0,36}");
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
new file mode 100644
index 0000000..65197af
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+
+import com.amazonaws.services.kinesis.model.Shard;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A legacy serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper
+ * class around the information provided along with {@link
+ * com.amazonaws.services.kinesis.model.Shard}.
+ *
+ * @deprecated Will be remove in a future version in favor of {@link StreamShardHandle}.
+ */
+@Deprecated
+@Internal
+public class KinesisStreamShard implements Serializable {
+
+ private static final long serialVersionUID = -6004217801761077536L;
+
+ private final String streamName;
+ private final Shard shard;
+
+ private final int cachedHash;
+
+ /**
+ * Create a new KinesisStreamShard.
+ *
+ * @param streamName the name of the Kinesis stream that this shard belongs to
+ * @param shard the actual AWS Shard instance that will be wrapped within this
+ * KinesisStreamShard
+ */
+ public KinesisStreamShard(String streamName, Shard shard) {
+ this.streamName = checkNotNull(streamName);
+ this.shard = checkNotNull(shard);
+
+ // since our description of Kinesis Streams shards can be fully defined with the stream name
+ // and shard id,
+ // our hash doesn't need to use hash code of Amazon's description of Shards, which uses
+ // other info for calculation
+ int hash = 17;
+ hash = 37 * hash + streamName.hashCode();
+ hash = 37 * hash + shard.getShardId().hashCode();
+ this.cachedHash = hash;
+ }
+
+ public String getStreamName() {
+ return streamName;
+ }
+
+ public boolean isClosed() {
+ return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null);
+ }
+
+ public Shard getShard() {
+ return shard;
+ }
+
+ @Override
+ public String toString() {
+ return "KinesisStreamShard{"
+ + "streamName='"
+ + streamName
+ + "'"
+ + ", shard='"
+ + shard.toString()
+ + "'}";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof KinesisStreamShard)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ KinesisStreamShard other = (KinesisStreamShard) obj;
+
+ return streamName.equals(other.getStreamName()) && shard.equals(other.getShard());
+ }
+
+ @Override
+ public int hashCode() {
+ return cachedHash;
+ }
+
+ /**
+ * Utility function to convert {@link KinesisStreamShard} into the new {@link
+ * StreamShardMetadata} model.
+ *
+ * @param kinesisStreamShard the {@link KinesisStreamShard} to be converted
+ * @return the converted {@link StreamShardMetadata}
+ */
+ public static StreamShardMetadata convertToStreamShardMetadata(
+ KinesisStreamShard kinesisStreamShard) {
+ StreamShardMetadata streamShardMetadata = new StreamShardMetadata();
+
+ streamShardMetadata.setStreamName(kinesisStreamShard.getStreamName());
+ streamShardMetadata.setShardId(kinesisStreamShard.getShard().getShardId());
+ streamShardMetadata.setParentShardId(kinesisStreamShard.getShard().getParentShardId());
+ streamShardMetadata.setAdjacentParentShardId(
+ kinesisStreamShard.getShard().getAdjacentParentShardId());
+
+ if (kinesisStreamShard.getShard().getHashKeyRange() != null) {
+ streamShardMetadata.setStartingHashKey(
+ kinesisStreamShard.getShard().getHashKeyRange().getStartingHashKey());
+ streamShardMetadata.setEndingHashKey(
+ kinesisStreamShard.getShard().getHashKeyRange().getEndingHashKey());
+ }
+
+ if (kinesisStreamShard.getShard().getSequenceNumberRange() != null) {
+ streamShardMetadata.setStartingSequenceNumber(
+ kinesisStreamShard
+ .getShard()
+ .getSequenceNumberRange()
+ .getStartingSequenceNumber());
+ streamShardMetadata.setEndingSequenceNumber(
+ kinesisStreamShard
+ .getShard()
+ .getSequenceNumberRange()
+ .getEndingSequenceNumber());
+ }
+
+ return streamShardMetadata;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
new file mode 100644
index 0000000..e4731fc
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+import com.amazonaws.services.kinesis.model.Shard;
+
+/**
+ * A wrapper class that bundles a {@link StreamShardHandle} with its last processed sequence number.
+ */
+@Internal
+public class KinesisStreamShardState {
+
+ /** A handle object that wraps the actual {@link Shard} instance and stream name. */
+ private StreamShardHandle streamShardHandle;
+
+ /** The checkpointed state for each Kinesis stream shard. */
+ private StreamShardMetadata streamShardMetadata;
+
+ private SequenceNumber lastProcessedSequenceNum;
+
+ public KinesisStreamShardState(
+ StreamShardMetadata streamShardMetadata,
+ StreamShardHandle streamShardHandle,
+ SequenceNumber lastProcessedSequenceNum) {
+
+ this.streamShardMetadata = Preconditions.checkNotNull(streamShardMetadata);
+ this.streamShardHandle = Preconditions.checkNotNull(streamShardHandle);
+ this.lastProcessedSequenceNum = Preconditions.checkNotNull(lastProcessedSequenceNum);
+ }
+
+ public StreamShardMetadata getStreamShardMetadata() {
+ return this.streamShardMetadata;
+ }
+
+ public StreamShardHandle getStreamShardHandle() {
+ return this.streamShardHandle;
+ }
+
+ public SequenceNumber getLastProcessedSequenceNum() {
+ return this.lastProcessedSequenceNum;
+ }
+
+ public void setLastProcessedSequenceNum(SequenceNumber update) {
+ this.lastProcessedSequenceNum = update;
+ }
+
+ @Override
+ public String toString() {
+ return "KinesisStreamShardState{"
+ + "streamShardMetadata='"
+ + streamShardMetadata.toString()
+ + "'"
+ + ", streamShardHandle='"
+ + streamShardHandle.toString()
+ + "'"
+ + ", lastProcessedSequenceNumber='"
+ + lastProcessedSequenceNum.toString()
+ + "'}";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof KinesisStreamShardState)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ KinesisStreamShardState other = (KinesisStreamShardState) obj;
+
+ return streamShardMetadata.equals(other.getStreamShardMetadata())
+ && streamShardHandle.equals(other.getStreamShardHandle())
+ && lastProcessedSequenceNum.equals(other.getLastProcessedSequenceNum());
+ }
+
+ @Override
+ public int hashCode() {
+ return 37
+ * (streamShardMetadata.hashCode()
+ + streamShardHandle.hashCode()
+ + lastProcessedSequenceNum.hashCode());
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
new file mode 100644
index 0000000..f86d838
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+
+/**
+ * Special flag values for sequence numbers in shards to indicate special positions. The value is
+ * initially set by {@link FlinkKinesisConsumer} when {@link KinesisDataFetcher}s are created. The
+ * KinesisDataFetchers will use this value to determine how to retrieve the starting shard iterator
+ * from AWS Kinesis.
+ */
+@Internal
+public enum SentinelSequenceNumber {
+
+ /**
+ * Flag value for shard's sequence numbers to indicate that the shard should start to be read
+ * from the latest incoming records.
+ */
+ SENTINEL_LATEST_SEQUENCE_NUM(new SequenceNumber("LATEST_SEQUENCE_NUM")),
+
+ /**
+ * Flag value for shard's sequence numbers to indicate that the shard should start to be read
+ * from the earliest records that haven't expired yet.
+ */
+ SENTINEL_EARLIEST_SEQUENCE_NUM(new SequenceNumber("EARLIEST_SEQUENCE_NUM")),
+
+ /**
+ * Flag value for shard's sequence numbers to indicate that the shard should start to be read
+ * from the specified timestamp.
+ */
+ SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM(new SequenceNumber("AT_TIMESTAMP_SEQUENCE_NUM")),
+
+ /**
+ * Flag value to indicate that we have already read the last record of this shard (Note: Kinesis
+ * shards that have been closed due to a split or merge will have an ending data record).
+ */
+ SENTINEL_SHARD_ENDING_SEQUENCE_NUM(new SequenceNumber("SHARD_ENDING_SEQUENCE_NUM"));
+
+ private SequenceNumber sentinel;
+
+ SentinelSequenceNumber(SequenceNumber sentinel) {
+ this.sentinel = sentinel;
+ }
+
+ public SequenceNumber get() {
+ return sentinel;
+ }
+
+ /** Returns {@code true} if the given {@link SequenceNumber} is a sentinel. */
+ public static boolean isSentinelSequenceNumber(SequenceNumber candidateSequenceNumber) {
+ for (SentinelSequenceNumber sentinel : values()) {
+ if (candidateSequenceNumber.equals(sentinel.get())) {
+ return true;
+ }
+ }
+ return false;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
new file mode 100644
index 0000000..16f6185
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SequenceNumber.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A serializable representation of a Kinesis record's sequence number. It has two fields: the main
+ * sequence number, and also a subsequence number. If this {@link SequenceNumber} is referring to an
+ * aggregated Kinesis record, the subsequence number will be a non-negative value representing the
+ * order of the sub-record within the aggregation.
+ */
+@Internal
+public class SequenceNumber implements Serializable {
+
+ private static final long serialVersionUID = 876972197938972667L;
+
+ private static final String DELIMITER = "-";
+
+ private final String sequenceNumber;
+ private final long subSequenceNumber;
+
+ private final int cachedHash;
+
+ /**
+ * Create a new instance for a non-aggregated Kinesis record without a subsequence number.
+ *
+ * @param sequenceNumber the sequence number
+ */
+ public SequenceNumber(String sequenceNumber) {
+ this(sequenceNumber, -1);
+ }
+
+ /**
+ * Create a new instance, with the specified sequence number and subsequence number. To
+ * represent the sequence number for a non-aggregated Kinesis record, the subsequence number
+ * should be -1. Otherwise, give a non-negative sequence number to represent an aggregated
+ * Kinesis record.
+ *
+ * @param sequenceNumber the sequence number
+ * @param subSequenceNumber the subsequence number (-1 to represent non-aggregated Kinesis
+ * records)
+ */
+ public SequenceNumber(String sequenceNumber, long subSequenceNumber) {
+ this.sequenceNumber = checkNotNull(sequenceNumber);
+ this.subSequenceNumber = subSequenceNumber;
+
+ this.cachedHash =
+ 37 * (sequenceNumber.hashCode() + Long.valueOf(subSequenceNumber).hashCode());
+ }
+
+ public boolean isAggregated() {
+ return subSequenceNumber >= 0;
+ }
+
+ public String getSequenceNumber() {
+ return sequenceNumber;
+ }
+
+ public long getSubSequenceNumber() {
+ return subSequenceNumber;
+ }
+
+ @Override
+ public String toString() {
+ if (isAggregated()) {
+ return sequenceNumber + DELIMITER + subSequenceNumber;
+ } else {
+ return sequenceNumber;
+ }
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof SequenceNumber)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ SequenceNumber other = (SequenceNumber) obj;
+
+ return sequenceNumber.equals(other.getSequenceNumber())
+ && (subSequenceNumber == other.getSubSequenceNumber());
+ }
+
+ @Override
+ public int hashCode() {
+ return cachedHash;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
new file mode 100644
index 0000000..2cb9b26
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+
+import javax.annotation.Nullable;
+
+import java.util.Date;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.TRIM_HORIZON;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.isSentinelSequenceNumber;
+
+/** The position in which to start consuming from a stream. */
+@Internal
+public class StartingPosition {
+
+ private final ShardIteratorType shardIteratorType;
+
+ private final Object startingMarker;
+
+ private StartingPosition(
+ final ShardIteratorType shardIteratorType, @Nullable final Object startingMarker) {
+ this.shardIteratorType = shardIteratorType;
+ this.startingMarker = startingMarker;
+ }
+
+ public ShardIteratorType getShardIteratorType() {
+ return shardIteratorType;
+ }
+
+ @Nullable
+ public Object getStartingMarker() {
+ return startingMarker;
+ }
+
+ public static StartingPosition fromTimestamp(final Date date) {
+ return new StartingPosition(AT_TIMESTAMP, date);
+ }
+
+ /**
+ * Returns the starting position for the next record to consume from the given sequence number.
+ * The difference between {@code restartFromSequenceNumber()} and {@code
+ * continueFromSequenceNumber()} is that for {@code restartFromSequenceNumber()} aggregated
+ * records are reread to support subsequence failure.
+ *
+ * @param sequenceNumber the last successful sequence number, or sentinel marker
+ * @return the start position in which to consume from
+ */
+ public static StartingPosition continueFromSequenceNumber(final SequenceNumber sequenceNumber) {
+ return fromSequenceNumber(sequenceNumber, false);
+ }
+
+ /**
+ * Returns the starting position to restart record consumption from the given sequence number
+ * after failure. The difference between {@code restartFromSequenceNumber()} and {@code
+ * continueFromSequenceNumber()} is that for {@code restartFromSequenceNumber()} aggregated
+ * records are reread to support subsequence failure.
+ *
+ * @param sequenceNumber the last successful sequence number, or sentinel marker
+ * @return the start position in which to consume from
+ */
+ public static StartingPosition restartFromSequenceNumber(final SequenceNumber sequenceNumber) {
+ return fromSequenceNumber(sequenceNumber, true);
+ }
+
+ private static StartingPosition fromSequenceNumber(
+ final SequenceNumber sequenceNumber, final boolean restart) {
+ if (isSentinelSequenceNumber(sequenceNumber)) {
+ return new StartingPosition(fromSentinelSequenceNumber(sequenceNumber), null);
+ } else {
+ // we will be starting from an actual sequence number (due to restore from failure).
+ return new StartingPosition(
+ getShardIteratorType(sequenceNumber, restart),
+ sequenceNumber.getSequenceNumber());
+ }
+ }
+
+ private static ShardIteratorType getShardIteratorType(
+ final SequenceNumber sequenceNumber, final boolean restart) {
+ return restart && sequenceNumber.isAggregated()
+ ? AT_SEQUENCE_NUMBER
+ : AFTER_SEQUENCE_NUMBER;
+ }
+
+ private static ShardIteratorType fromSentinelSequenceNumber(
+ final SequenceNumber sequenceNumber) {
+ if (sequenceNumber.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
+ return LATEST;
+ } else if (sequenceNumber.equals(
+ SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
+ return TRIM_HORIZON;
+ } else {
+ throw new IllegalArgumentException("Unexpected sentinel type: " + sequenceNumber);
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardHandle.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardHandle.java
new file mode 100644
index 0000000..92997e6
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardHandle.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+
+import com.amazonaws.services.kinesis.model.Shard;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A wrapper class around the information provided along with streamName and {@link
+ * com.amazonaws.services.kinesis.model.Shard}, with some extra utility methods to determine whether
+ * or not a shard is closed and whether or not the shard is a result of parent shard splits or
+ * merges.
+ */
+@Internal
+public class StreamShardHandle {
+
+ private final String streamName;
+ private final Shard shard;
+
+ private final int cachedHash;
+
+ /**
+ * Create a new StreamShardHandle.
+ *
+ * @param streamName the name of the Kinesis stream that this shard belongs to
+ * @param shard the actual AWS Shard instance that will be wrapped within this StreamShardHandle
+ */
+ public StreamShardHandle(String streamName, Shard shard) {
+ this.streamName = checkNotNull(streamName);
+ this.shard = checkNotNull(shard);
+
+ // since our description of Kinesis Streams shards can be fully defined with the stream name
+ // and shard id,
+ // our hash doesn't need to use hash code of Amazon's description of Shards, which uses
+ // other info for calculation
+ int hash = 17;
+ hash = 37 * hash + streamName.hashCode();
+ hash = 37 * hash + shard.getShardId().hashCode();
+ this.cachedHash = hash;
+ }
+
+ public String getStreamName() {
+ return streamName;
+ }
+
+ public boolean isClosed() {
+ return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null);
+ }
+
+ public Shard getShard() {
+ return shard;
+ }
+
+ @Override
+ public String toString() {
+ return "StreamShardHandle{"
+ + "streamName='"
+ + streamName
+ + "'"
+ + ", shard='"
+ + shard.toString()
+ + "'}";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof StreamShardHandle)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ StreamShardHandle other = (StreamShardHandle) obj;
+
+ return streamName.equals(other.getStreamName()) && shard.equals(other.getShard());
+ }
+
+ @Override
+ public int hashCode() {
+ return cachedHash;
+ }
+
+ /**
+ * Utility function to compare two shard ids.
+ *
+ * @param firstShardId first shard id to compare
+ * @param secondShardId second shard id to compare
+ * @return a value less than 0 if the first shard id is smaller than the second shard id, or a
+ * value larger than 0 the first shard is larger than the second shard id, or 0 if they are
+ * equal
+ */
+ public static int compareShardIds(String firstShardId, String secondShardId) {
+ return firstShardId.compareTo(secondShardId);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardMetadata.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardMetadata.java
new file mode 100644
index 0000000..23aa89a
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StreamShardMetadata.java
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.model;
+
+import org.apache.flink.annotation.Internal;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper class
+ * around the information disintegrated from {@link com.amazonaws.services.kinesis.model.Shard} and
+ * its nested classes. The disintegration is required to avoid being locked-in to a specific AWS SDK
+ * version in order to maintain the consumer's state backwards compatibility.
+ */
+@Internal
+public class StreamShardMetadata implements Serializable {
+
+ private static final long serialVersionUID = 5134869582298563604L;
+
+ private String streamName;
+ private String shardId;
+ private String parentShardId;
+ private String adjacentParentShardId;
+ private String startingHashKey;
+ private String endingHashKey;
+ private String startingSequenceNumber;
+ private String endingSequenceNumber;
+
+ public void setStreamName(String streamName) {
+ this.streamName = streamName;
+ }
+
+ public void setShardId(String shardId) {
+ this.shardId = shardId;
+ }
+
+ public void setParentShardId(String parentShardId) {
+ this.parentShardId = parentShardId;
+ }
+
+ public void setAdjacentParentShardId(String adjacentParentShardId) {
+ this.adjacentParentShardId = adjacentParentShardId;
+ }
+
+ public void setStartingHashKey(String startingHashKey) {
+ this.startingHashKey = startingHashKey;
+ }
+
+ public void setEndingHashKey(String endingHashKey) {
+ this.endingHashKey = endingHashKey;
+ }
+
+ public void setStartingSequenceNumber(String startingSequenceNumber) {
+ this.startingSequenceNumber = startingSequenceNumber;
+ }
+
+ public void setEndingSequenceNumber(String endingSequenceNumber) {
+ this.endingSequenceNumber = endingSequenceNumber;
+ }
+
+ public String getStreamName() {
+ return this.streamName;
+ }
+
+ public String getShardId() {
+ return this.shardId;
+ }
+
+ public String getParentShardId() {
+ return this.parentShardId;
+ }
+
+ public String getAdjacentParentShardId() {
+ return this.adjacentParentShardId;
+ }
+
+ public String getStartingHashKey() {
+ return this.startingHashKey;
+ }
+
+ public String getEndingHashKey() {
+ return this.endingHashKey;
+ }
+
+ public String getStartingSequenceNumber() {
+ return this.startingSequenceNumber;
+ }
+
+ public String getEndingSequenceNumber() {
+ return this.endingSequenceNumber;
+ }
+
+ @Override
+ public String toString() {
+ return "StreamShardMetadata{"
+ + "streamName='"
+ + streamName
+ + "'"
+ + ", shardId='"
+ + shardId
+ + "'"
+ + ", parentShardId='"
+ + parentShardId
+ + "'"
+ + ", adjacentParentShardId='"
+ + adjacentParentShardId
+ + "'"
+ + ", startingHashKey='"
+ + startingHashKey
+ + "'"
+ + ", endingHashKey='"
+ + endingHashKey
+ + "'"
+ + ", startingSequenceNumber='"
+ + startingSequenceNumber
+ + "'"
+ + ", endingSequenceNumber='"
+ + endingSequenceNumber
+ + "'}";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof StreamShardMetadata)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ StreamShardMetadata other = (StreamShardMetadata) obj;
+
+ return streamName.equals(other.getStreamName())
+ && shardId.equals(other.getShardId())
+ && Objects.equals(parentShardId, other.getParentShardId())
+ && Objects.equals(adjacentParentShardId, other.getAdjacentParentShardId())
+ && Objects.equals(startingHashKey, other.getStartingHashKey())
+ && Objects.equals(endingHashKey, other.getEndingHashKey())
+ && Objects.equals(startingSequenceNumber, other.getStartingSequenceNumber())
+ && Objects.equals(endingSequenceNumber, other.getEndingSequenceNumber());
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 17;
+
+ if (streamName != null) {
+ hash = 37 * hash + streamName.hashCode();
+ }
+ if (shardId != null) {
+ hash = 37 * hash + shardId.hashCode();
+ }
+ if (parentShardId != null) {
+ hash = 37 * hash + parentShardId.hashCode();
+ }
+ if (adjacentParentShardId != null) {
+ hash = 37 * hash + adjacentParentShardId.hashCode();
+ }
+ if (startingHashKey != null) {
+ hash = 37 * hash + startingHashKey.hashCode();
+ }
+ if (endingHashKey != null) {
+ hash = 37 * hash + endingHashKey.hashCode();
+ }
+ if (startingSequenceNumber != null) {
+ hash = 37 * hash + startingSequenceNumber.hashCode();
+ }
+ if (endingSequenceNumber != null) {
+ hash = 37 * hash + endingSequenceNumber.hashCode();
+ }
+
+ return hash;
+ }
+
+ /** An equivalence wrapper that only checks for the stream name and shard id for equality. */
+ public static class EquivalenceWrapper {
+
+ private final StreamShardMetadata shardMetadata;
+
+ public EquivalenceWrapper(StreamShardMetadata shardMetadata) {
+ this.shardMetadata = checkNotNull(shardMetadata);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof EquivalenceWrapper)) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ EquivalenceWrapper other = (EquivalenceWrapper) obj;
+
+ return shardMetadata.getStreamName().equals(other.shardMetadata.getStreamName())
+ && shardMetadata.getShardId().equals(other.shardMetadata.getShardId());
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 17;
+
+ if (shardMetadata.getStreamName() != null) {
+ hash = 37 * hash + shardMetadata.getStreamName().hashCode();
+ }
+ if (shardMetadata.getShardId() != null) {
+ hash = 37 * hash + shardMetadata.getShardId().hashCode();
+ }
+ return hash;
+ }
+
+ public StreamShardMetadata getShardMetadata() {
+ return shardMetadata;
+ }
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java
new file mode 100644
index 0000000..65c4035
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ClientConfigurationFactory;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.regions.RegionUtils;
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import com.amazonaws.services.kinesis.model.Shard;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_ENDPOINT;
+import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_REGION;
+import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.getCredentialsProvider;
+import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.setAwsClientConfigProperties;
+
+/** DynamoDB streams proxy: interface interacting with the DynamoDB streams. */
+public class DynamoDBStreamsProxy extends KinesisProxy {
+ private static final Logger LOG = LoggerFactory.getLogger(DynamoDBStreamsProxy.class);
+
+ /** Used for formatting Flink-specific user agent string when creating Kinesis client. */
+ private static final String USER_AGENT_FORMAT =
+ "Apache Flink %s (%s) DynamoDB Streams Connector";
+
+ protected DynamoDBStreamsProxy(Properties configProps) {
+ super(configProps);
+ }
+
+ /**
+ * Creates a DynamoDB streams proxy.
+ *
+ * @param configProps configuration properties
+ * @return the created DynamoDB streams proxy
+ */
+ public static KinesisProxyInterface create(Properties configProps) {
+ return new DynamoDBStreamsProxy(configProps);
+ }
+
+ /**
+ * Creates an AmazonDynamoDBStreamsAdapterClient. Uses it as the internal client interacting
+ * with the DynamoDB streams.
+ *
+ * @param configProps configuration properties
+ * @return an AWS DynamoDB streams adapter client
+ */
+ @Override
+ protected AmazonKinesis createKinesisClient(Properties configProps) {
+ ClientConfiguration awsClientConfig = new ClientConfigurationFactory().getConfig();
+ setAwsClientConfigProperties(awsClientConfig, configProps);
+
+ AWSCredentialsProvider credentials = getCredentialsProvider(configProps);
+ awsClientConfig.setUserAgentPrefix(
+ String.format(
+ USER_AGENT_FORMAT,
+ EnvironmentInformation.getVersion(),
+ EnvironmentInformation.getRevisionInformation().commitId));
+
+ AmazonDynamoDBStreamsAdapterClient adapterClient =
+ new AmazonDynamoDBStreamsAdapterClient(credentials, awsClientConfig);
+
+ if (configProps.containsKey(AWS_ENDPOINT)) {
+ adapterClient.setEndpoint(configProps.getProperty(AWS_ENDPOINT));
+ } else {
+ adapterClient.setRegion(RegionUtils.getRegion(configProps.getProperty(AWS_REGION)));
+ }
+
+ return adapterClient;
+ }
+
+ @Override
+ public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds)
+ throws InterruptedException {
+ GetShardListResult result = new GetShardListResult();
+
+ for (Map.Entry<String, String> streamNameWithLastSeenShardId :
+ streamNamesWithLastSeenShardIds.entrySet()) {
+ String stream = streamNameWithLastSeenShardId.getKey();
+ String lastSeenShardId = streamNameWithLastSeenShardId.getValue();
+ result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId));
+ }
+ return result;
+ }
+
+ private List<StreamShardHandle> getShardsOfStream(
+ String streamName, @Nullable String lastSeenShardId) throws InterruptedException {
+ List<StreamShardHandle> shardsOfStream = new ArrayList<>();
+
+ DescribeStreamResult describeStreamResult;
+ do {
+ describeStreamResult = describeStream(streamName, lastSeenShardId);
+ List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
+ for (Shard shard : shards) {
+ shardsOfStream.add(new StreamShardHandle(streamName, shard));
+ }
+
+ if (shards.size() != 0) {
+ lastSeenShardId = shards.get(shards.size() - 1).getShardId();
+ }
+ } while (describeStreamResult.getStreamDescription().isHasMoreShards());
+
+ return shardsOfStream;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/FullJitterBackoff.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/FullJitterBackoff.java
new file mode 100644
index 0000000..553e862
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/FullJitterBackoff.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.Random;
+
+/**
+ * Used to calculate full jitter backoff sleep durations.
+ *
+ * @see <a href="https://aws.amazon.com/blogs/architecture/exponential-backoff-and-jitter/">
+ * Exponential Backoff and Jitter </a>
+ */
+@Internal
+public class FullJitterBackoff {
+
+ /** Random seed used to calculate backoff jitter for Kinesis operations. */
+ private final Random seed = new Random();
+
+ /**
+ * Calculates the sleep time for full jitter based on the given parameters.
+ *
+ * @param baseMillis the base backoff time in milliseconds
+ * @param maxMillis the maximum backoff time in milliseconds
+ * @param power the power constant for exponential backoff
+ * @param attempt the attempt number
+ * @return the time to wait before trying again
+ */
+ public long calculateFullJitterBackoff(
+ long baseMillis, long maxMillis, double power, int attempt) {
+ long exponentialBackoff = (long) Math.min(maxMillis, baseMillis * Math.pow(power, attempt));
+ return (long) (seed.nextDouble() * exponentialBackoff);
+ }
+
+ /**
+ * Puts the current thread to sleep for the specified number of millis. Simply delegates to
+ * {@link Thread#sleep}.
+ *
+ * @param millisToSleep the number of milliseconds to sleep for
+ * @throws InterruptedException
+ */
+ public void sleep(long millisToSleep) throws InterruptedException {
+ Thread.sleep(millisToSleep);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
new file mode 100644
index 0000000..82f0536
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Basic model class to bundle the shards retrieved from Kinesis on a {@link
+ * KinesisProxyInterface#getShardList(Map)} call.
+ */
+@Internal
+public class GetShardListResult {
+
+ private final Map<String, LinkedList<StreamShardHandle>> streamsToRetrievedShardList =
+ new HashMap<>();
+
+ public void addRetrievedShardToStream(String stream, StreamShardHandle retrievedShard) {
+ if (!streamsToRetrievedShardList.containsKey(stream)) {
+ streamsToRetrievedShardList.put(stream, new LinkedList<StreamShardHandle>());
+ }
+ streamsToRetrievedShardList.get(stream).add(retrievedShard);
+ }
+
+ public void addRetrievedShardsToStream(String stream, List<StreamShardHandle> retrievedShards) {
+ if (retrievedShards.size() != 0) {
+ if (!streamsToRetrievedShardList.containsKey(stream)) {
+ streamsToRetrievedShardList.put(stream, new LinkedList<StreamShardHandle>());
+ }
+ streamsToRetrievedShardList.get(stream).addAll(retrievedShards);
+ }
+ }
+
+ public List<StreamShardHandle> getRetrievedShardListOfStream(String stream) {
+ if (!streamsToRetrievedShardList.containsKey(stream)) {
+ return null;
+ } else {
+ return streamsToRetrievedShardList.get(stream);
+ }
+ }
+
+ public StreamShardHandle getLastSeenShardOfStream(String stream) {
+ if (!streamsToRetrievedShardList.containsKey(stream)) {
+ return null;
+ } else {
+ return streamsToRetrievedShardList.get(stream).getLast();
+ }
+ }
+
+ public boolean hasRetrievedShards() {
+ return !streamsToRetrievedShardList.isEmpty();
+ }
+
+ public Set<String> getStreamsWithRetrievedShards() {
+ return streamsToRetrievedShardList.keySet();
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
new file mode 100644
index 0000000..458c029
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java
@@ -0,0 +1,654 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.util.ExceptionUtils;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ClientConfigurationFactory;
+import com.amazonaws.SdkClientException;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.model.DescribeStreamRequest;
+import com.amazonaws.services.kinesis.model.DescribeStreamResult;
+import com.amazonaws.services.kinesis.model.ExpiredNextTokenException;
+import com.amazonaws.services.kinesis.model.GetRecordsRequest;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import com.amazonaws.services.kinesis.model.GetShardIteratorRequest;
+import com.amazonaws.services.kinesis.model.GetShardIteratorResult;
+import com.amazonaws.services.kinesis.model.InvalidArgumentException;
+import com.amazonaws.services.kinesis.model.LimitExceededException;
+import com.amazonaws.services.kinesis.model.ListShardsRequest;
+import com.amazonaws.services.kinesis.model.ListShardsResult;
+import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.kinesis.model.ResourceInUseException;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import com.amazonaws.services.kinesis.model.StreamStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.EOFException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Kinesis proxy implementation - a utility class that is used as a proxy to make calls to AWS
+ * Kinesis for several functions, such as getting a list of shards and fetching a batch of data
+ * records starting from a specified record sequence number.
+ *
+ * <p>NOTE: In the AWS KCL library, there is a similar implementation - {@link
+ * com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy}. This implementation differs
+ * mainly in that we can make operations to arbitrary Kinesis streams, which is a needed
+ * functionality for the Flink Kinesis Connector since the consumer may simultaneously read from
+ * multiple Kinesis streams.
+ */
+@Internal
+public class KinesisProxy implements KinesisProxyInterface {
+
+ private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class);
+
+ /** Calculates full jitter backoff delays. */
+ private static final FullJitterBackoff BACKOFF = new FullJitterBackoff();
+
+ /** The actual Kinesis client from the AWS SDK that we will be using to make calls. */
+ private final AmazonKinesis kinesisClient;
+
+ // ------------------------------------------------------------------------
+ // listShards() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Base backoff millis for the list shards operation. */
+ private final long listShardsBaseBackoffMillis;
+
+ /** Maximum backoff millis for the list shards operation. */
+ private final long listShardsMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the list shards operation. */
+ private final double listShardsExpConstant;
+
+ /** Maximum retry attempts for the list shards operation. */
+ private final int listShardsMaxRetries;
+
+ // ------------------------------------------------------------------------
+ // getRecords() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Base backoff millis for the get records operation. */
+ private final long getRecordsBaseBackoffMillis;
+
+ /** Maximum backoff millis for the get records operation. */
+ private final long getRecordsMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the get records operation. */
+ private final double getRecordsExpConstant;
+
+ /** Maximum retry attempts for the get records operation. */
+ private final int getRecordsMaxRetries;
+
+ // ------------------------------------------------------------------------
+ // getShardIterator() related performance settings
+ // ------------------------------------------------------------------------
+
+ /** Base backoff millis for the get shard iterator operation. */
+ private final long getShardIteratorBaseBackoffMillis;
+
+ /** Maximum backoff millis for the get shard iterator operation. */
+ private final long getShardIteratorMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the get shard iterator operation. */
+ private final double getShardIteratorExpConstant;
+
+ /** Maximum retry attempts for the get shard iterator operation. */
+ private final int getShardIteratorMaxRetries;
+
+ /** Backoff millis for the describe stream operation. */
+ private final long describeStreamBaseBackoffMillis;
+
+ /** Maximum backoff millis for the describe stream operation. */
+ private final long describeStreamMaxBackoffMillis;
+
+ /** Exponential backoff power constant for the describe stream operation. */
+ private final double describeStreamExpConstant;
+
+ /**
+ * Create a new KinesisProxy based on the supplied configuration properties.
+ *
+ * @param configProps configuration properties containing AWS credential and AWS region info
+ */
+ protected KinesisProxy(Properties configProps) {
+ checkNotNull(configProps);
+ KinesisConfigUtil.backfillConsumerKeys(configProps);
+
+ this.kinesisClient = createKinesisClient(configProps);
+
+ this.listShardsBaseBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.LIST_SHARDS_BACKOFF_BASE,
+ Long.toString(
+ ConsumerConfigConstants.DEFAULT_LIST_SHARDS_BACKOFF_BASE)));
+ this.listShardsMaxBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.LIST_SHARDS_BACKOFF_MAX,
+ Long.toString(
+ ConsumerConfigConstants.DEFAULT_LIST_SHARDS_BACKOFF_MAX)));
+ this.listShardsExpConstant =
+ Double.parseDouble(
+ configProps.getProperty(
+ ConsumerConfigConstants.LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT,
+ Double.toString(
+ ConsumerConfigConstants
+ .DEFAULT_LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT)));
+ this.listShardsMaxRetries =
+ Integer.parseInt(
+ configProps.getProperty(
+ ConsumerConfigConstants.LIST_SHARDS_RETRIES,
+ Long.toString(
+ ConsumerConfigConstants.DEFAULT_LIST_SHARDS_RETRIES)));
+ this.describeStreamBaseBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE)));
+ this.describeStreamMaxBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX)));
+ this.describeStreamExpConstant =
+ Double.parseDouble(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT,
+ Double.toString(
+ ConsumerConfigConstants
+ .DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT)));
+ this.getRecordsBaseBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE)));
+ this.getRecordsMaxBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_MAX,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX)));
+ this.getRecordsExpConstant =
+ Double.parseDouble(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT,
+ Double.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT)));
+ this.getRecordsMaxRetries =
+ Integer.parseInt(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETRECORDS_RETRIES,
+ Long.toString(
+ ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_RETRIES)));
+
+ this.getShardIteratorBaseBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_BASE,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE)));
+ this.getShardIteratorMaxBackoffMillis =
+ Long.parseLong(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETITERATOR_BACKOFF_MAX,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX)));
+ this.getShardIteratorExpConstant =
+ Double.parseDouble(
+ configProps.getProperty(
+ ConsumerConfigConstants
+ .SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT,
+ Double.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT)));
+ this.getShardIteratorMaxRetries =
+ Integer.parseInt(
+ configProps.getProperty(
+ ConsumerConfigConstants.SHARD_GETITERATOR_RETRIES,
+ Long.toString(
+ ConsumerConfigConstants
+ .DEFAULT_SHARD_GETITERATOR_RETRIES)));
+ }
+
+ /**
+ * Create the Kinesis client, using the provided configuration properties and default {@link
+ * ClientConfiguration}. Derived classes can override this method to customize the client
+ * configuration.
+ */
+ protected AmazonKinesis createKinesisClient(Properties configProps) {
+
+ ClientConfiguration awsClientConfig = new ClientConfigurationFactory().getConfig();
+ AWSUtil.setAwsClientConfigProperties(awsClientConfig, configProps);
+ return AWSUtil.createKinesisClient(configProps, awsClientConfig);
+ }
+
+ /**
+ * Creates a Kinesis proxy.
+ *
+ * @param configProps configuration properties
+ * @return the created kinesis proxy
+ */
+ public static KinesisProxyInterface create(Properties configProps) {
+ return new KinesisProxy(configProps);
+ }
+
+ @Override
+ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet)
+ throws InterruptedException {
+ final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
+ getRecordsRequest.setShardIterator(shardIterator);
+ getRecordsRequest.setLimit(maxRecordsToGet);
+
+ GetRecordsResult getRecordsResult = null;
+
+ int retryCount = 0;
+ while (retryCount <= getRecordsMaxRetries && getRecordsResult == null) {
+ try {
+ getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
+ } catch (SdkClientException ex) {
+ if (isRecoverableSdkClientException(ex)) {
+ long backoffMillis =
+ BACKOFF.calculateFullJitterBackoff(
+ getRecordsBaseBackoffMillis,
+ getRecordsMaxBackoffMillis,
+ getRecordsExpConstant,
+ retryCount++);
+ LOG.warn(
+ "Got recoverable SdkClientException. Backing off for "
+ + backoffMillis
+ + " millis ("
+ + ex.getClass().getName()
+ + ": "
+ + ex.getMessage()
+ + ")");
+ BACKOFF.sleep(backoffMillis);
+ } else {
+ throw ex;
+ }
+ }
+ }
+
+ if (getRecordsResult == null) {
+ throw new RuntimeException(
+ "Retries exceeded for getRecords operation - all "
+ + getRecordsMaxRetries
+ + " retry attempts failed.");
+ }
+
+ return getRecordsResult;
+ }
+
+ @Override
+ public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds)
+ throws InterruptedException {
+ GetShardListResult result = new GetShardListResult();
+
+ for (Map.Entry<String, String> streamNameWithLastSeenShardId :
+ streamNamesWithLastSeenShardIds.entrySet()) {
+ String stream = streamNameWithLastSeenShardId.getKey();
+ String lastSeenShardId = streamNameWithLastSeenShardId.getValue();
+ result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId));
+ }
+ return result;
+ }
+
+ @Override
+ public String getShardIterator(
+ StreamShardHandle shard, String shardIteratorType, @Nullable Object startingMarker)
+ throws InterruptedException {
+ GetShardIteratorRequest getShardIteratorRequest =
+ new GetShardIteratorRequest()
+ .withStreamName(shard.getStreamName())
+ .withShardId(shard.getShard().getShardId())
+ .withShardIteratorType(shardIteratorType);
+
+ switch (ShardIteratorType.fromValue(shardIteratorType)) {
+ case TRIM_HORIZON:
+ case LATEST:
+ break;
+ case AT_TIMESTAMP:
+ if (startingMarker instanceof Date) {
+ getShardIteratorRequest.setTimestamp((Date) startingMarker);
+ } else {
+ throw new IllegalArgumentException(
+ "Invalid object given for GetShardIteratorRequest() when ShardIteratorType is AT_TIMESTAMP. Must be a Date object.");
+ }
+ break;
+ case AT_SEQUENCE_NUMBER:
+ case AFTER_SEQUENCE_NUMBER:
+ if (startingMarker instanceof String) {
+ getShardIteratorRequest.setStartingSequenceNumber((String) startingMarker);
+ } else {
+ throw new IllegalArgumentException(
+ "Invalid object given for GetShardIteratorRequest() when ShardIteratorType is AT_SEQUENCE_NUMBER or AFTER_SEQUENCE_NUMBER. Must be a String.");
+ }
+ }
+ return getShardIterator(getShardIteratorRequest);
+ }
+
+ private String getShardIterator(GetShardIteratorRequest getShardIteratorRequest)
+ throws InterruptedException {
+ GetShardIteratorResult getShardIteratorResult = null;
+
+ int retryCount = 0;
+ while (retryCount <= getShardIteratorMaxRetries && getShardIteratorResult == null) {
+ try {
+ getShardIteratorResult = kinesisClient.getShardIterator(getShardIteratorRequest);
+ } catch (AmazonServiceException ex) {
+ if (isRecoverableException(ex)) {
+ long backoffMillis =
+ BACKOFF.calculateFullJitterBackoff(
+ getShardIteratorBaseBackoffMillis,
+ getShardIteratorMaxBackoffMillis,
+ getShardIteratorExpConstant,
+ retryCount++);
+ LOG.warn(
+ "Got recoverable AmazonServiceException. Backing off for "
+ + backoffMillis
+ + " millis ("
+ + ex.getClass().getName()
+ + ": "
+ + ex.getMessage()
+ + ")");
+ BACKOFF.sleep(backoffMillis);
+ } else {
+ throw ex;
+ }
+ }
+ }
+
+ if (getShardIteratorResult == null) {
+ throw new RuntimeException(
+ "Retries exceeded for getShardIterator operation - all "
+ + getShardIteratorMaxRetries
+ + " retry attempts failed.");
+ }
+ return getShardIteratorResult.getShardIterator();
+ }
+
+ /**
+ * Determines whether the exception is recoverable using exponential-backoff.
+ *
+ * @param ex Exception to inspect
+ * @return <code>true</code> if the exception can be recovered from, else <code>false</code>
+ */
+ protected boolean isRecoverableSdkClientException(SdkClientException ex) {
+ if (ex instanceof AmazonServiceException) {
+ return KinesisProxy.isRecoverableException((AmazonServiceException) ex);
+ } else if (isRecoverableConnectionException(ex)) {
+ return true;
+ }
+ // customizations may decide to retry other errors, such as read timeouts
+ return false;
+ }
+
+ private boolean isRecoverableConnectionException(SdkClientException ex) {
+ return ExceptionUtils.findThrowable(ex, SocketTimeoutException.class).isPresent()
+ || ExceptionUtils.findThrowable(ex, EOFException.class).isPresent();
+ }
+
+ /**
+ * Determines whether the exception is recoverable using exponential-backoff.
+ *
+ * @param ex Exception to inspect
+ * @return <code>true</code> if the exception can be recovered from, else <code>false</code>
+ */
+ protected static boolean isRecoverableException(AmazonServiceException ex) {
+ if (ex.getErrorType() == null) {
+ return false;
+ }
+
+ switch (ex.getErrorType()) {
+ case Client:
+ return ex instanceof ProvisionedThroughputExceededException;
+ case Service:
+ case Unknown:
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ private List<StreamShardHandle> getShardsOfStream(
+ String streamName, @Nullable String lastSeenShardId) throws InterruptedException {
+ List<StreamShardHandle> shardsOfStream = new ArrayList<>();
+
+ // List Shards returns just the first 1000 shard entries. In order to read the entire
+ // stream,
+ // we need to use the returned nextToken to get additional shards.
+ ListShardsResult listShardsResult;
+ String startShardToken = null;
+ do {
+ listShardsResult = listShards(streamName, lastSeenShardId, startShardToken);
+ if (listShardsResult == null) {
+ // In case we have exceptions while retrieving all shards, ensure that incomplete
+ // shard list is not returned.
+ // Hence clearing the incomplete shard list before returning it.
+ shardsOfStream.clear();
+ return shardsOfStream;
+ }
+ List<Shard> shards = listShardsResult.getShards();
+ for (Shard shard : shards) {
+ shardsOfStream.add(new StreamShardHandle(streamName, shard));
+ }
+ startShardToken = listShardsResult.getNextToken();
+ } while (startShardToken != null);
+
+ return shardsOfStream;
+ }
+
+ /**
+ * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis
+ * stream possess.
+ *
+ * <p>This method is using a "full jitter" approach described in AWS's article, <a
+ * href="https://www.awsarchitectureblog.com/2015/03/backoff.html">"Exponential Backoff and
+ * Jitter"</a>. This is necessary because concurrent calls will be made by all parallel
+ * subtask's fetcher. This jitter backoff approach will help distribute calls across the
+ * fetchers over time.
+ *
+ * @param streamName the stream to describe
+ * @param startShardId which shard to start with for this describe operation (earlier shard's
+ * infos will not appear in result)
+ * @return the result of the describe stream operation
+ */
+ private ListShardsResult listShards(
+ String streamName, @Nullable String startShardId, @Nullable String startNextToken)
+ throws InterruptedException {
+ final ListShardsRequest listShardsRequest = new ListShardsRequest();
+ if (startNextToken == null) {
+ listShardsRequest.setExclusiveStartShardId(startShardId);
+ listShardsRequest.setStreamName(streamName);
+ } else {
+ // Note the nextToken returned by AWS expires within 300 sec.
+ listShardsRequest.setNextToken(startNextToken);
+ }
+
+ ListShardsResult listShardsResults = null;
+
+ // Call ListShards, with full-jitter backoff (if we get LimitExceededException).
+ int retryCount = 0;
+ // List Shards returns just the first 1000 shard entries. Make sure that all entries
+ // are taken up.
+ while (retryCount <= listShardsMaxRetries
+ && listShardsResults == null) { // retry until we get a result
+ try {
+
+ listShardsResults = kinesisClient.listShards(listShardsRequest);
+ } catch (LimitExceededException le) {
+ long backoffMillis =
+ BACKOFF.calculateFullJitterBackoff(
+ listShardsBaseBackoffMillis,
+ listShardsMaxBackoffMillis,
+ listShardsExpConstant,
+ retryCount++);
+ LOG.warn(
+ "Got LimitExceededException when listing shards from stream "
+ + streamName
+ + ". Backing off for "
+ + backoffMillis
+ + " millis.");
+ BACKOFF.sleep(backoffMillis);
+ } catch (ResourceInUseException reInUse) {
+ if (LOG.isWarnEnabled()) {
+ // List Shards will throw an exception if stream in not in active state. Return
+ // and re-use previous state available.
+ LOG.info(
+ "The stream is currently not in active state. Reusing the older state "
+ + "for the time being");
+ break;
+ }
+ } catch (ResourceNotFoundException reNotFound) {
+ throw new RuntimeException(
+ "Stream not found. Error while getting shard list.", reNotFound);
+ } catch (InvalidArgumentException inArg) {
+ throw new RuntimeException("Invalid Arguments to listShards.", inArg);
+ } catch (ExpiredNextTokenException expiredToken) {
+ LOG.warn("List Shards has an expired token. Reusing the previous state.");
+ break;
+ } catch (SdkClientException ex) {
+ if (retryCount < listShardsMaxRetries && isRecoverableSdkClientException(ex)) {
+ long backoffMillis =
+ BACKOFF.calculateFullJitterBackoff(
+ listShardsBaseBackoffMillis,
+ listShardsMaxBackoffMillis,
+ listShardsExpConstant,
+ retryCount++);
+ LOG.warn(
+ "Got SdkClientException when listing shards from stream {}. Backing off for {} millis.",
+ streamName,
+ backoffMillis);
+ BACKOFF.sleep(backoffMillis);
+ } else {
+ // propagate if retries exceeded or not recoverable
+ // (otherwise would return null result and keep trying forever)
+ throw ex;
+ }
+ }
+ }
+
+ // Kinesalite (mock implementation of Kinesis) does not correctly exclude shards before
+ // the exclusive start shard id in the returned shards list; check if we need to remove
+ // these erroneously returned shards.
+ // Related issues:
+ // https://github.com/mhart/kinesalite/pull/77
+ // https://github.com/lyft/kinesalite/pull/4
+ if (startShardId != null && listShardsResults != null) {
+ List<Shard> shards = listShardsResults.getShards();
+ shards.removeIf(
+ shard ->
+ StreamShardHandle.compareShardIds(shard.getShardId(), startShardId)
+ <= 0);
+ }
+
+ return listShardsResults;
+ }
+
+ /**
+ * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis
+ * stream possess.
+ *
+ * <p>This method is using a "full jitter" approach described in AWS's article, <a
+ * href="https://www.awsarchitectureblog.com/2015/03/backoff.html">"Exponential Backoff and
+ * Jitter"</a>. This is necessary because concurrent calls will be made by all parallel
+ * subtask's fetcher. This jitter backoff approach will help distribute calls across the
+ * fetchers over time.
+ *
+ * @param streamName the stream to describe
+ * @param startShardId which shard to start with for this describe operation
+ * @return the result of the describe stream operation
+ */
+ protected DescribeStreamResult describeStream(String streamName, @Nullable String startShardId)
+ throws InterruptedException {
+ final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
+ describeStreamRequest.setStreamName(streamName);
+ describeStreamRequest.setExclusiveStartShardId(startShardId);
+
+ DescribeStreamResult describeStreamResult = null;
+
+ // Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
+ int attemptCount = 0;
+ while (describeStreamResult == null) { // retry until we get a result
+ try {
+ describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
+ } catch (LimitExceededException le) {
+ long backoffMillis =
+ BACKOFF.calculateFullJitterBackoff(
+ describeStreamBaseBackoffMillis,
+ describeStreamMaxBackoffMillis,
+ describeStreamExpConstant,
+ attemptCount++);
+ LOG.warn(
+ String.format(
+ "Got LimitExceededException when describing stream %s. "
+ + "Backing off for %d millis.",
+ streamName, backoffMillis));
+ BACKOFF.sleep(backoffMillis);
+ } catch (ResourceNotFoundException re) {
+ throw new RuntimeException("Error while getting stream details", re);
+ }
+ }
+
+ String streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
+ if (!(streamStatus.equals(StreamStatus.ACTIVE.toString())
+ || streamStatus.equals(StreamStatus.UPDATING.toString()))) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn(
+ String.format(
+ "The status of stream %s is %s ; result of the current "
+ + "describeStream operation will not contain any shard information.",
+ streamName, streamStatus));
+ }
+ }
+
+ return describeStreamResult;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
new file mode 100644
index 0000000..3728d72
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+
+import java.util.Map;
+
+/**
+ * Interface for a Kinesis proxy that operates on multiple Kinesis streams within the same AWS
+ * service region.
+ */
+@Internal
+public interface KinesisProxyInterface {
+
+ /**
+ * Get a shard iterator from the specified position in a shard. The retrieved shard iterator can
+ * be used in {@link KinesisProxyInterface#getRecords(String, int)}} to read data from the
+ * Kinesis shard.
+ *
+ * @param shard the shard to get the iterator
+ * @param shardIteratorType the iterator type, defining how the shard is to be iterated (one of:
+ * TRIM_HORIZON, LATEST, AT_TIMESTAMP, AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER)
+ * @param startingMarker should be {@code null} if shardIteratorType is TRIM_HORIZON or LATEST,
+ * should be a {@code Date} value if shardIteratorType is AT_TIMESTAMP, should be a {@code
+ * String} representing the sequence number if shardIteratorType is AT_SEQUENCE_NUMBER,
+ * AFTER_SEQUENCE_NUMBER
+ * @return shard iterator which can be used to read data from Kinesis
+ * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains
+ * that the operation has exceeded the rate limit; this exception will be thrown if the
+ * backoff is interrupted.
+ */
+ String getShardIterator(
+ StreamShardHandle shard, String shardIteratorType, Object startingMarker)
+ throws InterruptedException;
+
+ /**
+ * Get the next batch of data records using a specific shard iterator.
+ *
+ * @param shardIterator a shard iterator that encodes info about which shard to read and where
+ * to start reading
+ * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
+ * @return the batch of retrieved records, also with a shard iterator that can be used to get
+ * the next batch
+ * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains
+ * that the operation has exceeded the rate limit; this exception will be thrown if the
+ * backoff is interrupted.
+ */
+ GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet)
+ throws InterruptedException;
+
+ /**
+ * Get shard list of multiple Kinesis streams, ignoring the shards of each stream before a
+ * specified last seen shard id.
+ *
+ * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as
+ * value
+ * @return result of the shard list query
+ * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains
+ * that the operation has exceeded the rate limit; this exception will be thrown if the
+ * backoff is interrupted.
+ */
+ GetShardListResult getShardList(Map<String, String> streamNamesWithLastSeenShardIds)
+ throws InterruptedException;
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2.java
new file mode 100644
index 0000000..0677765
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutRecordPublisherConfiguration;
+import org.apache.flink.streaming.connectors.kinesis.util.AwsV2Util;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.DeregisterStreamConsumerRequest;
+import software.amazon.awssdk.services.kinesis.model.DeregisterStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerRequest;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse;
+import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerRequest;
+import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Kinesis proxy implementation using AWS SDK v2.x - a utility class that is used as a proxy to make
+ * calls to AWS Kinesis for several EFO (Enhanced Fan Out) functions, such as de-/registering stream
+ * consumers, subscribing to a shard and receiving records from a shard.
+ */
+@Internal
+public class KinesisProxyV2 implements KinesisProxyV2Interface {
+
+ private static final Logger LOG = LoggerFactory.getLogger(KinesisProxyV2.class);
+
+ /** An Asynchronous client used to communicate with AWS services. */
+ private final KinesisAsyncClient kinesisAsyncClient;
+
+ private final SdkAsyncHttpClient httpClient;
+
+ private final FanOutRecordPublisherConfiguration fanOutRecordPublisherConfiguration;
+
+ private final FullJitterBackoff backoff;
+
+ /**
+ * Create a new KinesisProxyV2.
+ *
+ * @param kinesisAsyncClient AWS SDK v2 Kinesis client used to communicate with AWS services
+ * @param httpClient the underlying HTTP client, reference required for close only
+ * @param fanOutRecordPublisherConfiguration the configuration for Fan Out features
+ * @param backoff the backoff utility used to introduce Full Jitter delays
+ */
+ public KinesisProxyV2(
+ final KinesisAsyncClient kinesisAsyncClient,
+ final SdkAsyncHttpClient httpClient,
+ final FanOutRecordPublisherConfiguration fanOutRecordPublisherConfiguration,
+ final FullJitterBackoff backoff) {
+ this.kinesisAsyncClient = Preconditions.checkNotNull(kinesisAsyncClient);
+ this.httpClient = httpClient;
+ this.fanOutRecordPublisherConfiguration = fanOutRecordPublisherConfiguration;
+ this.backoff = backoff;
+ }
+
+ @Override
+ public CompletableFuture<Void> subscribeToShard(
+ final SubscribeToShardRequest request,
+ final SubscribeToShardResponseHandler responseHandler) {
+ return kinesisAsyncClient.subscribeToShard(request, responseHandler);
+ }
+
+ @Override
+ public void close() {
+ kinesisAsyncClient.close();
+ httpClient.close();
+ }
+
+ @Override
+ public DescribeStreamSummaryResponse describeStreamSummary(String stream)
+ throws InterruptedException, ExecutionException {
+ DescribeStreamSummaryRequest describeStreamRequest =
+ DescribeStreamSummaryRequest.builder().streamName(stream).build();
+
+ return invokeWithRetryAndBackoff(
+ () -> kinesisAsyncClient.describeStreamSummary(describeStreamRequest).get(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamBaseBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamMaxBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamExpConstant(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamMaxRetries());
+ }
+
+ @Override
+ public DescribeStreamConsumerResponse describeStreamConsumer(
+ final String streamArn, final String consumerName)
+ throws InterruptedException, ExecutionException {
+ DescribeStreamConsumerRequest describeStreamConsumerRequest =
+ DescribeStreamConsumerRequest.builder()
+ .streamARN(streamArn)
+ .consumerName(consumerName)
+ .build();
+
+ return describeStreamConsumer(describeStreamConsumerRequest);
+ }
+
+ @Override
+ public DescribeStreamConsumerResponse describeStreamConsumer(final String streamConsumerArn)
+ throws InterruptedException, ExecutionException {
+ DescribeStreamConsumerRequest describeStreamConsumerRequest =
+ DescribeStreamConsumerRequest.builder().consumerARN(streamConsumerArn).build();
+
+ return describeStreamConsumer(describeStreamConsumerRequest);
+ }
+
+ private DescribeStreamConsumerResponse describeStreamConsumer(
+ final DescribeStreamConsumerRequest request)
+ throws InterruptedException, ExecutionException {
+ return invokeWithRetryAndBackoff(
+ () -> kinesisAsyncClient.describeStreamConsumer(request).get(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamConsumerBaseBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamConsumerMaxBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamConsumerExpConstant(),
+ fanOutRecordPublisherConfiguration.getDescribeStreamConsumerMaxRetries());
+ }
+
+ @Override
+ public RegisterStreamConsumerResponse registerStreamConsumer(
+ final String streamArn, final String consumerName)
+ throws InterruptedException, ExecutionException {
+ RegisterStreamConsumerRequest registerStreamConsumerRequest =
+ RegisterStreamConsumerRequest.builder()
+ .streamARN(streamArn)
+ .consumerName(consumerName)
+ .build();
+
+ return invokeWithRetryAndBackoff(
+ () ->
+ kinesisAsyncClient
+ .registerStreamConsumer(registerStreamConsumerRequest)
+ .get(),
+ fanOutRecordPublisherConfiguration.getRegisterStreamBaseBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getRegisterStreamMaxBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getRegisterStreamExpConstant(),
+ fanOutRecordPublisherConfiguration.getRegisterStreamMaxRetries());
+ }
+
+ @Override
+ public DeregisterStreamConsumerResponse deregisterStreamConsumer(final String consumerArn)
+ throws InterruptedException, ExecutionException {
+ DeregisterStreamConsumerRequest deregisterStreamConsumerRequest =
+ DeregisterStreamConsumerRequest.builder().consumerARN(consumerArn).build();
+
+ return invokeWithRetryAndBackoff(
+ () ->
+ kinesisAsyncClient
+ .deregisterStreamConsumer(deregisterStreamConsumerRequest)
+ .get(),
+ fanOutRecordPublisherConfiguration.getDeregisterStreamBaseBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDeregisterStreamMaxBackoffMillis(),
+ fanOutRecordPublisherConfiguration.getDeregisterStreamExpConstant(),
+ fanOutRecordPublisherConfiguration.getDeregisterStreamMaxRetries());
+ }
+
+ private <T> T invokeWithRetryAndBackoff(
+ final ResponseSupplier<T> responseSupplier,
+ final long jitterBase,
+ final long jitterMax,
+ final double jitterExponent,
+ final int maximumNumberOfRetries)
+ throws InterruptedException, ExecutionException {
+ T response = null;
+ int attempt = 0;
+
+ while (attempt < maximumNumberOfRetries && response == null) {
+ try {
+ response = responseSupplier.get();
+ } catch (Exception ex) {
+ if (AwsV2Util.isRecoverableException(ex)) {
+ long backoffMillis =
+ backoff.calculateFullJitterBackoff(
+ jitterBase, jitterMax, jitterExponent, ++attempt);
+ LOG.warn(
+ "Encountered recoverable error: {}. Backing off for {} millis.",
+ ex.getClass().getSimpleName(),
+ backoffMillis,
+ ex);
+
+ backoff.sleep(backoffMillis);
+ } else {
+ throw ex;
+ }
+ }
+ }
+
+ if (response == null) {
+ throw new RuntimeException(
+ "Retries exceeded - all " + maximumNumberOfRetries + " retry attempts failed.");
+ }
+
+ return response;
+ }
+
+ private interface ResponseSupplier<T> {
+ T get() throws ExecutionException, InterruptedException;
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Factory.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Factory.java
new file mode 100644
index 0000000..24c3cb3
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Factory.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.aws.util.AWSAsyncSinkUtil;
+import org.apache.flink.connector.aws.util.AWSGeneralUtil;
+import org.apache.flink.connector.kinesis.sink.KinesisStreamsConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutRecordPublisherConfiguration;
+import org.apache.flink.streaming.connectors.kinesis.util.AwsV2Util;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.utils.AttributeMap;
+
+import java.util.Properties;
+
+import static java.util.Collections.emptyList;
+import static software.amazon.awssdk.http.SdkHttpConfigurationOption.TCP_KEEPALIVE;
+
+/** Creates instances of {@link KinesisProxyV2}. */
+@Internal
+public class KinesisProxyV2Factory {
+
+ private static final FullJitterBackoff BACKOFF = new FullJitterBackoff();
+
+ /**
+ * Uses the given properties to instantiate a new instance of {@link KinesisProxyV2}.
+ *
+ * @param configProps the properties used to parse configuration
+ * @return the Kinesis proxy
+ */
+ public static KinesisProxyV2Interface createKinesisProxyV2(final Properties configProps) {
+ Preconditions.checkNotNull(configProps);
+
+ final AttributeMap convertedProperties = AwsV2Util.convertProperties(configProps);
+ final AttributeMap.Builder clientConfiguration = AttributeMap.builder();
+ populateDefaultValues(clientConfiguration);
+
+ final SdkAsyncHttpClient httpClient =
+ AWSGeneralUtil.createAsyncHttpClient(
+ convertedProperties.merge(clientConfiguration.build()),
+ NettyNioAsyncHttpClient.builder());
+ final FanOutRecordPublisherConfiguration configuration =
+ new FanOutRecordPublisherConfiguration(configProps, emptyList());
+
+ Properties asyncClientProperties =
+ KinesisConfigUtil.getV2ConsumerAsyncClientProperties(configProps);
+
+ final KinesisAsyncClient client =
+ AWSAsyncSinkUtil.createAwsAsyncClient(
+ asyncClientProperties,
+ httpClient,
+ KinesisAsyncClient.builder(),
+ KinesisStreamsConfigConstants.BASE_KINESIS_USER_AGENT_PREFIX_FORMAT,
+ KinesisStreamsConfigConstants.KINESIS_CLIENT_USER_AGENT_PREFIX);
+
+ return new KinesisProxyV2(client, httpClient, configuration, BACKOFF);
+ }
+
+ private static void populateDefaultValues(final AttributeMap.Builder clientConfiguration) {
+ clientConfiguration.put(TCP_KEEPALIVE, true);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Interface.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Interface.java
new file mode 100644
index 0000000..8c30db4
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyV2Interface.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+
+import software.amazon.awssdk.services.kinesis.model.DeregisterStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse;
+import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerResponse;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
+import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Interface for a Kinesis proxy using AWS SDK v2.x operating on multiple Kinesis streams within the
+ * same AWS service region.
+ */
+@Internal
+public interface KinesisProxyV2Interface {
+
+ DescribeStreamSummaryResponse describeStreamSummary(String stream)
+ throws InterruptedException, ExecutionException;
+
+ DescribeStreamConsumerResponse describeStreamConsumer(final String streamConsumerArn)
+ throws InterruptedException, ExecutionException;
+
+ DescribeStreamConsumerResponse describeStreamConsumer(
+ final String streamArn, final String consumerName)
+ throws InterruptedException, ExecutionException;
+
+ RegisterStreamConsumerResponse registerStreamConsumer(
+ final String streamArn, final String consumerName)
+ throws InterruptedException, ExecutionException;
+
+ DeregisterStreamConsumerResponse deregisterStreamConsumer(final String consumerArn)
+ throws InterruptedException, ExecutionException;
+
+ CompletableFuture<Void> subscribeToShard(
+ SubscribeToShardRequest request, SubscribeToShardResponseHandler responseHandler);
+
+ /** Destroy any open resources used by the factory. */
+ default void close() {
+ // Do nothing by default
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java
new file mode 100644
index 0000000..e34a91c
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.serialization;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import com.amazonaws.services.dynamodbv2.model.Record;
+import com.amazonaws.services.dynamodbv2.streamsadapter.model.RecordObjectMapper;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+
+/** Schema used for deserializing DynamoDB streams records. */
+public class DynamoDBStreamsSchema implements KinesisDeserializationSchema<Record> {
+ private static final ObjectMapper MAPPER = new RecordObjectMapper();
+
+ @Override
+ public Record deserialize(
+ byte[] message,
+ String partitionKey,
+ String seqNum,
+ long approxArrivalTimestamp,
+ String stream,
+ String shardId)
+ throws IOException {
+ return MAPPER.readValue(message, Record.class);
+ }
+
+ @Override
+ public TypeInformation<Record> getProducedType() {
+ return TypeInformation.of(Record.class);
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
new file mode 100644
index 0000000..84f5863
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.serialization;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the
+ * basic {@link DeserializationSchema}, this schema offers additional Kinesis-specific information
+ * about the record that may be useful to the user application.
+ *
+ * @param <T> The type created by the keyed deserialization schema.
+ */
+@PublicEvolving
+public interface KinesisDeserializationSchema<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 a Kinesis record's bytes. If the record cannot be deserialized, {@code null} may
+ * be returned. This informs the Flink Kinesis Consumer to process the Kinesis record without
+ * producing any output for it, i.e. effectively "skipping" the record.
+ *
+ * @param recordValue the record's value as a byte array
+ * @param partitionKey the record's partition key at the time of writing
+ * @param seqNum the sequence number of this record in the Kinesis shard
+ * @param approxArrivalTimestamp the server-side timestamp of when Kinesis received and stored
+ * the record
+ * @param stream the name of the Kinesis stream that this record was sent to
+ * @param shardId The identifier of the shard the record was sent to
+ * @return the deserialized message as an Java object ({@code null} if the message cannot be
+ * deserialized).
+ * @throws IOException
+ */
+ T deserialize(
+ byte[] recordValue,
+ String partitionKey,
+ String seqNum,
+ long approxArrivalTimestamp,
+ String stream,
+ String shardId)
+ throws IOException;
+
+ /**
+ * 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
+ */
+ // TODO FLINK-4194 ADD SUPPORT FOR boolean isEndOfStream(T nextElement);
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java
new file mode 100644
index 0000000..a836d5b
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.serialization;
+
+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.util.Collector;
+
+import java.io.IOException;
+
+/**
+ * A simple wrapper for using the {@link DeserializationSchema} with the {@link
+ * KinesisDeserializationSchema} interface.
+ *
+ * @param <T> The type created by the deserialization schema.
+ */
+@Internal
+public class KinesisDeserializationSchemaWrapper<T> implements KinesisDeserializationSchema<T> {
+ private static final long serialVersionUID = 9143148962928375886L;
+
+ private final DeserializationSchema<T> deserializationSchema;
+
+ public KinesisDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
+ try {
+ Class<? extends DeserializationSchema> deserilizationClass =
+ deserializationSchema.getClass();
+ if (!deserilizationClass
+ .getMethod("deserialize", byte[].class, Collector.class)
+ .isDefault()) {
+ throw new IllegalArgumentException(
+ "Kinesis consumer does not support DeserializationSchema that implements "
+ + "deserialization with a Collector. Unsupported DeserializationSchema: "
+ + deserilizationClass.getName());
+ }
+ } catch (NoSuchMethodException e) {
+ // swallow the exception
+ }
+ this.deserializationSchema = deserializationSchema;
+ }
+
+ @Override
+ public void open(DeserializationSchema.InitializationContext context) throws Exception {
+ this.deserializationSchema.open(context);
+ }
+
+ @Override
+ public T deserialize(
+ byte[] recordValue,
+ String partitionKey,
+ String seqNum,
+ long approxArrivalTimestamp,
+ String stream,
+ String shardId)
+ throws IOException {
+ return deserializationSchema.deserialize(recordValue);
+ }
+
+ /*
+ FLINK-4194
+
+ @Override
+ public boolean isEndOfStream(T nextElement) {
+ return deserializationSchema.isEndOfStream(nextElement);
+ } */
+
+ @Override
+ public TypeInformation<T> getProducedType() {
+ return deserializationSchema.getProducedType();
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java
new file mode 100644
index 0000000..88bc209
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.serialization;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema.InitializationContext;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+/**
+ * Kinesis-specific serialization schema, allowing users to specify a target stream based on a
+ * record's contents.
+ *
+ * @param <T>
+ */
+@PublicEvolving
+public interface KinesisSerializationSchema<T> extends Serializable {
+ /**
+ * Initialization method for the schema. It is called before the actual working methods {@link
+ * #serialize(Object)} and thus suitable for one time setup work.
+ *
+ * <p>The provided {@link 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(InitializationContext context) throws Exception {}
+
+ /**
+ * Serialize the given element into a ByteBuffer.
+ *
+ * @param element The element to serialize
+ * @return Serialized representation of the element
+ */
+ ByteBuffer serialize(T element);
+
+ /**
+ * Optional method to determine the target stream based on the element. Return <code>null</code>
+ * to use the default stream
+ *
+ * @param element The element to determine the target stream from
+ * @return target stream name
+ */
+ String getTargetStream(T element);
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConnectorOptionsUtil.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConnectorOptionsUtil.java
new file mode 100644
index 0000000..3964773
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConnectorOptionsUtil.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.aws.table.util.AWSOptionUtils;
+import org.apache.flink.connector.aws.table.util.AsyncClientOptionsUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.connector.kinesis.table.KinesisConnectorOptions.STREAM;
+
+/**
+ * Class for handling kinesis table options, including key mapping and validations and property
+ * extraction. Class uses options decorators {@link AWSOptionUtils}, {@link AsyncClientOptionsUtils}
+ * and {@link KinesisConsumerOptionsUtil} for handling each specified set of options.
+ */
+@Internal
+public class KinesisConnectorOptionsUtil {
+
+ private final KinesisConsumerOptionsUtil kinesisConsumerOptionsUtil;
+ private final Map<String, String> resolvedOptions;
+ private final ReadableConfig tableOptions;
+
+ public KinesisConnectorOptionsUtil(Map<String, String> options, ReadableConfig tableOptions) {
+ this.resolvedOptions = options;
+ this.tableOptions = tableOptions;
+ this.kinesisConsumerOptionsUtil =
+ new KinesisConsumerOptionsUtil(resolvedOptions, tableOptions.get(STREAM));
+ }
+
+ public Properties getValidatedSourceConfigurations() {
+ return kinesisConsumerOptionsUtil.getValidatedConfigurations();
+ }
+
+ public List<String> getNonValidatedPrefixes() {
+ return kinesisConsumerOptionsUtil.getNonValidatedPrefixes();
+ }
+}
diff --git a/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConsumerOptionsUtil.java b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConsumerOptionsUtil.java
new file mode 100644
index 0000000..c3c2307
--- /dev/null
+++ b/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisConsumerOptionsUtil.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.aws.table.util.AWSOptionUtils;
+import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
... 17473 lines suppressed ...