xintongsong commented on code in PR #23927: URL: https://github.com/apache/flink/pull/23927#discussion_r1436062729
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionIndexRange.java: ########## @@ -0,0 +1,102 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.runtime.executiongraph.IndexRange; + +import java.util.Iterator; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A {@link ResultSubpartitionIndexSet} represented as a range of indexes. The range is inclusive. + */ +public class ResultSubpartitionIndexRange implements ResultSubpartitionIndexSet { Review Comment: How is this different from `org.apache.flink.runtime.executiongraph`? Can we extract a common abstract from these two? ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageConsumerClient.java: ########## @@ -52,33 +55,39 @@ public class TieredStorageConsumerClient { Map<TieredStorageSubpartitionId, Tuple2<TierConsumerAgent, Integer>>> currentConsumerAgentAndSegmentIds = new HashMap<>(); + private final List<TieredStorageConsumerSpec> tieredStorageConsumerSpecs; + public TieredStorageConsumerClient( List<TierFactory> tierFactories, List<TieredStorageConsumerSpec> tieredStorageConsumerSpecs, TieredStorageNettyService nettyService) { this.tierFactories = tierFactories; this.nettyService = nettyService; this.tierConsumerAgents = createTierConsumerAgents(tieredStorageConsumerSpecs); + this.tieredStorageConsumerSpecs = tieredStorageConsumerSpecs; } public void start() { for (TierConsumerAgent tierConsumerAgent : tierConsumerAgents) { tierConsumerAgent.start(); + for (TieredStorageConsumerSpec spec : tieredStorageConsumerSpecs) { + tierConsumerAgent.notifyRequiredSegmentId( + spec.getPartitionId(), spec.getSubpartitionId(), 0); + } Review Comment: Why notifying for all sub-partitions at starting? ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReader.java: ########## @@ -19,18 +19,26 @@ package org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId; import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierConsumerAgent; import java.util.Optional; /** {@link NettyConnectionReader} is used by {@link TierConsumerAgent} to read buffer from netty. */ public interface NettyConnectionReader { + /** + * Notify the upstream the id of required segment that should be sent to netty connection. + * + * @param subpartitionId The id of the corresponding subpartition. + * @param segmentId The id of required segment. + */ + void notifyRequiredSegmentId(TieredStorageSubpartitionId subpartitionId, int segmentId); Review Comment: Could you remind me why do we need to separate the notify from readBuffer? It feels like this notify interface is used for multiple purposes. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionIndexSet.java: ########## @@ -0,0 +1,28 @@ +/* + * 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.runtime.io.network.partition; + +/** A collection of subpartition indexes. */ +public interface ResultSubpartitionIndexSet { Review Comment: BTW, `getStartIndex()` is called in `StatusWatermarkValve` anyway, indicating that the implementation of this interface is not really transparent to the callers. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java: ########## @@ -860,7 +868,8 @@ private Optional<Buffer> readBufferFromInputChannel(InputChannel inputChannel) return Optional.of(bufferAndAvailability.buffer()); } - private Optional<Buffer> readBufferFromTieredStore(InputChannel inputChannel) { + private Optional<Buffer> readBufferFromTieredStore(InputChannel inputChannel) + throws IOException { Review Comment: This change seems unrelated to this commit. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierConsumerAgent.java: ########## @@ -73,20 +73,35 @@ public void start() { } @Override - public Optional<Buffer> getNextBuffer( + public void notifyRequiredSegmentId( TieredStoragePartitionId partitionId, TieredStorageSubpartitionId subpartitionId, int segmentId) { - // Get current segment id and buffer index. Tuple2<Integer, Integer> bufferIndexAndSegmentId = currentBufferIndexAndSegmentIds .computeIfAbsent(partitionId, ignore -> new HashMap<>()) .getOrDefault(subpartitionId, Tuple2.of(0, -1)); int currentBufferIndex = bufferIndexAndSegmentId.f0; int currentSegmentId = bufferIndexAndSegmentId.f1; + if (segmentId != currentSegmentId) { remoteStorageScanner.watchSegment(partitionId, subpartitionId, segmentId); + currentBufferIndexAndSegmentIds + .get(partitionId) + .put(subpartitionId, Tuple2.of(currentBufferIndex, segmentId)); Review Comment: Why do we need this? ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionIndexSet.java: ########## @@ -0,0 +1,28 @@ +/* + * 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.runtime.io.network.partition; + +/** A collection of subpartition indexes. */ +public interface ResultSubpartitionIndexSet { Review Comment: Why is this an interface? I understand this can be implemented differently in future. But I don't think we should provide multiple different implementations at the same time. On the other hand, making this a class would simplify a lot of things: no need for a separate util class for SerDes, no need for the unsupported errors, no need to make getStart/EndIndex public, etc. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierConsumerAgent.java: ########## @@ -40,21 +41,30 @@ public class MemoryTierConsumerAgent implements TierConsumerAgent { private final Map< TieredStoragePartitionId, - Map<TieredStorageSubpartitionId, CompletableFuture<NettyConnectionReader>>> + Map<TieredStorageInputChannelId, CompletableFuture<NettyConnectionReader>>> Review Comment: Not sure that tiers should be aware of the channels. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org