hlteoh37 commented on code in PR #49: URL: https://github.com/apache/flink-connector-aws/pull/49#discussion_r1194715570
########## 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 Interesting thought. I think in this case, I don't think we should expose the option to turn this off, since this is an essential mechanism for correctly handling merged/split shards.. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org