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


##########
flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/reader/split/RedisStreamsSplitReader.java:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.reader.split;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+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.SplitsAddition;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import 
org.apache.flink.connector.redis.streams.source.config.RedisStreamsSourceConfig;
+import org.apache.flink.connector.redis.streams.source.config.StartupMode;
+import 
org.apache.flink.connector.redis.streams.source.split.RedisStreamsSourceSplit;
+
+import io.lettuce.core.AbstractRedisClient;
+import io.lettuce.core.ClientOptions;
+import io.lettuce.core.Consumer;
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.RedisConnectionException;
+import io.lettuce.core.RedisURI;
+import io.lettuce.core.StreamMessage;
+import io.lettuce.core.XGroupCreateArgs;
+import io.lettuce.core.XReadArgs;
+import io.lettuce.core.api.StatefulConnection;
+import io.lettuce.core.cluster.ClusterClientOptions;
+import io.lettuce.core.cluster.ClusterTopologyRefreshOptions;
+import io.lettuce.core.cluster.RedisClusterClient;
+import io.lettuce.core.cluster.api.sync.RedisClusterCommands;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Split reader that fetches records from Redis Streams via consumer groups 
and defers XACK until
+ * the next completed Flink checkpoint.
+ */
+@Internal
+public class RedisStreamsSplitReader
+        implements SplitReader<StreamMessage<String, String>, 
RedisStreamsSourceSplit> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(RedisStreamsSplitReader.class);
+
+    private static final long INITIAL_RECONNECT_DELAY_MS = 100L;
+    private static final long MAX_RECONNECT_DELAY_MS = 30000L;
+    private static final double BACKOFF_MULTIPLIER = 1.5;
+    private static final int MAX_CONSUMER_GROUP_RETRIES = 3;
+    private static final long CONSUMER_GROUP_RETRY_DELAY_MS = 100L;
+
+    private final RedisStreamsSourceConfig config;
+    private final int subtaskId;
+    private final ReentrantLock stateLock = new ReentrantLock();
+
+    // Owned by the fetch thread; commands is also read by the checkpoint 
thread under stateLock.
+    private AbstractRedisClient redisClient;
+    private StatefulConnection<String, String> connection;
+    private RedisClusterCommands<String, String> commands;
+
+    private final AtomicBoolean wokenUp = new AtomicBoolean(false);
+
+    // Guarded by stateLock.
+
+    private final Map<String, RedisStreamsSourceSplit> assignedSplits = new 
HashMap<>();
+    private final Set<String> pausedSplits = new HashSet<>();
+    private final Map<String, Deque<String>> deferredAcks = new HashMap<>();
+    private final Map<String, CircuitBreaker> splitCircuitBreakers = new 
HashMap<>();
+    // checkpointId -> splitId -> max #IDs to ack from front of deferredAcks 
at that barrier.
+    private final Map<Long, Map<String, Integer>> checkpointAckSnapshots = new 
HashMap<>();

Review Comment:
   I have fixed this issue. Added an eviction sweep inside 
`acknowledgeAllPendingMessagesAtCheckpoint`. When checkpoint N completes, all 
entries with key < N that were never explicitly discarded are removed from 
checkpointAckSnapshots, and their IDs are also removed from deferredAcks to 
release the back-pressure counter. The entries themselves are safe. they remain 
in Redis's PEL and will be re-delivered on recovery. 



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