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


##########
flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/split/RedisStreamsSourceSplit.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.PublicEvolving;
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * A split representing a Redis Stream key.
+ *
+ * <p>Each split corresponds to one Redis Stream key and tracks the last read 
entry ID for
+ * checkpointing purposes.
+ */
+@PublicEvolving
+public class RedisStreamsSourceSplit implements SourceSplit, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String streamKey;

Review Comment:
   Please add a comment for all these fields. 



##########
flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/split/RedisStreamsSourceSplit.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.PublicEvolving;
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * A split representing a Redis Stream key.
+ *
+ * <p>Each split corresponds to one Redis Stream key and tracks the last read 
entry ID for
+ * checkpointing purposes.
+ */
+@PublicEvolving
+public class RedisStreamsSourceSplit implements SourceSplit, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String streamKey;
+    private final String lastReadEntryId;

Review Comment:
   Two things here:
   
   1. The constructor Javadoc says "null if starting fresh" — so null is a 
valid documented value. But getLastReadEntryId() has no @Nullable on its return 
type.  Now, a better approach could be to replace null with the Redis sentinel 
"0-0". _Redis accepts "0-0" in XREADGROUP as "before all entries"_ — it is the 
natural "start from beginning" ID. This eliminates the nullability entirely.
   
   2. This field i assume, exists to resume reading from the last checkpointed 
position on recovery, but _RedisStreamsSplitReader.fetchFromSplit()_ ignores it 
completely — it always calls                                    
XReadArgs.StreamOffset.lastConsumed(streamKey), which delegates position 
tracking entirely to Redis's server-side consumer group state. On recovery, the 
reader doesn't read it, doesn't pass it to Redis, and just picks up wherever 
the consumer group left off server-side.



##########
flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/split/RedisStreamsSourceSplit.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.PublicEvolving;
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * A split representing a Redis Stream key.
+ *
+ * <p>Each split corresponds to one Redis Stream key and tracks the last read 
entry ID for
+ * checkpointing purposes.
+ */
+@PublicEvolving
+public class RedisStreamsSourceSplit implements SourceSplit, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String streamKey;
+    private final String lastReadEntryId;
+
+    /**
+     * Creates a new split for a Redis Stream.
+     *
+     * @param streamKey The Redis Stream key
+     * @param lastReadEntryId The ID of the last entry read from this stream 
(null if starting
+     *     fresh)
+     */
+    public RedisStreamsSourceSplit(String streamKey, String lastReadEntryId) {

Review Comment:
   The split class itself is missing critical fields. The correct design IMO:   
                                                                                
                                                        
    streamKey        — which Redis stream (identity)
    consumerName     — which consumer in the group, deterministic from subtask 
index (identity)                                                                
                                                      
    startingEntryId  — "0-0" on fresh start, last checkpointed entry ID on 
recovery                                                                        
                                                          
    stoppingEntryId  — null for unbounded streaming, concrete entry ID for 
bounded mode   
   
   As of now, lastReadEntryId is actually mutable and should be a part of 
RedisStreamSourceSplitState.



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