xintongsong commented on code in PR #22733: URL: https://github.com/apache/flink/pull/22733#discussion_r1228923781
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierConsumerAgent.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.hybrid.tiered.tier.memory; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionReader; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierConsumerAgent; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** The data client is used to fetch data from memory tier. */ +public class MemoryTierConsumerAgent implements TierConsumerAgent { + private final List<CompletableFuture<NettyConnectionReader>> nettyConnectionReaders = + new ArrayList<>(); + + public MemoryTierConsumerAgent( + Map<TieredStoragePartitionId, TieredStorageSubpartitionId> + partitionIdAndSubpartitionIds, + TieredStorageNettyService nettyService) { + for (TieredStoragePartitionId storagePartitionId : partitionIdAndSubpartitionIds.keySet()) { + nettyConnectionReaders.add( + nettyService.registerConsumer( + storagePartitionId, + partitionIdAndSubpartitionIds.get(storagePartitionId))); + } + } + + @Override + public void start() { + // nothing to do. Review Comment: minor: `noop` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierConsumerAgent.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.hybrid.tiered.tier.memory; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionReader; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierConsumerAgent; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** The data client is used to fetch data from memory tier. */ +public class MemoryTierConsumerAgent implements TierConsumerAgent { + private final List<CompletableFuture<NettyConnectionReader>> nettyConnectionReaders = + new ArrayList<>(); + + public MemoryTierConsumerAgent( + Map<TieredStoragePartitionId, TieredStorageSubpartitionId> Review Comment: This doesn't seem right. There's no guarantee that we only read one subpartition from each partition. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageNettyServiceImpl.java: ########## @@ -170,18 +143,12 @@ public void setupInputChannels( for (int index = 0; index < partitionIds.size(); ++index) { TieredStoragePartitionId partitionId = partitionIds.get(index); TieredStorageSubpartitionId subpartitionId = subpartitionIds.get(index); - - registeredChannelIndexes - .computeIfAbsent(partitionId, ignored -> new HashMap<>()) - .put(subpartitionId, index); - - registeredInputChannelProviders - .computeIfAbsent(partitionId, ignored -> new HashMap<>()) - .put(subpartitionId, inputChannelProviders.get(index)); - - registeredNettyConnectionReaderAvailabilityAndPriorityHelpers - .computeIfAbsent(partitionId, ignored -> new HashMap<>()) - .put(subpartitionId, helper); + registeredNettyConnectionReaders + .remove(partitionId) + .remove(subpartitionId) + .complete( + new NettyConnectionReaderImpl( + index, inputChannelProviders.get(index), helper)); Review Comment: How do you ensure that the consumer is already registered? -- 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