TanYuxin-tyx commented on code in PR #22733: URL: https://github.com/apache/flink/pull/22733#discussion_r1233875486
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierFactory.java: ########## @@ -28,7 +30,9 @@ import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierMasterAgent; import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent; -/** The implementation of {@link TierFactory} for memory tier. */ +import java.util.List; + +/** The memory tier factory implementation for {@link TierFactory}. */ Review Comment: The doc need not be modified. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierConsumerAgent.java: ########## @@ -0,0 +1,82 @@ +/* + * 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.api.java.tuple.Tuple2; +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.HashMap; +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 { Review Comment: The class need not be public, package-level access is enough. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java: ########## @@ -266,6 +280,12 @@ public SingleInputGate( this.throughputCalculator = checkNotNull(throughputCalculator); this.tieredStorageConsumerClient = tieredStorageConsumerClient; + this.tieredStoragePartitionIds = tieredStoragePartitionIds; + this.tieredStorageSubpartitionIds = tieredStorageSubpartitionIds; + if (enabledTieredStore()) { Review Comment: Through `enabledTieredStore` is introduced in the previous change, I suggest changing the method to `enabledTieredStorage` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java: ########## @@ -895,6 +898,52 @@ public String toString() { } } + /** Message to notify producer about the id of required segment. */ + static class SegmentId extends NettyMessage { + + private static final byte ID = 11; + + final int segmentId; + + final InputChannelID receiverId; + + SegmentId(int segmentId, InputChannelID receiverId) { + checkArgument(segmentId > 0L, "The segmentId should be greater than 0"); + this.segmentId = segmentId; + this.receiverId = receiverId; + } + + @Override + void write(ChannelOutboundInvoker out, ChannelPromise promise, ByteBufAllocator allocator) + throws IOException { + ByteBuf result = null; + + try { + result = + allocateBuffer( + allocator, ID, Integer.BYTES + InputChannelID.getByteBufLength()); + result.writeInt(segmentId); + receiverId.writeTo(result); + + out.write(result, promise); + } catch (Throwable t) { + handleException(result, null, t); + } + } + + static SegmentId readFrom(ByteBuf buffer) { + int segmentId = buffer.readInt(); + InputChannelID receiverId = InputChannelID.fromByteBuf(buffer); + + return new SegmentId(segmentId, receiverId); + } + + @Override + public String toString() { + return String.format("ContainSegment(%s : %d)", receiverId, segmentId); Review Comment: ```suggestion return String.format("SegmentId(%s : %d)", receiverId, segmentId); ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TestingNettyConnectionReader.java: ########## @@ -0,0 +1,56 @@ +/* + * 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.netty; + +import org.apache.flink.runtime.io.network.buffer.Buffer; + +import java.util.Optional; +import java.util.function.Function; + +/** Test implementation for {@link NettyConnectionReader}. */ +public class TestingNettyConnectionReader implements NettyConnectionReader { + + private final Function<Integer, Buffer> readBufferFunction; + + private TestingNettyConnectionReader(Function<Integer, Buffer> readBufferFunction) { + this.readBufferFunction = readBufferFunction; + } + + @Override + public Optional<Buffer> readBuffer(int segmentId) { + return Optional.of(readBufferFunction.apply(segmentId)); + } + + /** Builder for {@link TestingNettyConnectionReader}. */ + public static class Builder { + + private Function<Integer, Buffer> readBufferFunction; Review Comment: Use a default value to init the `readBufferFunction`, which can simplify the usage. ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageConsumerClientTest.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.netty; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageIdMappingUtils; +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.storage.TieredStorageConsumerClient; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.memory.MemoryTierFactory; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link TieredStorageConsumerClient}. */ +public class TieredStorageConsumerClientTest { Review Comment: This need not be public. ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageConsumerClientTest.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.netty; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageIdMappingUtils; +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.storage.TieredStorageConsumerClient; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.memory.MemoryTierFactory; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link TieredStorageConsumerClient}. */ +public class TieredStorageConsumerClientTest { + + private final TieredStoragePartitionId partitionId = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + + private final TieredStorageSubpartitionId subpartitionId = new TieredStorageSubpartitionId(0); + + @Test + void testGetNextBufferFromMemoryTier() { Review Comment: When testing consumer client, this test need not be aware of the tier implementation. I think we can add a `TestingTierFactory` to test this. Then rename the test method to `testGetNextBuffer`. -- 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