reswqa commented on code in PR #22728: URL: https://github.com/apache/flink/pull/22728#discussion_r1227508536
########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TestingNettyConnectionWriter.java: ########## @@ -0,0 +1,109 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.function.Supplier; + +/** Test implementation for {@link NettyConnectionWriter}. */ +public class TestingNettyConnectionWriter implements NettyConnectionWriter { + + private final Runnable writeBufferFunction; + + private final Supplier<NettyConnectionId> nettyConnectionIdSupplier; + + private final Supplier<Integer> numQueuedBuffersSupplier; + + private final Runnable closeFunction; + + private TestingNettyConnectionWriter( + Runnable writeBufferFunction, + Supplier<NettyConnectionId> nettyConnectionIdSupplier, + Supplier<Integer> numQueuedBuffersSupplier, + Runnable closeFunction) { + this.writeBufferFunction = writeBufferFunction; + this.nettyConnectionIdSupplier = nettyConnectionIdSupplier; + this.numQueuedBuffersSupplier = numQueuedBuffersSupplier; + this.closeFunction = closeFunction; + } + + @Override + public void writeBuffer(NettyPayload nettyPayload) { + writeBufferFunction.run(); Review Comment: Why not use a `Function` instead use a `Runnable` here. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/NoopMasterAgent.java: ########## @@ -0,0 +1,39 @@ +/* + * 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; + +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId; + +/** The noop implementation for {@link TierMasterAgent}. */ +public class NoopMasterAgent implements TierMasterAgent { Review Comment: ```suggestion public class NoOpMasterAgent implements TierMasterAgent { ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgent.java: ########## @@ -0,0 +1,175 @@ +/* + * 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.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.api.EndOfSegmentEvent; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +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.NettyConnectionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent; +import org.apache.flink.util.ExceptionUtils; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType.END_OF_SEGMENT; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The memory tier implementation of {@link TierProducerAgent}. */ +public class MemoryTierProducerAgent implements TierProducerAgent, NettyServiceProducer { + + private final int numBuffersPerSegment; + + private final TieredStorageMemoryManager memoryManager; + + /** + * Record the writ bytes to each subpartition. When starting a new segment, the value will be + * reset to 0. + */ + private final int[] currentSubpartitionWriteBuffers; + + /** + * Whether a subpartition's netty connection has been established. The array index is + * corresponding to the subpartition id. + */ + private final boolean[] nettyConnectionEstablished; + + private final MemoryTierSubpartitionProducerAgent[] subpartitionProducerAgents; + + public MemoryTierProducerAgent( + TieredStoragePartitionId partitionId, + int numSubpartitions, + int bufferSizeBytes, + int segmentSizeBytes, + boolean isBroadcastOnly, + TieredStorageMemoryManager memoryManager, + TieredStorageNettyService nettyService, + TieredStorageResourceRegistry resourceRegistry) { + checkArgument( + segmentSizeBytes >= bufferSizeBytes, "One segment contains at least one buffer."); Review Comment: ```suggestion segmentSizeBytes >= bufferSizeBytes, "One segment should contains at least one buffer."); ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyServiceImpl; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The subpartition producer agent for the memory tier. */ +class MemoryTierSubpartitionProducerAgent { + + private final int subpartitionId; + + private final TieredStorageNettyService nettyService; + + /** + * The {@link NettyConnectionWriter} is used to write buffers to the netty connection. + * + * <p>Note that this field can be null before the netty connection is established. + */ + @Nullable private NettyConnectionWriter nettyConnectionWriter; + + private int finishedBufferIndex; + + MemoryTierSubpartitionProducerAgent( + int subpartitionId, TieredStorageNettyService nettyService) { + this.subpartitionId = subpartitionId; + this.nettyService = nettyService; + } + + // ------------------------------------------------------------------------ + // Called by MemoryTierProducerAgent + // ------------------------------------------------------------------------ + + void connectionEstablished(NettyConnectionWriter nettyConnectionWriter) { + this.nettyConnectionWriter = nettyConnectionWriter; + } + + void addFinishedBuffer(Buffer buffer) { + NettyPayload toAddBuffer = + NettyPayload.newBuffer(buffer, finishedBufferIndex, subpartitionId); + addFinishedBuffer(toAddBuffer); + } + + void addSegmentId(int segmentId) { + NettyPayload segmentNettyPayload = NettyPayload.newSegment(segmentId); + addFinishedBuffer(segmentNettyPayload); + } + + void release() { + if (nettyConnectionWriter != null) { + nettyConnectionWriter.close(null); + } + } + + // ------------------------------------------------------------------------ + // Internal Methods + // ------------------------------------------------------------------------ + + private void addFinishedBuffer(NettyPayload nettyPayload) { + finishedBufferIndex++; + checkNotNull(nettyConnectionWriter).writeBuffer(nettyPayload); + if (nettyConnectionWriter.numQueuedBuffers() <= 1 + && nettyService.getClass() == TieredStorageNettyServiceImpl.class) { + ((TieredStorageNettyServiceImpl) nettyService) Review Comment: I wonder why `notifyResultSubpartitionViewSendBuffer` is not a interface method. If method about `subpartitionView` is not belong to the protocol of `TieredStorageNettyService`, we can consider let `TieredStorageNettyServiceImpl` implementing another interface at the same time. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyServiceImpl; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The subpartition producer agent for the memory tier. */ +class MemoryTierSubpartitionProducerAgent { + + private final int subpartitionId; + + private final TieredStorageNettyService nettyService; + + /** + * The {@link NettyConnectionWriter} is used to write buffers to the netty connection. + * + * <p>Note that this field can be null before the netty connection is established. + */ + @Nullable private NettyConnectionWriter nettyConnectionWriter; + + private int finishedBufferIndex; + + MemoryTierSubpartitionProducerAgent( + int subpartitionId, TieredStorageNettyService nettyService) { + this.subpartitionId = subpartitionId; + this.nettyService = nettyService; + } + + // ------------------------------------------------------------------------ + // Called by MemoryTierProducerAgent + // ------------------------------------------------------------------------ + + void connectionEstablished(NettyConnectionWriter nettyConnectionWriter) { + this.nettyConnectionWriter = nettyConnectionWriter; + } + + void addFinishedBuffer(Buffer buffer) { + NettyPayload toAddBuffer = + NettyPayload.newBuffer(buffer, finishedBufferIndex, subpartitionId); + addFinishedBuffer(toAddBuffer); + } + + void addSegmentId(int segmentId) { Review Comment: I'd prefer rename this to `updateSegmentId` or `increaseSegmentId`. ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isTrue(); + } + + @Test + void testTryWrite() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isTrue(); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isFalse(); + } + + @Test + void testBroadcastOnlyPartitionCanNotUseMemoryTier() { + assertThatThrownBy(() -> createMemoryTierProducerAgent(true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("not allowed to use the memory tier"); + } + + @Test + void testRelease() { + TieredStorageResourceRegistry resourceRegistry = new TieredStorageResourceRegistry(); + MemoryTierProducerAgent memoryTierProducerAgent = + createMemoryTierProducerAgent(false, resourceRegistry); + + AtomicBoolean isClosed = new AtomicBoolean(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, + new TestingNettyConnectionWriter.Builder() + .setCloseFunction(() -> isClosed.set(true)) + .build()); + + resourceRegistry.clearResourceFor(PARTITION_ID); + assertThat(isClosed.get()).isTrue(); + } + + private static MemoryTierProducerAgent createMemoryTierProducerAgent(boolean isBroadcast) { Review Comment: ```suggestion private static MemoryTierProducerAgent createMemoryTierProducerAgent(boolean isBroadcastOnly) { ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isTrue(); + } + + @Test + void testTryWrite() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isTrue(); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isFalse(); + } + + @Test + void testBroadcastOnlyPartitionCanNotUseMemoryTier() { + assertThatThrownBy(() -> createMemoryTierProducerAgent(true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("not allowed to use the memory tier"); + } + + @Test + void testRelease() { + TieredStorageResourceRegistry resourceRegistry = new TieredStorageResourceRegistry(); + MemoryTierProducerAgent memoryTierProducerAgent = + createMemoryTierProducerAgent(false, resourceRegistry); + + AtomicBoolean isClosed = new AtomicBoolean(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, + new TestingNettyConnectionWriter.Builder() + .setCloseFunction(() -> isClosed.set(true)) + .build()); + + resourceRegistry.clearResourceFor(PARTITION_ID); + assertThat(isClosed.get()).isTrue(); Review Comment: ```suggestion assertThat(isClosed).isTrue(); ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isTrue(); + } + + @Test + void testTryWrite() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isTrue(); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isFalse(); Review Comment: This strongly rely on `BUFFER_SIZE = SEGMENT_SIZE_BYTES = 1024`, this will make this test case a bit obscure. I suggest setting the default `SEGMENT_SIZE_BYTES` in the class to be greater than `BUFFER_SIZE`(e.g. `BUFFER_SIZE * 2`) and introducing `createMemoryTierProducerAgent (boolean isBroadcast, int segmentSizeBytes)`. As for this test, we can call `createMemoryTierProducerAgent(false, BUFFER_SIZE)` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/NoopMasterAgent.java: ########## @@ -0,0 +1,39 @@ +/* + * 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; + +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId; + +/** The noop implementation for {@link TierMasterAgent}. */ Review Comment: ```suggestion /** The empty implementation for {@link TierMasterAgent}. */ ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgent.java: ########## @@ -0,0 +1,175 @@ +/* + * 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.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.api.EndOfSegmentEvent; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +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.NettyConnectionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent; +import org.apache.flink.util.ExceptionUtils; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType.END_OF_SEGMENT; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The memory tier implementation of {@link TierProducerAgent}. */ +public class MemoryTierProducerAgent implements TierProducerAgent, NettyServiceProducer { + + private final int numBuffersPerSegment; + + private final TieredStorageMemoryManager memoryManager; + + /** + * Record the writ bytes to each subpartition. When starting a new segment, the value will be + * reset to 0. + */ + private final int[] currentSubpartitionWriteBuffers; + + /** + * Whether a subpartition's netty connection has been established. The array index is + * corresponding to the subpartition id. + */ + private final boolean[] nettyConnectionEstablished; + + private final MemoryTierSubpartitionProducerAgent[] subpartitionProducerAgents; + + public MemoryTierProducerAgent( + TieredStoragePartitionId partitionId, + int numSubpartitions, + int bufferSizeBytes, + int segmentSizeBytes, + boolean isBroadcastOnly, + TieredStorageMemoryManager memoryManager, + TieredStorageNettyService nettyService, + TieredStorageResourceRegistry resourceRegistry) { + checkArgument( + segmentSizeBytes >= bufferSizeBytes, "One segment contains at least one buffer."); + checkArgument( + !isBroadcastOnly, Review Comment: I'm not sure if memory tier can used when partition reuse enable. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierFactory.java: ########## @@ -0,0 +1,74 @@ +/* + * 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.partition.hybrid.tiered.common.TieredStoragePartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.NoopMasterAgent; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierConsumerAgent; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierFactory; +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 memory tier factory implementation for {@link TierFactory}. */ Review Comment: ```suggestion /** The implementation of {@link TierFactory} for memory tier. */ ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TestingNettyConnectionWriter.java: ########## @@ -0,0 +1,109 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.function.Supplier; + +/** Test implementation for {@link NettyConnectionWriter}. */ +public class TestingNettyConnectionWriter implements NettyConnectionWriter { + + private final Runnable writeBufferFunction; + + private final Supplier<NettyConnectionId> nettyConnectionIdSupplier; + + private final Supplier<Integer> numQueuedBuffersSupplier; + + private final Runnable closeFunction; + + private TestingNettyConnectionWriter( + Runnable writeBufferFunction, + Supplier<NettyConnectionId> nettyConnectionIdSupplier, + Supplier<Integer> numQueuedBuffersSupplier, + Runnable closeFunction) { + this.writeBufferFunction = writeBufferFunction; + this.nettyConnectionIdSupplier = nettyConnectionIdSupplier; + this.numQueuedBuffersSupplier = numQueuedBuffersSupplier; + this.closeFunction = closeFunction; + } + + @Override + public void writeBuffer(NettyPayload nettyPayload) { + writeBufferFunction.run(); + } + + @Override + public NettyConnectionId getNettyConnectionId() { + return nettyConnectionIdSupplier.get(); + } + + @Override + public int numQueuedBuffers() { + return numQueuedBuffersSupplier.get(); + } + + @Override + public void close(@Nullable Throwable error) { + closeFunction.run(); Review Comment: Why not use a Function instead use a Runnable here. ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( Review Comment: Maybe we should also test another condition: ``` memoryManager.getMaxNonReclaimableBuffers(this) - memoryManager.numOwnerRequestedBuffer(this)) > numBuffersPerSegment; ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); Review Comment: Please using `try-with-resource` pattern. ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isTrue(); + } + + @Test + void testTryWrite() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isTrue(); + assertThat( + memoryTierProducerAgent.tryWrite( + SUBPARTITION_ID, BufferBuilderTestUtils.buildSomeBuffer())) + .isFalse(); + } + + @Test + void testBroadcastOnlyPartitionCanNotUseMemoryTier() { + assertThatThrownBy(() -> createMemoryTierProducerAgent(true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("not allowed to use the memory tier"); + } + + @Test + void testRelease() { + TieredStorageResourceRegistry resourceRegistry = new TieredStorageResourceRegistry(); + MemoryTierProducerAgent memoryTierProducerAgent = + createMemoryTierProducerAgent(false, resourceRegistry); + + AtomicBoolean isClosed = new AtomicBoolean(false); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, + new TestingNettyConnectionWriter.Builder() + .setCloseFunction(() -> isClosed.set(true)) + .build()); + + resourceRegistry.clearResourceFor(PARTITION_ID); + assertThat(isClosed.get()).isTrue(); + } + + private static MemoryTierProducerAgent createMemoryTierProducerAgent(boolean isBroadcast) { + return createMemoryTierProducerAgent(isBroadcast, new TieredStorageResourceRegistry()); + } + + private static MemoryTierProducerAgent createMemoryTierProducerAgent( + boolean isBroadcast, TieredStorageResourceRegistry resourceRegistry) { + TestingTieredStorageMemoryManager memoryManager = + new TestingTieredStorageMemoryManager.Builder() + .setGetMaxNonReclaimableBuffersFunction(ignore -> Integer.MAX_VALUE) + .build(); + TestingTieredStorageNettyService nettyService = + new TestingTieredStorageNettyService.Builder().build(); + TestingNettyServiceProducer nettyServiceProducer = + new TestingNettyServiceProducer.Builder().build(); + nettyService.registerProducer(PARTITION_ID, nettyServiceProducer); + + return new MemoryTierProducerAgent( + PARTITION_ID, + NUM_SUBPARTITIONS, + BUFFER_SIZE, + SEGMENT_SIZE_BYTES, + isBroadcast, Review Comment: ```suggestion isBroadcastOnly, ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyServiceImpl; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The subpartition producer agent for the memory tier. */ +class MemoryTierSubpartitionProducerAgent { + + private final int subpartitionId; + + private final TieredStorageNettyService nettyService; + + /** + * The {@link NettyConnectionWriter} is used to write buffers to the netty connection. + * + * <p>Note that this field can be null before the netty connection is established. + */ + @Nullable private NettyConnectionWriter nettyConnectionWriter; + + private int finishedBufferIndex; + + MemoryTierSubpartitionProducerAgent( + int subpartitionId, TieredStorageNettyService nettyService) { + this.subpartitionId = subpartitionId; + this.nettyService = nettyService; + } + + // ------------------------------------------------------------------------ + // Called by MemoryTierProducerAgent + // ------------------------------------------------------------------------ + + void connectionEstablished(NettyConnectionWriter nettyConnectionWriter) { + this.nettyConnectionWriter = nettyConnectionWriter; + } + + void addFinishedBuffer(Buffer buffer) { + NettyPayload toAddBuffer = + NettyPayload.newBuffer(buffer, finishedBufferIndex, subpartitionId); + addFinishedBuffer(toAddBuffer); + } + + void addSegmentId(int segmentId) { + NettyPayload segmentNettyPayload = NettyPayload.newSegment(segmentId); + addFinishedBuffer(segmentNettyPayload); + } + + void release() { + if (nettyConnectionWriter != null) { Review Comment: I wonder how we can ensure that the visibility of `nettyConnectionWriter` here. 🤔 ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgentTest.java: ########## @@ -0,0 +1,127 @@ +/* + * 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.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.TestingTieredStorageMemoryManager; +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.netty.TestingNettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingNettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TestingTieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link MemoryTierProducerAgent}. */ +class MemoryTierProducerAgentTest { + + private static final int NUM_SUBPARTITIONS = 10; + + private static final int BUFFER_SIZE = 1024; + + private static final int SEGMENT_SIZE_BYTES = 1024; + + private static final TieredStoragePartitionId PARTITION_ID = + TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); + private static final TieredStorageSubpartitionId SUBPARTITION_ID = + new TieredStorageSubpartitionId(0); + + @Test + void testTryStartNewSegment() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isFalse(); + memoryTierProducerAgent.connectionEstablished( + SUBPARTITION_ID, new TestingNettyConnectionWriter.Builder().build()); + assertThat(memoryTierProducerAgent.tryStartNewSegment(SUBPARTITION_ID, 0)).isTrue(); + } + + @Test + void testTryWrite() { + MemoryTierProducerAgent memoryTierProducerAgent = createMemoryTierProducerAgent(false); Review Comment: Please using `try-with-resource` pattern. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyServiceImpl; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The subpartition producer agent for the memory tier. */ +class MemoryTierSubpartitionProducerAgent { Review Comment: It seems like we haven't tested this class yet. -- 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]
