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


##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/assigner/ShardAssignerFactory.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Experimental;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisShardAssigner;
+
+/**
+ * Factory that provides an instance of {@link KinesisShardAssigner} 
pre-packaged with the
+ * connector.
+ */
+@Experimental
+public class ShardAssignerFactory {
+    private ShardAssignerFactory() {
+        // No-op private constructor to prevent instantiation of class
+    }
+
+    public static KinesisShardAssigner uniformShardAssigner() {
+        return new UniformShardAssigner();
+    }
+
+    public static KinesisShardAssigner hashShardAssigner() {

Review Comment:
   It looks like the `HashShardAssigner` is only used in tests. What about 
marking it as `@VisibleForTesting` or expose it to users with a configuration? 



##########
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:
   The Kinesis client should be closed here I think.



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 ShardAssignerContext 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 =

Review Comment:
   Maybe negative or zero interval can be interpreted as disabling the 
discovery. Currently it is not possible for users disable the feature, and a 
negative or zero value will lead to an exception. 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 ShardAssignerContext 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;

Review Comment:
   I'm not quite familiar with Kinesis so please correct me if I'm wrong. By 
`completed` does that mean there won't be new records in the shard (like a 
bounded shard), or the shard is going to be removed? 
   
   The only usage of this `completedSplitIds` is to avoid assigning a completed 
shard to readers, so if the SDK can promise that we won't get completed shards 
using the last seen shard ID, it's not necessary to have the 
`completedSplitIds` I guess. 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 ShardAssignerContext 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);

Review Comment:
   A split assignment needs to be triggered here. Otherwise `unassignedSplits` 
will never be assigned again if shard discovery is disabled. 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/config/SourceConfigConstants.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Experimental;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import 
org.apache.flink.connector.kinesis.source.reader.PollingKinesisShardSplitReader;
+
+import java.time.Duration;
+
+/** Constants to be used with the KinesisStreamsSource. */
+@Experimental
+public class SourceConfigConstants extends AWSConfigConstants {
+    /** Marks the initial position to use when reading from the Kinesis 
stream. */
+    public enum InitialPosition {
+        LATEST,
+        TRIM_HORIZON,
+        AT_TIMESTAMP
+    }
+
+    /** 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). */
+    public static final String RECORD_PUBLISHER_TYPE = 
"flink.stream.recordpublisher";

Review Comment:
   It's a bit weird that all configs start with `flink.`, which look like 
configs for Flink instead of the connector 🤔 



-- 
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