Poorvankbhatia commented on code in PR #6: URL: https://github.com/apache/flink-connector-redis-streams/pull/6#discussion_r3170226908
########## flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/enumerator/RedisStreamsSourceEnumerator.java: ########## @@ -0,0 +1,287 @@ +/* + * 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.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.redis.streams.source.config.RedisStreamsSourceConfig; +import org.apache.flink.connector.redis.streams.source.split.RedisStreamsSourceSplit; +import org.apache.flink.util.FlinkRuntimeException; + +import io.lettuce.core.ClientOptions; +import io.lettuce.core.RedisClient; +import io.lettuce.core.RedisURI; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.cluster.ClusterClientOptions; +import io.lettuce.core.cluster.RedisClusterClient; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +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.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +/** + * Enumerator for the Redis Streams Source. One split per stream key, round-robin assignment. + * Bounded mode freezes a stopping entry ID per key via {@code XINFO STREAM} at startup. + */ +@Internal +public class RedisStreamsSourceEnumerator + implements SplitEnumerator<RedisStreamsSourceSplit, RedisStreamsSourceEnumeratorState> { + + private static final Logger LOG = LoggerFactory.getLogger(RedisStreamsSourceEnumerator.class); + + private final RedisStreamsSourceConfig sourceConfig; + private final SplitEnumeratorContext<RedisStreamsSourceSplit> context; + + // SplitEnumerator callbacks are serialized by the SourceCoordinator event loop, so no + // synchronization on this state is required. + private final Set<String> pendingSplitKeys; + private final Map<Integer, Set<String>> readerAssignments = new HashMap<>(); + private final Set<Integer> readersSignaledNoMoreSplits = new HashSet<>(); + private final Map<String, String> stoppingEntryIds = new HashMap<>(); + private final Function<String, String> lastGeneratedIdLookup; + + public RedisStreamsSourceEnumerator( + RedisStreamsSourceConfig sourceConfig, + SplitEnumeratorContext<RedisStreamsSourceSplit> context, + @Nullable RedisStreamsSourceEnumeratorState restoredState) { + this(sourceConfig, context, restoredState, defaultLookup(sourceConfig)); + } + + @VisibleForTesting + RedisStreamsSourceEnumerator( + RedisStreamsSourceConfig sourceConfig, + SplitEnumeratorContext<RedisStreamsSourceSplit> context, + @Nullable RedisStreamsSourceEnumeratorState restoredState, + Function<String, String> lastGeneratedIdLookup) { + this.sourceConfig = sourceConfig; + this.context = context; + this.lastGeneratedIdLookup = lastGeneratedIdLookup; + this.pendingSplitKeys = new HashSet<>(); + if (restoredState != null) { + this.pendingSplitKeys.addAll(restoredState.getPendingSplits()); + this.stoppingEntryIds.putAll(restoredState.getStoppingEntryIds()); + } else { + this.pendingSplitKeys.addAll(sourceConfig.getStreamKeys()); + } + } + + @Override + public void start() { + LOG.info("Starting Redis Streams Source Enumerator (bounded={})", sourceConfig.isBounded()); + if (sourceConfig.isBounded()) { + for (String streamKey : sourceConfig.getStreamKeys()) { + if (stoppingEntryIds.containsKey(streamKey)) { + continue; // restored from checkpoint; do not re-query XINFO. + } + String stoppingId = lastGeneratedIdLookup.apply(streamKey); Review Comment: Each stream key opens its own Redis connection, runs one XINFO, and closes sequentially. So with 100 streams, that's 100 connect/disconnect cycles in the JM thread.? -- 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]
