Poorvankbhatia commented on code in PR #6: URL: https://github.com/apache/flink-connector-redis-streams/pull/6#discussion_r3008003853
########## flink-connector-redis-streams/src/main/java/org/apache/flink/connector/redis/streams/source/split/RedisStreamsSourceSplit.java: ########## @@ -0,0 +1,101 @@ +/* Review Comment: I have a fundamental design concern: this connector maps one split to one Redis stream key, which limits parallelism within a single stream. In this model, the unit of parallelism is the number of distinct stream keys configured. If you configure ["orders-stream"] with job parallelism set to 8, exactly one TaskManager reads orders-stream, and the other 7 sit idle. Redis Streams consumer groups are specifically designed to solve this. Now I agree, adding consumer-group parallelism breaks Redis's total stream ordering — each consumer's subset is ordered, but cross-consumer ordering is lost. A parallelismPerStream config (default 1) should let users choose explicitly. ########## 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 Review Comment: Should be marked `@Internal`, not `@PublicEvolving`. ########## 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 { Review Comment: IMO implements Serializable is not required. Splits are serialized by RedisStreamsSourceSplitSerializer, not Java serialization ########## 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; Review Comment: This can also be removed. -- 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]
