xintongsong commented on code in PR #22728:
URL: https://github.com/apache/flink/pull/22728#discussion_r1225133162


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/NoopMasterAgent.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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:
   This can be singleton.



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.
+    private final int[] numSubpartitionEmitBytes;
+
+    /**
+     * Whether a subpartition has been registered to the netty service. The 
array index is
+     * corresponding to the subpartition id.
+     */
+    private final boolean[] nettyServiceRegistered;
+
+    private final MemoryTierSubpartitionProducerAgent[] 
subpartitionProducerAgents;
+
+    public MemoryTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int bufferSize,
+            int segmentSizeBytes,
+            boolean isBroadcastOnly,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageNettyService nettyService,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at 
least one buffer.");
+
+        this.segmentSizeBytes = segmentSizeBytes;
+        this.segmentSizeBuffers = segmentSizeBytes / bufferSize;
+        this.isBroadcastOnly = isBroadcastOnly;
+        this.memoryManager = memoryManager;
+        this.numSubpartitionEmitBytes = new int[numSubpartitions];
+        this.nettyServiceRegistered = new boolean[numSubpartitions];
+        this.subpartitionProducerAgents = new 
MemoryTierSubpartitionProducerAgent[numSubpartitions];
+
+        Arrays.fill(numSubpartitionEmitBytes, 0);
+        nettyService.registerProducer(partitionId, this);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionProducerAgents[subpartitionId] =
+                    new MemoryTierSubpartitionProducerAgent(subpartitionId, 
nettyService);
+        }
+        resourceRegistry.registerResource(partitionId, this::releaseResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        if (isBroadcastOnly) {
+            return false;
+        }
+        boolean canStartNewSegment =
+                isSubpartitionRegistered(subpartitionId)
+                        && (memoryManager.getMaxNonReclaimableBuffers(this)
+                                        - 
memoryManager.numOwnerRequestedBuffer(this))
+                                > segmentSizeBuffers;
+        if (canStartNewSegment) {
+            getSubpartitionProducerAgent(subpartitionId.getSubpartitionId())
+                    .addSegmentId(segmentId);
+        }
+        return canStartNewSegment;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
finishedBuffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (numSubpartitionEmitBytes[subpartitionIndex] != 0
+                && numSubpartitionEmitBytes[subpartitionIndex] + 
finishedBuffer.readableBytes()
+                        > segmentSizeBytes) {
+            appendEndOfSegmentEvent(subpartitionIndex);
+            numSubpartitionEmitBytes[subpartitionIndex] = 0;
+            return false;
+        }
+        numSubpartitionEmitBytes[subpartitionIndex] += 
finishedBuffer.readableBytes();
+        append(finishedBuffer, subpartitionIndex);
+        return true;
+    }
+
+    @Override
+    public void connectionEstablished(
+            TieredStorageSubpartitionId subpartitionId,
+            NettyConnectionWriter nettyConnectionWriter) {
+        if (isBroadcastOnly) {
+            throw new RuntimeException(
+                    "Broadcast-only result partition is not allowed to use the 
memory tier.");
+        }

Review Comment:
   Same here. Checkstate in constructor.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.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.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.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.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 java.nio.ByteBuffer;
+
+import static 
org.apache.flink.runtime.io.network.buffer.Buffer.DataType.END_OF_SEGMENT;
+
+/** The subpartition producer agent for the memory tier. */
+class MemoryTierSubpartitionProducerAgent {
+
+    private final int subpartitionId;
+
+    private final TieredStorageNettyService nettyService;
+
+    private NettyConnectionWriter nettyConnectionWriter;
+
+    private int finishedBufferIndex;
+
+    MemoryTierSubpartitionProducerAgent(
+            int subpartitionId, TieredStorageNettyService nettyService) {
+        this.subpartitionId = subpartitionId;
+        this.nettyService = nettyService;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Called by MemoryTierProducerAgent
+    // ------------------------------------------------------------------------
+
+    void registerNettyService(NettyConnectionWriter nettyConnectionWriter) {
+        this.nettyConnectionWriter = nettyConnectionWriter;
+    }
+
+    void addFinishedBuffer(Buffer buffer) {
+        NettyPayload toAddBuffer =
+                NettyPayload.newBuffer(buffer, finishedBufferIndex, 
subpartitionId);
+        addFinishedBuffer(toAddBuffer);
+    }
+
+    void addEndOfSegmentEvent(ByteBuffer event) {
+        MemorySegment data = MemorySegmentFactory.wrap(event.array());
+        Buffer buffer =
+                new NetworkBuffer(
+                        data, FreeingBufferRecycler.INSTANCE, END_OF_SEGMENT, 
data.size());
+        NettyPayload nettyPayload =
+                NettyPayload.newBuffer(buffer, finishedBufferIndex, 
subpartitionId);
+        addFinishedBuffer(nettyPayload);
+    }

Review Comment:
   It's weird that both `MemoryTierProducerAgent` and 
`MemoryTierSubpartitionProducerAgent` need to be aware of the details 
end-of-segment event. Can we prepare the event outside the subpartition agent?



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.
+    private final int[] numSubpartitionEmitBytes;
+
+    /**
+     * Whether a subpartition has been registered to the netty service. The 
array index is
+     * corresponding to the subpartition id.
+     */
+    private final boolean[] nettyServiceRegistered;
+
+    private final MemoryTierSubpartitionProducerAgent[] 
subpartitionProducerAgents;
+
+    public MemoryTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int bufferSize,
+            int segmentSizeBytes,
+            boolean isBroadcastOnly,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageNettyService nettyService,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at 
least one buffer.");
+
+        this.segmentSizeBytes = segmentSizeBytes;
+        this.segmentSizeBuffers = segmentSizeBytes / bufferSize;
+        this.isBroadcastOnly = isBroadcastOnly;
+        this.memoryManager = memoryManager;
+        this.numSubpartitionEmitBytes = new int[numSubpartitions];
+        this.nettyServiceRegistered = new boolean[numSubpartitions];
+        this.subpartitionProducerAgents = new 
MemoryTierSubpartitionProducerAgent[numSubpartitions];
+
+        Arrays.fill(numSubpartitionEmitBytes, 0);
+        nettyService.registerProducer(partitionId, this);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionProducerAgents[subpartitionId] =
+                    new MemoryTierSubpartitionProducerAgent(subpartitionId, 
nettyService);
+        }
+        resourceRegistry.registerResource(partitionId, this::releaseResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        if (isBroadcastOnly) {
+            return false;
+        }

Review Comment:
   For broadcast only result partition, the memory tier shouldn't be created at 
all. So this should be a check-state, and it should be mored to the constructor.



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.

Review Comment:
   JavaDoc



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;

Review Comment:
   Why do we need both of them?



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.
+    private final int[] numSubpartitionEmitBytes;
+
+    /**
+     * Whether a subpartition has been registered to the netty service. The 
array index is
+     * corresponding to the subpartition id.
+     */
+    private final boolean[] nettyServiceRegistered;
+
+    private final MemoryTierSubpartitionProducerAgent[] 
subpartitionProducerAgents;
+
+    public MemoryTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int bufferSize,
+            int segmentSizeBytes,
+            boolean isBroadcastOnly,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageNettyService nettyService,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at 
least one buffer.");
+
+        this.segmentSizeBytes = segmentSizeBytes;
+        this.segmentSizeBuffers = segmentSizeBytes / bufferSize;
+        this.isBroadcastOnly = isBroadcastOnly;
+        this.memoryManager = memoryManager;
+        this.numSubpartitionEmitBytes = new int[numSubpartitions];
+        this.nettyServiceRegistered = new boolean[numSubpartitions];
+        this.subpartitionProducerAgents = new 
MemoryTierSubpartitionProducerAgent[numSubpartitions];
+
+        Arrays.fill(numSubpartitionEmitBytes, 0);
+        nettyService.registerProducer(partitionId, this);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionProducerAgents[subpartitionId] =
+                    new MemoryTierSubpartitionProducerAgent(subpartitionId, 
nettyService);
+        }
+        resourceRegistry.registerResource(partitionId, this::releaseResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        if (isBroadcastOnly) {
+            return false;
+        }
+        boolean canStartNewSegment =
+                isSubpartitionRegistered(subpartitionId)
+                        && (memoryManager.getMaxNonReclaimableBuffers(this)
+                                        - 
memoryManager.numOwnerRequestedBuffer(this))
+                                > segmentSizeBuffers;
+        if (canStartNewSegment) {
+            getSubpartitionProducerAgent(subpartitionId.getSubpartitionId())
+                    .addSegmentId(segmentId);
+        }
+        return canStartNewSegment;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
finishedBuffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (numSubpartitionEmitBytes[subpartitionIndex] != 0
+                && numSubpartitionEmitBytes[subpartitionIndex] + 
finishedBuffer.readableBytes()
+                        > segmentSizeBytes) {
+            appendEndOfSegmentEvent(subpartitionIndex);
+            numSubpartitionEmitBytes[subpartitionIndex] = 0;
+            return false;
+        }
+        numSubpartitionEmitBytes[subpartitionIndex] += 
finishedBuffer.readableBytes();
+        append(finishedBuffer, subpartitionIndex);
+        return true;
+    }
+
+    @Override
+    public void connectionEstablished(
+            TieredStorageSubpartitionId subpartitionId,
+            NettyConnectionWriter nettyConnectionWriter) {
+        if (isBroadcastOnly) {
+            throw new RuntimeException(
+                    "Broadcast-only result partition is not allowed to use the 
memory tier.");
+        }
+        
this.subpartitionProducerAgents[subpartitionId.getSubpartitionId()].registerNettyService(
+                nettyConnectionWriter);
+        nettyServiceRegistered[subpartitionId.getSubpartitionId()] = true;
+    }
+
+    @Override
+    public void connectionBroken(NettyConnectionId connectionId) {
+        // noop
+    }
+
+    @Override
+    public void close() {
+        // noop

Review Comment:
   I think buffers in memory tier should be released at the closing time, not 
release resources. The former means the upstream task has finished, while the 
latter means the result partition can be released.



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.
+    private final int[] numSubpartitionEmitBytes;
+
+    /**
+     * Whether a subpartition has been registered to the netty service. The 
array index is
+     * corresponding to the subpartition id.
+     */
+    private final boolean[] nettyServiceRegistered;
+
+    private final MemoryTierSubpartitionProducerAgent[] 
subpartitionProducerAgents;
+
+    public MemoryTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int bufferSize,
+            int segmentSizeBytes,
+            boolean isBroadcastOnly,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageNettyService nettyService,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at 
least one buffer.");
+
+        this.segmentSizeBytes = segmentSizeBytes;
+        this.segmentSizeBuffers = segmentSizeBytes / bufferSize;
+        this.isBroadcastOnly = isBroadcastOnly;
+        this.memoryManager = memoryManager;
+        this.numSubpartitionEmitBytes = new int[numSubpartitions];
+        this.nettyServiceRegistered = new boolean[numSubpartitions];
+        this.subpartitionProducerAgents = new 
MemoryTierSubpartitionProducerAgent[numSubpartitions];
+
+        Arrays.fill(numSubpartitionEmitBytes, 0);
+        nettyService.registerProducer(partitionId, this);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionProducerAgents[subpartitionId] =
+                    new MemoryTierSubpartitionProducerAgent(subpartitionId, 
nettyService);
+        }
+        resourceRegistry.registerResource(partitionId, this::releaseResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        if (isBroadcastOnly) {
+            return false;
+        }
+        boolean canStartNewSegment =
+                isSubpartitionRegistered(subpartitionId)
+                        && (memoryManager.getMaxNonReclaimableBuffers(this)
+                                        - 
memoryManager.numOwnerRequestedBuffer(this))
+                                > segmentSizeBuffers;
+        if (canStartNewSegment) {
+            getSubpartitionProducerAgent(subpartitionId.getSubpartitionId())
+                    .addSegmentId(segmentId);
+        }
+        return canStartNewSegment;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
finishedBuffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (numSubpartitionEmitBytes[subpartitionIndex] != 0
+                && numSubpartitionEmitBytes[subpartitionIndex] + 
finishedBuffer.readableBytes()
+                        > segmentSizeBytes) {
+            appendEndOfSegmentEvent(subpartitionIndex);
+            numSubpartitionEmitBytes[subpartitionIndex] = 0;
+            return false;
+        }
+        numSubpartitionEmitBytes[subpartitionIndex] += 
finishedBuffer.readableBytes();
+        append(finishedBuffer, subpartitionIndex);
+        return true;
+    }
+
+    @Override
+    public void connectionEstablished(
+            TieredStorageSubpartitionId subpartitionId,
+            NettyConnectionWriter nettyConnectionWriter) {
+        if (isBroadcastOnly) {
+            throw new RuntimeException(
+                    "Broadcast-only result partition is not allowed to use the 
memory tier.");
+        }
+        
this.subpartitionProducerAgents[subpartitionId.getSubpartitionId()].registerNettyService(
+                nettyConnectionWriter);
+        nettyServiceRegistered[subpartitionId.getSubpartitionId()] = true;
+    }
+
+    @Override
+    public void connectionBroken(NettyConnectionId connectionId) {
+        // noop
+    }
+
+    @Override
+    public void close() {
+        // noop
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private void releaseResources() {
+        Arrays.stream(subpartitionProducerAgents)
+                .forEach(MemoryTierSubpartitionProducerAgent::release);
+    }
+
+    private void appendEndOfSegmentEvent(int targetChannel) {
+        try {
+            getSubpartitionProducerAgent(targetChannel)
+                    .addEndOfSegmentEvent(
+                            
EventSerializer.toSerializedEvent(EndOfSegmentEvent.INSTANCE));
+        } catch (IOException e) {
+            ExceptionUtils.rethrow(e, "Failed to append end of segment 
event,");
+        }
+    }
+
+    private void append(Buffer finishedBuffer, int targetChannel) {
+        
getSubpartitionProducerAgent(targetChannel).addFinishedBuffer(finishedBuffer);
+    }
+
+    private boolean isSubpartitionRegistered(TieredStorageSubpartitionId 
subpartitionId) {
+        return nettyServiceRegistered[subpartitionId.getSubpartitionId()];
+    }
+
+    private MemoryTierSubpartitionProducerAgent 
getSubpartitionProducerAgent(int targetChannel) {
+        return subpartitionProducerAgents[targetChannel];
+    }

Review Comment:
   This seems to be unnecessary.



##########
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}. */
+public class MemoryTierFactory implements TierFactory {
+
+    private final int segmentSizeBytes;
+
+    private final int bufferSize;

Review Comment:
   Better to name with explicit unit.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.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.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.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.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 java.nio.ByteBuffer;
+
+import static 
org.apache.flink.runtime.io.network.buffer.Buffer.DataType.END_OF_SEGMENT;
+
+/** The subpartition producer agent for the memory tier. */
+class MemoryTierSubpartitionProducerAgent {
+
+    private final int subpartitionId;
+
+    private final TieredStorageNettyService nettyService;
+
+    private NettyConnectionWriter nettyConnectionWriter;
+
+    private int finishedBufferIndex;
+
+    MemoryTierSubpartitionProducerAgent(
+            int subpartitionId, TieredStorageNettyService nettyService) {
+        this.subpartitionId = subpartitionId;
+        this.nettyService = nettyService;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Called by MemoryTierProducerAgent
+    // ------------------------------------------------------------------------
+
+    void registerNettyService(NettyConnectionWriter nettyConnectionWriter) {

Review Comment:
   Name is inconsistent.



##########
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.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.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.util.Preconditions.checkArgument;
+
+/** The memory tier implementation of {@link TierProducerAgent}. */
+public class MemoryTierProducerAgent implements TierProducerAgent, 
NettyServiceProducer {
+
+    private final int segmentSizeBytes;
+
+    private final int segmentSizeBuffers;
+
+    private final TieredStorageMemoryManager memoryManager;
+
+    private final boolean isBroadcastOnly;
+
+    // Record the byte number currently written to each subpartition.
+    private final int[] numSubpartitionEmitBytes;
+
+    /**
+     * Whether a subpartition has been registered to the netty service. The 
array index is
+     * corresponding to the subpartition id.
+     */
+    private final boolean[] nettyServiceRegistered;
+
+    private final MemoryTierSubpartitionProducerAgent[] 
subpartitionProducerAgents;
+
+    public MemoryTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int bufferSize,
+            int segmentSizeBytes,
+            boolean isBroadcastOnly,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageNettyService nettyService,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at 
least one buffer.");
+
+        this.segmentSizeBytes = segmentSizeBytes;
+        this.segmentSizeBuffers = segmentSizeBytes / bufferSize;
+        this.isBroadcastOnly = isBroadcastOnly;
+        this.memoryManager = memoryManager;
+        this.numSubpartitionEmitBytes = new int[numSubpartitions];
+        this.nettyServiceRegistered = new boolean[numSubpartitions];
+        this.subpartitionProducerAgents = new 
MemoryTierSubpartitionProducerAgent[numSubpartitions];
+
+        Arrays.fill(numSubpartitionEmitBytes, 0);
+        nettyService.registerProducer(partitionId, this);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionProducerAgents[subpartitionId] =
+                    new MemoryTierSubpartitionProducerAgent(subpartitionId, 
nettyService);
+        }
+        resourceRegistry.registerResource(partitionId, this::releaseResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        if (isBroadcastOnly) {
+            return false;
+        }
+        boolean canStartNewSegment =
+                isSubpartitionRegistered(subpartitionId)
+                        && (memoryManager.getMaxNonReclaimableBuffers(this)
+                                        - 
memoryManager.numOwnerRequestedBuffer(this))
+                                > segmentSizeBuffers;
+        if (canStartNewSegment) {
+            getSubpartitionProducerAgent(subpartitionId.getSubpartitionId())
+                    .addSegmentId(segmentId);
+        }
+        return canStartNewSegment;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
finishedBuffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (numSubpartitionEmitBytes[subpartitionIndex] != 0
+                && numSubpartitionEmitBytes[subpartitionIndex] + 
finishedBuffer.readableBytes()
+                        > segmentSizeBytes) {
+            appendEndOfSegmentEvent(subpartitionIndex);
+            numSubpartitionEmitBytes[subpartitionIndex] = 0;
+            return false;
+        }

Review Comment:
   The name `numSubpartitionEmitBytes` is misleading. This is the emitted bytes 
only for the current segment of the subpartition.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgent.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.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.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.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 java.nio.ByteBuffer;
+
+import static 
org.apache.flink.runtime.io.network.buffer.Buffer.DataType.END_OF_SEGMENT;
+
+/** The subpartition producer agent for the memory tier. */
+class MemoryTierSubpartitionProducerAgent {
+
+    private final int subpartitionId;
+
+    private final TieredStorageNettyService nettyService;
+
+    private NettyConnectionWriter nettyConnectionWriter;

Review Comment:
   Lack `@Nullable` and semantic explaination.



-- 
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

Reply via email to