dannycranmer commented on code in PR #49:
URL: 
https://github.com/apache/flink-connector-aws/pull/49#discussion_r1201711292


##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/KinesisStreamsSourceBuilder.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisShardAssigner;
+import 
org.apache.flink.connector.kinesis.source.enumerator.assigner.ShardAssignerFactory;
+import 
org.apache.flink.connector.kinesis.source.enumerator.assigner.UniformShardAssigner;
+import 
org.apache.flink.connector.kinesis.source.serialization.KinesisDeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Builder to construct the {@link KinesisStreamsSource}.
+ *
+ * <p>The following example shows the minimum setup to create a {@link 
KinesisStreamsSource} that
+ * reads String values from a Kinesis Data Streams stream with ARN of
+ * arn:aws:kinesis:us-east-1:012345678901:stream/your_stream_name.
+ *
+ * <pre>{@code
+ * KinesisStreamsSource<String> kdsSource =
+ *                 KinesisStreamsSource.<String>builder()
+ *                         
.setStreamArn("arn:aws:kinesis:us-east-1:012345678901:stream/your_stream_name")
+ *                         .setDeserializationSchema(new SimpleStringSchema())
+ *                         .build();
+ * }</pre>
+ *
+ * <p>If the following parameters are not set in this builder, the following 
defaults will be used:
+ *
+ * <ul>
+ *   <li>{@code kinesisShardAssigner} will be {@link UniformShardAssigner}
+ * </ul>
+ *
+ * @param <T> type of elements that should be read from the source stream
+ */
+@Experimental
+public class KinesisStreamsSourceBuilder<T> {
+    private String streamArn;
+    private Properties consumerConfig = new Properties();

Review Comment:
   If we use an empty properties will we not fail later since missing region 
etc? If so, shall we default to null and then the precondition will fail 
faster? Unless we parse the region from the ARN?



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/config/KinesisStreamsSourceConfigUtil.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.connector.kinesis.source.config;
+
+import org.apache.flink.annotation.Internal;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Properties;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_TIMESTAMP;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT;
+
+/** Utility functions to use with {@link KinesisStreamsSourceConfigConstants}. 
*/
+@Internal
+public class KinesisStreamsSourceConfigUtil {

Review Comment:
   Is it worth validating somewhere that the region in the ARN matches the 
region in the config and give a nice error message? Happy to add as follow up 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/config/KinesisStreamsSourceConfigUtil.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.connector.kinesis.source.config;
+
+import org.apache.flink.annotation.Internal;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Properties;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_TIMESTAMP;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT;
+
+/** Utility functions to use with {@link KinesisStreamsSourceConfigConstants}. 
*/
+@Internal
+public class KinesisStreamsSourceConfigUtil {
+
+    private KinesisStreamsSourceConfigUtil() {
+        // private constructor to prevent initialization of utility class.
+    }
+
+    /**
+     * Parses the timestamp in which to start consuming from the stream, from 
the given properties.
+     *
+     * @param consumerConfig the properties to parse timestamp from
+     * @return the timestamp
+     */
+    public static Date parseStreamTimestampStartingPosition(final Properties 
consumerConfig) {
+        String timestamp = 
consumerConfig.getProperty(STREAM_INITIAL_TIMESTAMP);

Review Comment:
   nit: Add preconditions here to prevent NPE



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(

Review Comment:
   This will result in scheduling 2 in parallel `initialDiscoverSplits` and 
`periodicallyDiscoverSplits` . I wonder if we should instead setup 
`periodicallyDiscoverSplits` from the end of `initialDiscoverSplits` method in 
the `lastSeenShardId` is null case.
   
   There also could be a race condition here since both will invoke 
`assignSplits` and subsequently `updateLastSeenShardId` so best not schedule 2 
in parallel incase we end up discovering the same shard twice
    



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+
+        // Assign the newly returned splits

Review Comment:
   Can you elaborate on this comment as the `emptyList()` here is confusing 
based on the comment. 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+
+        // Assign the newly returned splits
+        assignSplits(Collections.emptyList(), null);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(unassignedSplits, 
lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        streamProxy.close();
+    }
+
+    private List<KinesisShardSplit> initialDiscoverSplits() {
+        LOG.info("initialDiscoverSplits");
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        return mapToSplits(
+                shards,
+                InitialPosition.valueOf(
+                        consumerConfig
+                                .getOrDefault(
+                                        STREAM_INITIAL_POSITION, 
DEFAULT_STREAM_INITIAL_POSITION)
+                                .toString()));
+    }
+
+    /**
+     * This method is used to discover Kinesis splits the job can subscribe 
to. It can be run in
+     * parallel, is important to not mutate any shared state.
+     *
+     * @return list of discovered splits
+     */
+    private List<KinesisShardSplit> periodicallyDiscoverSplits() {
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        // Any shard discovered after the initial startup should be read from 
the start, since they
+        // come from resharding
+        return mapToSplits(shards, InitialPosition.TRIM_HORIZON);
+    }
+
+    private List<KinesisShardSplit> mapToSplits(
+            List<Shard> shards, InitialPosition initialPosition) {
+        StartingPosition startingPosition;
+        switch (initialPosition) {
+            case LATEST:
+                // If LATEST is requested, we still set the starting position 
to the time of
+                // startup. This way, the job starts reading from a 
deterministic timestamp
+                // (i.e. time of job submission), even if it enters a restart 
loop immediately
+                // after submission.
+                startingPosition = 
StartingPosition.fromTimestamp(Instant.now());
+                break;
+            case AT_TIMESTAMP:
+                startingPosition =
+                        StartingPosition.fromTimestamp(
+                                
parseStreamTimestampStartingPosition(consumerConfig).toInstant());
+                break;
+            case TRIM_HORIZON:
+            default:
+                startingPosition = StartingPosition.fromStart();
+        }
+
+        List<KinesisShardSplit> splits = new ArrayList<>();
+        for (Shard shard : shards) {
+            splits.add(new KinesisShardSplit(streamArn, shard.shardId(), 
startingPosition));
+        }
+
+        return splits;
+    }
+
+    /**
+     * This method assigns a given set of Kinesis splits to the readers 
currently registered on the
+     * cluster. This assignment is done via a side-effect on the {@link 
SplitEnumeratorContext}
+     * object.
+     *
+     * @param discoveredSplits list of discovered splits
+     * @param t throwable thrown when discovering splits. Will be null if no 
throwable thrown.
+     */
+    private void assignSplits(List<KinesisShardSplit> discoveredSplits, 
Throwable t) {
+        LOG.info("assignSplits");

Review Comment:
   Same as above, looks like a debugging comment



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/assigner/UniformShardAssigner.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator.assigner;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisShardAssigner;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** An implementation of the {@link KinesisShardAssigner} that assigns splits 
uniformly. */
+@Internal
+public class UniformShardAssigner implements KinesisShardAssigner {
+    @Override
+    public int assign(KinesisShardSplit split, Context context) {
+        int selectedSubtask = -1;
+        int curMinAssignment = Integer.MAX_VALUE;
+        Map<Integer, Set<KinesisShardSplit>> splitAssignment = 
context.getCurrentSplitAssignment();
+        Map<Integer, List<KinesisShardSplit>> pendingSplitAssignments =
+                context.getPendingSplitAssignments();
+
+        for (int subtaskId : context.getRegisteredReaders().keySet()) {
+            int subtaskAssignmentSize =
+                    splitAssignment.getOrDefault(subtaskId, 
Collections.emptySet()).size()
+                            + pendingSplitAssignments
+                                    .getOrDefault(subtaskId, 
Collections.emptyList())
+                                    .size();
+            if (subtaskAssignmentSize < curMinAssignment) {
+                curMinAssignment = subtaskAssignmentSize;
+                selectedSubtask = subtaskId;
+            }
+        }

Review Comment:
   I commented on this above. If for whatever reason only 1 reader is 
registered on the first listShards we would end up assigning ALL shards to a 
single subtask. I think instead we should wait for the readers to attach since 
we assume that all subtasks will attach, right? 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/serialization/KinesisDeserializationSchemaWrapper.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.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 software.amazon.awssdk.services.kinesis.model.Record;
+
+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
+class KinesisDeserializationSchemaWrapper<T> implements 
KinesisDeserializationSchema<T> {
+    private static final long serialVersionUID = 9143148962928375886L;
+
+    private final DeserializationSchema<T> deserializationSchema;
+
+    KinesisDeserializationSchemaWrapper(DeserializationSchema<T> 
deserializationSchema) {
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    @Override
+    public void open(DeserializationSchema.InitializationContext context) 
throws Exception {
+        this.deserializationSchema.open(context);
+    }
+
+    @Override
+    public void deserialize(Record record, String stream, String shardId, 
Collector<T> output)

Review Comment:
   The old version supplies additional parameters, why did we drop them?
   
   
https://github.com/apache/flink-connector-aws/blob/main/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java#L64



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+
+        // Assign the newly returned splits
+        assignSplits(Collections.emptyList(), null);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(unassignedSplits, 
lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        streamProxy.close();
+    }
+
+    private List<KinesisShardSplit> initialDiscoverSplits() {
+        LOG.info("initialDiscoverSplits");
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        return mapToSplits(
+                shards,
+                InitialPosition.valueOf(
+                        consumerConfig
+                                .getOrDefault(
+                                        STREAM_INITIAL_POSITION, 
DEFAULT_STREAM_INITIAL_POSITION)
+                                .toString()));
+    }
+
+    /**
+     * This method is used to discover Kinesis splits the job can subscribe 
to. It can be run in
+     * parallel, is important to not mutate any shared state.
+     *
+     * @return list of discovered splits
+     */
+    private List<KinesisShardSplit> periodicallyDiscoverSplits() {
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        // Any shard discovered after the initial startup should be read from 
the start, since they
+        // come from resharding
+        return mapToSplits(shards, InitialPosition.TRIM_HORIZON);
+    }
+
+    private List<KinesisShardSplit> mapToSplits(
+            List<Shard> shards, InitialPosition initialPosition) {
+        StartingPosition startingPosition;
+        switch (initialPosition) {
+            case LATEST:
+                // If LATEST is requested, we still set the starting position 
to the time of
+                // startup. This way, the job starts reading from a 
deterministic timestamp
+                // (i.e. time of job submission), even if it enters a restart 
loop immediately
+                // after submission.
+                startingPosition = 
StartingPosition.fromTimestamp(Instant.now());
+                break;
+            case AT_TIMESTAMP:
+                startingPosition =
+                        StartingPosition.fromTimestamp(
+                                
parseStreamTimestampStartingPosition(consumerConfig).toInstant());
+                break;
+            case TRIM_HORIZON:
+            default:
+                startingPosition = StartingPosition.fromStart();
+        }
+
+        List<KinesisShardSplit> splits = new ArrayList<>();
+        for (Shard shard : shards) {
+            splits.add(new KinesisShardSplit(streamArn, shard.shardId(), 
startingPosition));
+        }
+
+        return splits;
+    }
+
+    /**
+     * This method assigns a given set of Kinesis splits to the readers 
currently registered on the
+     * cluster. This assignment is done via a side-effect on the {@link 
SplitEnumeratorContext}
+     * object.
+     *
+     * @param discoveredSplits list of discovered splits
+     * @param t throwable thrown when discovering splits. Will be null if no 
throwable thrown.
+     */
+    private void assignSplits(List<KinesisShardSplit> discoveredSplits, 
Throwable t) {

Review Comment:
   nit: Instead of `t` use a meaningful arg name



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.enumerator.assigner.ShardAssignerFactory;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.model.CompletedShardsEvent;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final KinesisShardAssigner.Context shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+    private final Set<String> completedSplitIds;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = ShardAssignerFactory.uniformShardAssigner();
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.completedSplitIds = new HashSet<>();
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.completedSplitIds = state.getCompletedSplitIds();
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(
+                completedSplitIds, unassignedSplits, lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {}
+
+    @Override
+    public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+        if (sourceEvent instanceof CompletedShardsEvent) {
+            Set<String> newlyCompletedSplitIds =
+                    ((CompletedShardsEvent) 
sourceEvent).getCompletedSplitIds();
+            LOG.info(
+                    "Received CompletedShardsEvent from subtask {}. Marking 
the following splits as complete: {}",
+                    subtaskId,
+                    newlyCompletedSplitIds);
+            completedSplitIds.addAll(newlyCompletedSplitIds);
+        } else {
+            SplitEnumerator.super.handleSourceEvent(subtaskId, sourceEvent);
+        }
+    }
+
+    private List<KinesisShardSplit> initialDiscoverSplits() {
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        return mapToSplits(shards, false);
+    }
+
+    /**
+     * This method is used to discover Kinesis splits the job can subscribe 
to. It can be run in
+     * parallel, is important to not mutate any shared state.
+     *
+     * @return list of discovered splits
+     */
+    private List<KinesisShardSplit> periodicallyDiscoverSplits() {
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        // Any shard discovered after the initial startup should be read from 
the start, since they
+        // come from resharding
+        return mapToSplits(shards, true);
+    }
+
+    private List<KinesisShardSplit> mapToSplits(List<Shard> shards, boolean 
shouldReadFromStart) {
+        InitialPosition initialPositionFromConfig =
+                shouldReadFromStart
+                        ? InitialPosition.TRIM_HORIZON
+                        : InitialPosition.valueOf(
+                                consumerConfig
+                                        .getOrDefault(
+                                                STREAM_INITIAL_POSITION,
+                                                
DEFAULT_STREAM_INITIAL_POSITION)
+                                        .toString());
+        StartingPosition startingPosition;
+        switch (initialPositionFromConfig) {
+            case LATEST:
+                // If LATEST is requested, we still set the starting position 
to the time of
+                // startup. This way, the job starts reading from a 
deterministic timestamp
+                // (i.e. time of job submission), even if it enters a restart 
loop immediately
+                // after submission.
+                startingPosition = 
StartingPosition.fromTimestamp(Instant.now());
+                break;
+            case AT_TIMESTAMP:
+                startingPosition =
+                        StartingPosition.fromTimestamp(
+                                
parseStreamTimestampStartingPosition(consumerConfig).toInstant());
+                break;
+            case TRIM_HORIZON:
+            default:
+                startingPosition = StartingPosition.fromStart();
+        }
+
+        List<KinesisShardSplit> splits = new ArrayList<>();
+        for (Shard shard : shards) {
+            splits.add(new KinesisShardSplit(streamArn, shard.shardId(), 
startingPosition));
+        }
+
+        return splits;
+    }
+
+    /**
+     * This method assigns a given set of Kinesis splits to the readers 
currently registered on the
+     * cluster. This assignment is done via a side-effect on the {@link 
SplitEnumeratorContext}
+     * object.
+     *
+     * @param discoveredSplits list of discovered splits
+     * @param t throwable thrown when discovering splits. Will be null if no 
throwable thrown.
+     */
+    private void assignSplits(List<KinesisShardSplit> discoveredSplits, 
Throwable t) {
+        if (t != null) {
+            throw new KinesisStreamsSourceException("Failed to list shards.", 
t);
+        }
+
+        if (context.registeredReaders().isEmpty()) {

Review Comment:
   Following on from this, it seems if there is at least 1 reader we will 
blindly assign splits to it based on the shard assigner output. I wonder if we 
should skip assignment when the reader is not registered? Another rejected 
option is to assign all splits to the registered readers, but this will result 
in skew so not a good idea.



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+
+        // Assign the newly returned splits
+        assignSplits(Collections.emptyList(), null);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(unassignedSplits, 
lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        streamProxy.close();
+    }
+
+    private List<KinesisShardSplit> initialDiscoverSplits() {
+        LOG.info("initialDiscoverSplits");

Review Comment:
   Either remove this or add more details, for example the Stream ARN. Looks 
like a debug comment



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/reader/PollingKinesisShardSplitReader.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.reader;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplitState;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.Record;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * An implementation of the SplitReader that periodically polls the Kinesis 
stream to retrieve
+ * records.
+ */
+@Internal
+public class PollingKinesisShardSplitReader implements SplitReader<Record, 
KinesisShardSplit> {
+
+    private static final RecordsWithSplitIds<Record> 
INCOMPLETE_SHARD_EMPTY_RECORDS =
+            new KinesisRecordsWithSplitIds(Collections.emptyIterator(), null, 
false);
+
+    private final StreamProxy kinesis;
+    private final Deque<KinesisShardSplitState> assignedSplits = new 
ArrayDeque<>();
+
+    public PollingKinesisShardSplitReader(StreamProxy kinesisProxy) {
+        this.kinesis = kinesisProxy;
+    }
+
+    @Override
+    public RecordsWithSplitIds<Record> fetch() throws IOException {
+        KinesisShardSplitState splitState = assignedSplits.poll();
+        if (splitState == null) {
+            return INCOMPLETE_SHARD_EMPTY_RECORDS;
+        }
+
+        GetRecordsResponse getRecordsResponse =
+                kinesis.getRecords(
+                        splitState.getStreamArn(),
+                        splitState.getShardId(),
+                        splitState.getNextStartingPosition());
+        boolean isComplete = getRecordsResponse.nextShardIterator() == null;
+
+        if (hasNoRecords(getRecordsResponse)) {
+            if (isComplete) {
+                return new KinesisRecordsWithSplitIds(
+                        Collections.emptyIterator(), splitState.splitId(), 
true);
+            } else {
+                assignedSplits.add(splitState);
+                return INCOMPLETE_SHARD_EMPTY_RECORDS;
+            }
+        }
+
+        splitState.setNextStartingPosition(
+                StartingPosition.continueFromSequenceNumber(
+                        getRecordsResponse
+                                .records()
+                                .get(getRecordsResponse.records().size() - 1)
+                                .sequenceNumber()));
+
+        assignedSplits.add(splitState);
+        return new KinesisRecordsWithSplitIds(
+                getRecordsResponse.records().iterator(), splitState.splitId(), 
isComplete);
+    }
+
+    private boolean hasNoRecords(GetRecordsResponse getRecordsResponse) {
+        return !getRecordsResponse.hasRecords() || 
getRecordsResponse.records().isEmpty();
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<KinesisShardSplit> 
splitsChanges) {
+        for (KinesisShardSplit split : splitsChanges.splits()) {
+            assignedSplits.add(new KinesisShardSplitState(split));
+        }
+    }
+
+    @Override
+    public void wakeUp() {
+        // Do nothing because we don't have any sleep mechanism
+    }
+
+    @Override
+    public void close() throws Exception {}

Review Comment:
   Resolved but not updated, unresolved



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.KinesisStreamsSourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final ShardAssignerContext shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisShardAssigner shardAssigner,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = shardAssigner;
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+
+        // Assign the newly returned splits
+        assignSplits(Collections.emptyList(), null);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(unassignedSplits, 
lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        streamProxy.close();
+    }
+
+    private List<KinesisShardSplit> initialDiscoverSplits() {
+        LOG.info("initialDiscoverSplits");
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        return mapToSplits(
+                shards,
+                InitialPosition.valueOf(
+                        consumerConfig
+                                .getOrDefault(
+                                        STREAM_INITIAL_POSITION, 
DEFAULT_STREAM_INITIAL_POSITION)
+                                .toString()));
+    }
+
+    /**
+     * This method is used to discover Kinesis splits the job can subscribe 
to. It can be run in
+     * parallel, is important to not mutate any shared state.
+     *
+     * @return list of discovered splits
+     */
+    private List<KinesisShardSplit> periodicallyDiscoverSplits() {
+        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
+        // Any shard discovered after the initial startup should be read from 
the start, since they
+        // come from resharding
+        return mapToSplits(shards, InitialPosition.TRIM_HORIZON);
+    }
+
+    private List<KinesisShardSplit> mapToSplits(
+            List<Shard> shards, InitialPosition initialPosition) {
+        StartingPosition startingPosition;
+        switch (initialPosition) {
+            case LATEST:
+                // If LATEST is requested, we still set the starting position 
to the time of
+                // startup. This way, the job starts reading from a 
deterministic timestamp
+                // (i.e. time of job submission), even if it enters a restart 
loop immediately
+                // after submission.
+                startingPosition = 
StartingPosition.fromTimestamp(Instant.now());
+                break;
+            case AT_TIMESTAMP:
+                startingPosition =
+                        StartingPosition.fromTimestamp(
+                                
parseStreamTimestampStartingPosition(consumerConfig).toInstant());
+                break;
+            case TRIM_HORIZON:
+            default:
+                startingPosition = StartingPosition.fromStart();
+        }
+
+        List<KinesisShardSplit> splits = new ArrayList<>();
+        for (Shard shard : shards) {
+            splits.add(new KinesisShardSplit(streamArn, shard.shardId(), 
startingPosition));
+        }
+
+        return splits;
+    }
+
+    /**
+     * This method assigns a given set of Kinesis splits to the readers 
currently registered on the
+     * cluster. This assignment is done via a side-effect on the {@link 
SplitEnumeratorContext}
+     * object.
+     *
+     * @param discoveredSplits list of discovered splits
+     * @param t throwable thrown when discovering splits. Will be null if no 
throwable thrown.
+     */
+    private void assignSplits(List<KinesisShardSplit> discoveredSplits, 
Throwable t) {
+        LOG.info("assignSplits");
+        if (t != null) {
+            throw new KinesisStreamsSourceException("Failed to list shards.", 
t);
+        }
+
+        if (context.registeredReaders().isEmpty()) {
+            LOG.info("No registered readers, skipping assignment of discovered 
splits.");
+            unassignedSplits.addAll(discoveredSplits);
+            return;
+        }
+
+        Map<Integer, List<KinesisShardSplit>> newSplitAssignments = new 
HashMap<>();
+        for (KinesisShardSplit split : unassignedSplits) {
+            assignSplitToSubtask(split, newSplitAssignments);
+        }
+        unassignedSplits.clear();
+        for (KinesisShardSplit split : discoveredSplits) {
+            assignSplitToSubtask(split, newSplitAssignments);
+        }
+
+        updateLastSeenShardId(discoveredSplits);
+        updateSplitAssignment(newSplitAssignments);
+        context.assignSplits(new SplitsAssignment<>(newSplitAssignments));
+    }
+
+    private void assignSplitToSubtask(
+            KinesisShardSplit split, Map<Integer, List<KinesisShardSplit>> 
newSplitAssignments) {
+        if (assignedSplitIds.contains(split.splitId())) {
+            LOG.info(
+                    "Skipping assignment of shard {} from stream {} because it 
is already assigned.",
+                    split.getShardId(),
+                    split.getStreamArn());
+            return;
+        }
+
+        int selectedSubtask =
+                shardAssigner.assign(
+                        split,
+                        
shardAssignerContext.withPendingSplitAssignments(newSplitAssignments));
+        LOG.info(
+                "Assigning shard {} from stream {} to subtask {}.",
+                split.getShardId(),
+                split.getStreamArn(),
+                selectedSubtask);
+
+        if (newSplitAssignments.containsKey(selectedSubtask)) {
+            newSplitAssignments.get(selectedSubtask).add(split);
+        } else {
+            List<KinesisShardSplit> subtaskList = new ArrayList<>();
+            subtaskList.add(split);
+            newSplitAssignments.put(selectedSubtask, subtaskList);
+        }
+        assignedSplitIds.add(split.splitId());
+    }
+
+    private void updateLastSeenShardId(List<KinesisShardSplit> 
discoveredSplits) {
+        if (!discoveredSplits.isEmpty()) {
+            KinesisShardSplit lastSplit = 
discoveredSplits.get(discoveredSplits.size() - 1);
+            lastSeenShardId = lastSplit.getShardId();
+        }
+    }
+
+    private void updateSplitAssignment(Map<Integer, List<KinesisShardSplit>> 
newSplitsAssignment) {
+        newSplitsAssignment.forEach(
+                (subtaskId, newSplits) -> {
+                    if (splitAssignment.containsKey(subtaskId)) {
+                        splitAssignment.get(subtaskId).addAll(newSplits);
+                    } else {
+                        splitAssignment.put(subtaskId, new 
HashSet<>(newSplits));
+                    }
+                });
+    }
+
+    @Internal
+    private static class ShardAssignerContext implements 
KinesisShardAssigner.Context {
+
+        private final Map<Integer, Set<KinesisShardSplit>> splitAssignment;
+        private final SplitEnumeratorContext<KinesisShardSplit> 
splitEnumeratorContext;
+        private Map<Integer, List<KinesisShardSplit>> pendingSplitAssignments =
+                Collections.emptyMap();
+
+        private ShardAssignerContext(
+                Map<Integer, Set<KinesisShardSplit>> splitAssignment,
+                SplitEnumeratorContext<KinesisShardSplit> 
splitEnumeratorContext) {
+            this.splitAssignment = splitAssignment;
+            this.splitEnumeratorContext = splitEnumeratorContext;
+        }
+
+        private ShardAssignerContext withPendingSplitAssignments(
+                Map<Integer, List<KinesisShardSplit>> pendingSplitAssignments) 
{
+            ImmutableMap.Builder<Integer, List<KinesisShardSplit>> mapBuilder =

Review Comment:
   nit: Why not just use a `HashMap` here? You want it to be immutable?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to