z3d1k commented on code in PR #145:
URL: 
https://github.com/apache/flink-connector-aws/pull/145#discussion_r1670444385


##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/tracker/SplitTracker.java:
##########
@@ -0,0 +1,147 @@
+package org.apache.flink.connector.kinesis.source.enumerator.tracker;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisShardSplitWithAssignmentStatus;
+import 
org.apache.flink.connector.kinesis.source.enumerator.SplitAssignmentStatus;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/** This class is used to track shard hierarchy. */
+@Internal
+public class SplitTracker {
+    /**
+     * Flag controlling if tracker should wait before all parent splits will 
be completed before
+     * assigning split to readers.
+     */
+    private final boolean preserveShardOrdering;
+
+    /** Map of all discovered splits that have not been completed. */
+    private final Map<String, KinesisShardSplit> knownSplits = new 
ConcurrentHashMap<>();

Review Comment:
   Added info about the key



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumeratorState.java:
##########
@@ -19,32 +19,31 @@
 package org.apache.flink.connector.kinesis.source.enumerator;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
 
 import javax.annotation.Nullable;
 
-import java.util.Set;
+import java.util.List;
 
 /**
  * State for the {@link KinesisStreamsSourceEnumerator}. This class is stored 
in state, so any
  * changes need to be backwards compatible
  */
 @Internal
 public class KinesisStreamsSourceEnumeratorState {
-    private final Set<KinesisShardSplit> unassignedSplits;
+    private final List<KinesisShardSplitWithAssignmentStatus> splits;
     @Nullable private final String lastSeenShardId;
 
     public KinesisStreamsSourceEnumeratorState(
-            Set<KinesisShardSplit> unassignedSplits, String lastSeenShardId) {
-        this.unassignedSplits = unassignedSplits;
+            List<KinesisShardSplitWithAssignmentStatus> splits, String 
lastSeenShardId) {

Review Comment:
   Renamed



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumeratorStateSerializer.java:
##########
@@ -109,13 +110,22 @@ public KinesisStreamsSourceEnumeratorState deserialize(
                                 + ". Serializer version is "
                                 + splitSerializer.getVersion());
             }
-            Set<KinesisShardSplit> unassignedSplits = new 
HashSet<>(numUnassignedSplits);
+            List<KinesisShardSplitWithAssignmentStatus> unassignedSplits =
+                    new ArrayList<>(numUnassignedSplits);
             for (int i = 0; i < numUnassignedSplits; i++) {
                 int serializedLength = in.readInt();
                 byte[] serializedSplit = new byte[serializedLength];
                 if (in.read(serializedSplit) != -1) {
+                    KinesisShardSplit deserializedSplit =
+                            
splitSerializer.deserialize(splitSerializerVersion, serializedSplit);
+                    SplitAssignmentStatus assignmentStatus = 
SplitAssignmentStatus.UNASSIGNED;
+                    if (version == CURRENT_VERSION) {
+                        assignmentStatus = 
SplitAssignmentStatus.fromStatusCode(in.readInt());
+                    }

Review Comment:
   Yes, previous version contains only unassigned splits.



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/tracker/SplitTracker.java:
##########
@@ -0,0 +1,147 @@
+package org.apache.flink.connector.kinesis.source.enumerator.tracker;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisShardSplitWithAssignmentStatus;
+import 
org.apache.flink.connector.kinesis.source.enumerator.SplitAssignmentStatus;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/** This class is used to track shard hierarchy. */
+@Internal
+public class SplitTracker {
+    /**
+     * Flag controlling if tracker should wait before all parent splits will 
be completed before
+     * assigning split to readers.
+     */
+    private final boolean preserveShardOrdering;
+
+    /** Map of all discovered splits that have not been completed. */
+    private final Map<String, KinesisShardSplit> knownSplits = new 
ConcurrentHashMap<>();
+
+    /** Set of currently assigned split id. */

Review Comment:
   fixed



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/reader/KinesisStreamsSourceReader.java:
##########
@@ -64,6 +66,8 @@ public KinesisStreamsSourceReader(
     @Override
     protected void onSplitFinished(Map<String, KinesisShardSplitState> 
finishedSplitIds) {
         finishedSplitIds.keySet().forEach(this::unregisterShardMetricGroup);
+        context.sendSourceEventToCoordinator(
+                new SplitsFinishedEvent(new 
HashSet<>(finishedSplitIds.keySet())));
     }

Review Comment:
   Done.



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/event/SplitsFinishedEvent.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kinesis.source.event;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceEvent;
+
+import java.util.Set;
+
+/** Source event used by source reader to communicate that splits are finished 
to enumerator. */
+@Internal
+public class SplitsFinishedEvent implements SourceEvent {

Review Comment:
   We don't need to implement these, as we don't perform equality checks or use 
this event as key. 
   
   Also, example of such event without these methods - 
https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/SwitchSourceEvent.java



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisShardSplitWithAssignmentStatus.java:
##########
@@ -0,0 +1,45 @@
+package org.apache.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+
+import java.util.Objects;
+
+/** Kinesis shard split with assignment status. */
+@Internal
+public class KinesisShardSplitWithAssignmentStatus {
+    private final KinesisShardSplit kinesisShardSplit;
+    private final SplitAssignmentStatus splitAssignmentStatus;
+
+    public KinesisShardSplitWithAssignmentStatus(
+            KinesisShardSplit kinesisShardSplit, SplitAssignmentStatus 
splitAssignmentStatus) {
+        this.kinesisShardSplit = kinesisShardSplit;
+        this.splitAssignmentStatus = splitAssignmentStatus;
+    }
+
+    public KinesisShardSplit split() {
+        return kinesisShardSplit;
+    }
+
+    public SplitAssignmentStatus assignmentStatus() {
+        return splitAssignmentStatus;
+    }
+
+    @Override
+    public boolean equals(Object o) {

Review Comment:
   Added



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumeratorTest.java:
##########
@@ -275,53 +262,8 @@ void testAddSplitsBackWithoutSplitIsNoOp() throws 
Throwable {
             // Given enumerator has no assigned splits
             // When we add splits back
             // Then handled gracefully with no exception thrown

Review Comment:
   Removed



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -143,17 +156,101 @@ public void addReader(int subtaskId) {
 
     @Override
     public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
-        return new KinesisStreamsSourceEnumeratorState(unassignedSplits, 
lastSeenShardId);
+        List<KinesisShardSplitWithAssignmentStatus> splitStates =
+                splitTracker.snapshotState(checkpointId);
+        return new KinesisStreamsSourceEnumeratorState(splitStates, 
lastSeenShardId);
     }
 
     @Override
     public void close() throws IOException {
         streamProxy.close();
     }
 
-    private List<KinesisShardSplit> initialDiscoverSplits() {
-        List<Shard> shards = streamProxy.listShards(streamArn, 
lastSeenShardId);
-        return mapToSplits(shards, sourceConfig.get(STREAM_INITIAL_POSITION));
+    @VisibleForTesting

Review Comment:
   In this case logic does not fit in the StreamsProxy, but also needs to be 
tested.
   Discussed with @hlteoh37 offline and we agreed that VisibleForTesting is ok 
here.



##########
flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -211,32 +301,33 @@ private void assignSplits(List<KinesisShardSplit> 
discoveredSplits, Throwable th
             throw new KinesisStreamsSourceException("Failed to list shards.", 
throwable);
         }
 
+        splitTracker.addSplits(discoveredSplits);
+        updateLastSeenShardId(discoveredSplits);
+
         if (context.registeredReaders().size() < context.currentParallelism()) 
{
             LOG.info(
                     "Insufficient registered readers, skipping assignment of 
discovered splits until all readers are registered. Required number of readers: 
{}, Registered readers: {}",
                     context.currentParallelism(),
                     context.registeredReaders().size());
-            unassignedSplits.addAll(discoveredSplits);
             return;
         }
 
+        assignSplits();
+    }
+
+    private void assignSplits() {

Review Comment:
   This method is performing split assignment to readers, IMO name represents 
the action well.
   
   Renamed method above to clearer represent meaning.



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

Reply via email to