Poorvankbhatia commented on code in PR #6:
URL: 
https://github.com/apache/flink-connector-redis-streams/pull/6#discussion_r3158725646


##########
flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/split/RedisStreamsSourceSplit.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.redis.streams.source.split;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+/**
+ * Immutable Redis Stream split: identity is {@link #streamKey}; {@link 
#startingEntryId} is an
+ * informational resume position (canonical state lives in the Redis consumer 
group);
+ * {@link #stoppingEntryId} is non-null only in bounded mode.
+ */
+@Internal
+public class RedisStreamsSourceSplit implements SourceSplit {
+
+    private final String streamKey;
+    @Nullable private final String startingEntryId;

Review Comment:
   I checked the code: "startingEntryId" is in a dead state, but it's 
checkpointed every snapshot. 
   
   `RedisStreamsSplitReader.fetchFromSplit` reads streamKey and 
stoppingEntryId, never startingEntryId. The field exists only so that:          
                                  
     - `RedisStreamsSourceSplitState` can initialize currentEntryId from it on 
restore                                                           
     - `currentEntryId` then gets advanced as records are emitted               
                                                                                
                                                          
     - `toSplit()` puts it back into the snapshot                               
                                                                 
                                                                                
                                                                                
                                                        
   But the SplitReader never reads it — recovery is driven entirely by the 
consumer group's PEL + XREADGROUP >. 



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