Samrat002 commented on code in PR #6: URL: https://github.com/apache/flink-connector-redis-streams/pull/6#discussion_r3055297122
########## flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/reader/split/RedisStreamsSplitReader.java: ########## @@ -0,0 +1,707 @@ +/* + * 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.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.split.RedisStreamsSourceSplit; + +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.StatefulRedisConnection; +import io.lettuce.core.api.sync.RedisCommands; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Split reader that fetches records from Redis Streams in a dedicated thread. + * + * <p>Implements the {@link SplitReader} interface and runs in a background thread managed by + * Flink's {@link org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher}. Continuously + * polls Redis Streams via consumer groups and supports deferred acknowledgment aligned with Flink + * checkpoints. + * + * <p>Thread safety: All mutable state is guarded by {@code stateLock}. Redis I/O is performed + * outside the lock to avoid blocking concurrent checkpoint operations. + */ +@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 ReentrantLock stateLock = new ReentrantLock(); + + private RedisClient redisClient; + private StatefulRedisConnection<String, String> connection; + private RedisCommands<String, String> commands; + private volatile boolean wokenUp = false; + + private final Map<String, RedisStreamsSourceSplit> assignedSplits; + private final Set<String> pausedSplits; + private final Set<String> finishedSplits; + private final Map<String, LinkedList<String>> deferredAcks; + private final Map<String, CircuitBreaker> splitCircuitBreakers; + private final Map<Long, Map<String, Integer>> checkpointAckWatermarks; + private final Map<String, Integer> lastCompletedWatermarkPerSplit; + + private long nextReconnectDelayMs = INITIAL_RECONNECT_DELAY_MS; + private long lastConnectionFailureTime = 0; + private boolean connectionInitialized = false; + + public RedisStreamsSplitReader(RedisStreamsSourceConfig config) { + this.config = config; + this.assignedSplits = new HashMap<>(); + this.pausedSplits = new HashSet<>(); + this.finishedSplits = new HashSet<>(); + this.deferredAcks = new HashMap<>(); + this.splitCircuitBreakers = new HashMap<>(); + this.checkpointAckWatermarks = new HashMap<>(); + this.lastCompletedWatermarkPerSplit = new HashMap<>(); + } + + private void initializeConnection() { + try { + RedisURI.Builder uriBuilder = + RedisURI.builder() + .withHost(config.getHost()) + .withPort(config.getPort()) + .withDatabase(config.getDatabase()) + .withTimeout(Duration.ofSeconds(5)); + + if (config.getPassword() != null && !config.getPassword().isEmpty()) { + uriBuilder.withPassword(config.getPassword().toCharArray()); + } + + this.redisClient = RedisClient.create(uriBuilder.build()); + this.connection = redisClient.connect(); + this.commands = connection.sync(); + this.nextReconnectDelayMs = INITIAL_RECONNECT_DELAY_MS; + this.lastConnectionFailureTime = 0; + this.connectionInitialized = true; + LOG.info("Connected to Redis at {}:{}", config.getHost(), config.getPort()); + } catch (Exception e) { + LOG.error("Failed to initialize Redis connection", e); + cleanupConnection(); + handleConnectionFailure(); + } + } + + private void handleConnectionFailure() { + lastConnectionFailureTime = System.currentTimeMillis(); + nextReconnectDelayMs = + Math.min( + (long) (nextReconnectDelayMs * BACKOFF_MULTIPLIER), + MAX_RECONNECT_DELAY_MS); + LOG.warn("Connection failure detected. Will retry in {} ms", nextReconnectDelayMs); + } + + private boolean ensureConnected() throws IOException { + try { + if (!connectionInitialized || connection == null || !connection.isOpen()) { + LOG.info( + "Connection not available, attempting to reconnect after {} ms", + nextReconnectDelayMs); + + long timeSinceLastFailure = + System.currentTimeMillis() - lastConnectionFailureTime; + if (lastConnectionFailureTime > 0 + && timeSinceLastFailure < nextReconnectDelayMs) { + return false; + } + + cleanupConnection(); + initializeConnection(); + + if (connection != null && connection.isOpen()) { + try { + commands.ping(); + LOG.info("Redis connection re-established and verified"); + return true; + } catch (Exception e) { + LOG.error("Redis health check failed after reconnect", e); + cleanupConnection(); + handleConnectionFailure(); + return false; + } + } + return false; + } + return true; + } catch (Exception e) { + LOG.error("Error checking connection status", e); + cleanupConnection(); + handleConnectionFailure(); + return false; + } + } + + @Override + public RecordsWithSplitIds<StreamMessage<String, String>> fetch() throws IOException { + if (wokenUp) { + wokenUp = false; + return new RedisStreamsRecords(Collections.emptyMap(), Collections.emptySet()); + } + + Map<String, RedisStreamsSourceSplit> splitsToFetch; + stateLock.lock(); + try { + if (assignedSplits.isEmpty()) { + return new RedisStreamsRecords( + Collections.emptyMap(), Collections.emptySet()); + } + + splitsToFetch = new HashMap<>(); + for (Map.Entry<String, RedisStreamsSourceSplit> entry : + assignedSplits.entrySet()) { + String splitId = entry.getKey(); + + if (pausedSplits.contains(splitId)) { + continue; + } + + CircuitBreaker breaker = splitCircuitBreakers.get(splitId); + if (breaker != null && breaker.isOpen()) { + LOG.debug( + "Circuit breaker open for split {}, skipping fetch", splitId); + continue; + } + + LinkedList<String> deferred = deferredAcks.get(splitId); + if (deferred != null + && deferred.size() >= config.getMaxDeferredAckQueueSize()) { + LOG.debug( + "Deferred ACK queue for split {} at capacity ({})", + splitId, + config.getMaxDeferredAckQueueSize()); + continue; + } + + splitsToFetch.put(splitId, entry.getValue()); + } + } finally { + stateLock.unlock(); + } + + if (splitsToFetch.isEmpty()) { + return new RedisStreamsRecords( + Collections.emptyMap(), Collections.emptySet()); + } + + if (!ensureConnected()) { + throw new IOException( + "Redis connection unavailable. Will retry in " + + nextReconnectDelayMs + + "ms"); + } + + Map<String, Collection<StreamMessage<String, String>>> recordsBySplit = new HashMap<>(); + Set<String> finishedSplitsInFetch = new HashSet<>(); + + for (Map.Entry<String, RedisStreamsSourceSplit> entry : splitsToFetch.entrySet()) { + if (wokenUp) { + break; + } + + String splitId = entry.getKey(); + RedisStreamsSourceSplit split = entry.getValue(); + + try { + List<StreamMessage<String, String>> messages = fetchFromSplit(split); + + if (!messages.isEmpty()) { + recordsBySplit.put(splitId, messages); + } else if (config.isBounded()) { + finishedSplitsInFetch.add(splitId); + LOG.info("Split {} finished in bounded mode", splitId); + } + } catch (RedisConnectionException e) { + LOG.warn( + "Redis connection error for split {}: {}", + splitId, + e.getMessage()); + handleConnectionFailure(); + splitCircuitBreakers + .computeIfAbsent(splitId, k -> new CircuitBreaker()) + .recordFailure(); + } catch (Exception e) { + LOG.error( + "Unexpected error fetching from Redis Stream split {}", + splitId, + e); + splitCircuitBreakers + .computeIfAbsent(splitId, k -> new CircuitBreaker()) + .recordFailure(); + } + } + + stateLock.lock(); + try { + for (Map.Entry<String, Collection<StreamMessage<String, String>>> entry : + recordsBySplit.entrySet()) { + String splitId = entry.getKey(); + Collection<StreamMessage<String, String>> records = entry.getValue(); + + if (!records.isEmpty()) { + LinkedList<String> deferred = + deferredAcks.computeIfAbsent(splitId, k -> new LinkedList<>()); + String lastId = null; + for (StreamMessage<String, String> msg : records) { + deferred.addLast(msg.getId()); + lastId = msg.getId(); + } + + RedisStreamsSourceSplit currentSplit = assignedSplits.get(splitId); + if (currentSplit != null && lastId != null) { + assignedSplits.put( + splitId, currentSplit.withLastReadEntryId(lastId)); + } + + CircuitBreaker breaker = splitCircuitBreakers.get(splitId); + if (breaker != null) { + breaker.recordSuccess(); + } + + LOG.debug( + "Fetched {} messages from split {}, last ID: {}, " + + "deferred queue size: {}", + records.size(), + splitId, + lastId, + deferred.size()); + } + } + + for (String splitId : finishedSplitsInFetch) { + assignedSplits.remove(splitId); + finishedSplits.add(splitId); + } + } finally { + stateLock.unlock(); + } + + return new RedisStreamsRecords(recordsBySplit, finishedSplitsInFetch); + } + + private List<StreamMessage<String, String>> fetchFromSplit(RedisStreamsSourceSplit split) + throws Exception { + String streamKey = split.getStreamKey(); + Consumer<String> consumer = + Consumer.from(config.getConsumerGroup(), config.getConsumerName()); + + XReadArgs readArgs = + XReadArgs.Builder.block(config.getPollTimeout()) + .count(config.getBatchSize()); + + List<StreamMessage<String, String>> messages = + commands.xreadgroup( + consumer, + readArgs, + XReadArgs.StreamOffset.lastConsumed(streamKey)); Review Comment: Excellent catch — this was a real bug. When consumerName is shared across parallel subtasks, Redis consumer groups treat all connections as one consumer, defeating the purpose of parallel reads. Fixed by appending -subtaskId to the consumer name in RedisStreamsSplitReader.fetchFromSplit(). The subtaskId is now passed from SourceReaderContext.getIndexOfSubtask() through RedisStreamsSource.createReader() → RedisStreamsSplitReader constructor. This ensures each subtask gets a unique consumer identity within the consumer group (e.g., flink-consumer-abc12345-0, flink-consumer-abc12345-1). ########## flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/config/RedisStreamsSourceConfig.java: ########## @@ -0,0 +1,257 @@ +/* + * 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.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +/** Configuration for the Redis Streams Source. */ +@PublicEvolving +public class RedisStreamsSourceConfig implements Serializable { + + private static final long serialVersionUID = 1L; + public static final String DEFAULT_CONSUMER_GROUP = "flink-consumer-group"; + + private final String host; + private final int port; + private final String password; + private final int database; + private final List<String> streamKeys; + private final String consumerGroup; + private final String consumerName; + private final boolean bounded; + private final long pollTimeout; + private final int batchSize; + private final long discoveryInterval; + private final int maxDeferredAckQueueSize; + private final long circuitBreakerOpenDurationMs; + private final int circuitBreakerFailureThreshold; + + private RedisStreamsSourceConfig( + String host, + int port, + String password, + int database, + List<String> streamKeys, + String consumerGroup, + String consumerName, + boolean bounded, + long pollTimeout, + int batchSize, + long discoveryInterval, + int maxDeferredAckQueueSize, + long circuitBreakerOpenDurationMs, + int circuitBreakerFailureThreshold) { + this.host = host; + this.port = port; + this.password = password; + this.database = database; + this.streamKeys = Collections.unmodifiableList(new ArrayList<>(streamKeys)); + this.consumerGroup = consumerGroup; + this.consumerName = consumerName; + this.bounded = bounded; + this.pollTimeout = pollTimeout; + this.batchSize = batchSize; + this.discoveryInterval = discoveryInterval; + this.maxDeferredAckQueueSize = maxDeferredAckQueueSize; + this.circuitBreakerOpenDurationMs = circuitBreakerOpenDurationMs; + this.circuitBreakerFailureThreshold = circuitBreakerFailureThreshold; + } + + public String getHost() { + return host; + } + + public int getPort() { + return port; + } + + public String getPassword() { + return password; + } + + public int getDatabase() { + return database; + } + + public List<String> getStreamKeys() { + return streamKeys; + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public String getConsumerName() { + return consumerName; + } + + public boolean isBounded() { + return bounded; + } + + public long getPollTimeout() { + return pollTimeout; + } + + public int getBatchSize() { + return batchSize; + } + + public long getDiscoveryInterval() { + return discoveryInterval; + } + + public int getMaxDeferredAckQueueSize() { + return maxDeferredAckQueueSize; + } + + public long getCircuitBreakerOpenDurationMs() { + return circuitBreakerOpenDurationMs; + } + + public int getCircuitBreakerFailureThreshold() { + return circuitBreakerFailureThreshold; + } + + public static Builder builder() { + return new Builder(); + } + + /** Builder for RedisStreamsSourceConfig. */ + public static class Builder { + private String host = "localhost"; + private int port = 6379; + private String password = null; + private int database = 0; + private List<String> streamKeys; + private String consumerGroup = DEFAULT_CONSUMER_GROUP; + private String consumerName; + private boolean bounded = false; + private long pollTimeout = 1000; + private int batchSize = 100; + private long discoveryInterval = 60000; + private int maxDeferredAckQueueSize = 10000; + private long circuitBreakerOpenDurationMs = 5000; + private int circuitBreakerFailureThreshold = 3; + + public Builder setHost(String host) { + this.host = host; + return this; + } + + public Builder setPort(int port) { + this.port = port; + return this; + } + + public Builder setPassword(String password) { + this.password = password; + return this; + } + + public Builder setDatabase(int database) { + this.database = database; + return this; + } + + public Builder setStreamKeys(List<String> streamKeys) { + this.streamKeys = streamKeys; + return this; + } + + public Builder setConsumerGroup(String consumerGroup) { + this.consumerGroup = consumerGroup; + return this; + } + + public Builder setConsumerName(String consumerName) { + this.consumerName = consumerName; + return this; + } + + public Builder setBounded(boolean bounded) { + this.bounded = bounded; + return this; + } + + public Builder setPollTimeout(long pollTimeout) { + this.pollTimeout = pollTimeout; + return this; + } + + public Builder setBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public Builder setDiscoveryInterval(long discoveryInterval) { + this.discoveryInterval = discoveryInterval; + return this; + } + + public Builder setMaxDeferredAckQueueSize(int maxDeferredAckQueueSize) { + this.maxDeferredAckQueueSize = maxDeferredAckQueueSize; + return this; + } + + public Builder setCircuitBreakerOpenDurationMs(long circuitBreakerOpenDurationMs) { + this.circuitBreakerOpenDurationMs = circuitBreakerOpenDurationMs; + return this; + } + + public Builder setCircuitBreakerFailureThreshold(int circuitBreakerFailureThreshold) { + this.circuitBreakerFailureThreshold = circuitBreakerFailureThreshold; + return this; + } + + public RedisStreamsSourceConfig build() { Review Comment: Excellent catch — this was a real bug. When consumerName is shared across parallel subtasks, Redis consumer groups treat all connections as one consumer, defeating the purpose of parallel reads. Fixed by appending -subtaskId to the consumer name in RedisStreamsSplitReader.fetchFromSplit(). The subtaskId is now passed from SourceReaderContext.getIndexOfSubtask() through RedisStreamsSource.createReader() → RedisStreamsSplitReader constructor. This ensures each subtask gets a unique consumer identity within the consumer group (e.g., flink-consumer-abc12345-0, flink-consumer-abc12345-1). -- 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]
