You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/18 08:29:14 UTC

[GitHub] [flink] reswqa opened a new pull request, #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

reswqa opened a new pull request, #20293:
URL: https://github.com/apache/flink/pull/20293

   ## What is the purpose of the change
   
   *Introduce HsDataBuffer to manage memory data of hybrid shuffle mode.*
   
   
   ## Brief change log
   
     - *Introduce BufferContext and make `HsSpillingStrategy` using `BufferContext` instead of `BufferWithIdentity`.*
     - *Introduce `HsMemoryDataManager`.*
   
   ## Verifying this change
    
   This change added `HsMemoryDataManagerTest`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924646640


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong closed pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
xintongsong closed pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager
URL: https://github.com/apache/flink/pull/20293


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r926896614


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
     /** One thread to perform spill operation. */
     private final ExecutorService ioExecutor =
             Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("hybrid spiller thread")
+                            // It is more appropriate to use task fail over than exit JVM here,
+                            // but the task thread will bring some extra overhead to check the
+                            // exception information set by other thread. As the spiller thread will
+                            // not encounter exceptions in most cases, we temporarily choose the
+                            // form of fatal error to deal except thrown by spiller thread.
+                            .setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());

Review Comment:
   ok, changes about spilling are separate to a single hot-fix commit.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924340998


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }

Review Comment:
   yes, scheduled it to spillFuture



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }

Review Comment:
   yes, scheduled it to spillFuture.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924645809


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924600342


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20293:
URL: https://github.com/apache/flink/pull/20293#issuecomment-1186919814

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b3194d8bb511e6d79ccceaefcc7367ee39df9af0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b3194d8bb511e6d79ccceaefcc7367ee39df9af0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b3194d8bb511e6d79ccceaefcc7367ee39df9af0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924337991


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924327006


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {

Review Comment:
   done.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r926896614


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
     /** One thread to perform spill operation. */
     private final ExecutorService ioExecutor =
             Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("hybrid spiller thread")
+                            // It is more appropriate to use task fail over than exit JVM here,
+                            // but the task thread will bring some extra overhead to check the
+                            // exception information set by other thread. As the spiller thread will
+                            // not encounter exceptions in most cases, we temporarily choose the
+                            // form of fatal error to deal except thrown by spiller thread.
+                            .setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());

Review Comment:
   Ok, changes about spiller are separate to a single hot-fix commit, except for the removal of TODO related to d074767d.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r926896614


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
     /** One thread to perform spill operation. */
     private final ExecutorService ioExecutor =
             Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("hybrid spiller thread")
+                            // It is more appropriate to use task fail over than exit JVM here,
+                            // but the task thread will bring some extra overhead to check the
+                            // exception information set by other thread. As the spiller thread will
+                            // not encounter exceptions in most cases, we temporarily choose the
+                            // form of fatal error to deal except thrown by spiller thread.
+                            .setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());

Review Comment:
   ok, changes about spiller are separate to a single hot-fix commit, d074767d only removes a todo related to it now.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r926896614


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
     /** One thread to perform spill operation. */
     private final ExecutorService ioExecutor =
             Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("hybrid spiller thread")
+                            // It is more appropriate to use task fail over than exit JVM here,
+                            // but the task thread will bring some extra overhead to check the
+                            // exception information set by other thread. As the spiller thread will
+                            // not encounter exceptions in most cases, we temporarily choose the
+                            // form of fatal error to deal except thrown by spiller thread.
+                            .setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());

Review Comment:
   ok, changes about spiller are separate to a single hot-fix commit.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
xintongsong commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r926541526


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
     /** One thread to perform spill operation. */
     private final ExecutorService ioExecutor =
             Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("hybrid spiller thread")
+                            // It is more appropriate to use task fail over than exit JVM here,
+                            // but the task thread will bring some extra overhead to check the
+                            // exception information set by other thread. As the spiller thread will
+                            // not encounter exceptions in most cases, we temporarily choose the
+                            // form of fatal error to deal except thrown by spiller thread.
+                            .setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());

Review Comment:
   This change seems unrelated to this commit. It should be a separate hotfix commit.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924647096


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924345884


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferConsumed() {
+            isConsumed = true;
+            // add ref count when buffer consumed.
+            buffer.retainBuffer();
+        }
+    }
+
+    /** Integrate the buffer and dataType of next buffer. */
+    public static class BufferAndNextDataType {
+        private final Buffer buffer;
+
+        private final Buffer.DataType nextDataType;
+
+        public BufferAndNextDataType(Buffer buffer, Buffer.DataType nextDataType) {
+            this.buffer = buffer;
+            this.nextDataType = nextDataType;
+        }
+
+        public Buffer getBuffer() {
+            return buffer;
+        }
+
+        public Buffer.DataType getNextDataType() {

Review Comment:
   I think `nextDataType` should never be null, if a buffer without next buffer, nextDataType of this buffer will be `Buffer.DataType.NONE`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924343452


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferConsumed() {
+            isConsumed = true;
+            // add ref count when buffer consumed.
+            buffer.retainBuffer();

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924338980


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924351170


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();

Review Comment:
   When append a record, it is sometimes necessary to request multiple `unfinished buffer`, and then fill the data on them in turn. Whenever one is filled, it will be polled out of the queue and a `finished buffer` will be created.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924307597


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -60,11 +60,10 @@ public HsMemoryDataSpiller(FileChannel dataFileChannel) {
      *     (subpartitionId, bufferIndex).
      * @return the completable future contains spilled buffers information.
      */
-    public CompletableFuture<List<SpilledBuffer>> spillAsync(
-            List<BufferWithIdentity> bufferToSpill) {
-        CompletableFuture<List<SpilledBuffer>> spilledFuture = new CompletableFuture<>();
-        ioExecutor.execute(() -> spill(bufferToSpill, spilledFuture));
-        return spilledFuture;
+    public void spillAsync(

Review Comment:
   good catch, updated.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] xintongsong commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
xintongsong commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924106288


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -60,11 +60,10 @@ public HsMemoryDataSpiller(FileChannel dataFileChannel) {
      *     (subpartitionId, bufferIndex).
      * @return the completable future contains spilled buffers information.
      */
-    public CompletableFuture<List<SpilledBuffer>> spillAsync(
-            List<BufferWithIdentity> bufferToSpill) {
-        CompletableFuture<List<SpilledBuffer>> spilledFuture = new CompletableFuture<>();
-        ioExecutor.execute(() -> spill(bufferToSpill, spilledFuture));
-        return spilledFuture;
+    public void spillAsync(

Review Comment:
   JavaDoc not updated.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();

Review Comment:
   ```suggestion
               if (!isReleased) {
                   isReleased = true;
                   buffer.recycleBuffer();
               }
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;

Review Comment:
   ```suggestion
           @Nullable
           private CompletableFuture<Void> spilledFuture;
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }

Review Comment:
   Why do we need this method? Can this be scheduled in `onBufferSpill`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {

Review Comment:
   Why not guarded by the read lock?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();

Review Comment:
   We should only pass `readLock` into the subpartition, to make sure it doesn't accidentally access the write lock.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {

Review Comment:
   ```suggestion
       class HsSubpartitionMemoryDataManager {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {

Review Comment:
   I'd suggest not to make this an inner class, for a better separation between `HsMemoryDataManager` and `HsSubpartitionDataBuffer`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;

Review Comment:
   ```suggestion
               checkState(!isReleased);
               checkState(!isSpill && this.spilledFuture == null, "error message");
               // retainBuffer and mark buffer is spill status.
               buffer.retainBuffer();
               isSpill = true;
               this.spilledFuture = spilledFuture;
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferConsumed() {
+            isConsumed = true;
+            // add ref count when buffer consumed.
+            buffer.retainBuffer();
+        }
+    }
+
+    /** Integrate the buffer and dataType of next buffer. */
+    public static class BufferAndNextDataType {
+        private final Buffer buffer;
+
+        private final Buffer.DataType nextDataType;
+
+        public BufferAndNextDataType(Buffer buffer, Buffer.DataType nextDataType) {
+            this.buffer = buffer;
+            this.nextDataType = nextDataType;
+        }
+
+        public Buffer getBuffer() {
+            return buffer;
+        }
+
+        public Buffer.DataType getNextDataType() {

Review Comment:
   ```suggestion
           public Optional<Buffer.DataType> getNextDataType() {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }

Review Comment:
   Can be deduplicated as a method.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {

Review Comment:
   This can be a separate class. To keep the internal status consistent, it is important to prevent access from outside. Moreover, it can be tested separately.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferSpill(CompletableFuture<Void> spilledFuture) {
+            // retainBuffer and mark buffer is spill status.
+            buffer.retainBuffer();
+            isSpill = true;
+            checkState(this.spilledFuture == null, "Already set spill future for buffer context.");
+            this.spilledFuture = spilledFuture;
+        }
+
+        public void onBufferSpilled() {
+            // decrease ref count when spilling is finished.
+            buffer.recycleBuffer();
+        }
+
+        public void onBufferConsumed() {
+            isConsumed = true;
+            // add ref count when buffer consumed.
+            buffer.retainBuffer();

Review Comment:
   ```suggestion
   checkState(!isReleased);
               checkState(!isConsumed);
               isConsumed = true;
               // increase ref count, will be decreased when downstream finish consuming
               buffer.retainBuffer();
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();

Review Comment:
   Why is this a queue?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {

Review Comment:
   What is this `expectedBufferIndex`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);

Review Comment:
   It might be better to always deal with the strategy and decision in the memory data manager.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {

Review Comment:
   I'd suggest to abstract the lock accessing logics with methods like: `runWithSubpartitionLock` and `runWithPartitionLock`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }
+                if (consumeBuffers.isEmpty()
+                        || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                != expectedBufferIndex) {
+                    return Buffer.DataType.NONE;
+                }
+                BufferContext bufferContext = checkNotNull(consumeBuffers.peekFirst());
+                return bufferContext.buffer.getDataType();
+            }
+        }
+
+        public BufferAndNextDataType pollBuffer(int expectedBufferIndex) {
+            BufferContext bufferContext;
+            Buffer.DataType nextDataType;
+            try {
+                lock.readLock().lock();
+                synchronized (subpartitionLock) {
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    if (consumeBuffers.isEmpty()
+                            || consumeBuffers.peekFirst().bufferIndexAndChannel.getBufferIndex()
+                                    != expectedBufferIndex) {
+                        return null;
+                    }
+
+                    bufferContext = checkNotNull(consumeBuffers.pollFirst());
+
+                    bufferContext.onBufferConsumed();
+                    while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                        consumeBuffers.pollFirst();
+                    }
+                    BufferContext next = consumeBuffers.peekFirst();
+                    nextDataType = next == null ? Buffer.DataType.NONE : next.buffer.getDataType();
+                }
+            } finally {
+                lock.readLock().unlock();
+            }
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferConsumed(bufferContext.bufferIndexAndChannel);
+            // Calling handleDecision must be outside the read lock, because write lock may be
+            // accessed inside, which may block forever.
+            handleDecision(decisionOpt);
+            return new BufferAndNextDataType(bufferContext.buffer, nextDataType);
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by MemoryDataManager
+        // ------------------------------------------------------------------------
+
+        private void append(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            if (dataType.isEvent()) {
+                writeEvent(record, dataType);
+            } else {
+                writeRecord(record, dataType);
+            }
+        }
+
+        private Deque<BufferIndexAndChannel> getBuffersSatisfyStatus(
+                SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            synchronized (subpartitionLock) {
+                // TODO return iterator to avoid completely traversing the queue for each call.
+                Deque<BufferIndexAndChannel> targetBuffers = new ArrayDeque<>();
+                // traverse buffers in order.
+                finishedBuffers.forEach(
+                        (bufferContext -> {
+                            if (isBufferSatisfyStatus(bufferContext, spillStatus, consumeStatus)) {
+                                targetBuffers.add(bufferContext.bufferIndexAndChannel);
+                            }
+                        }));
+                return targetBuffers;
+            }
+        }
+
+        private Object getSubpartitionLock() {
+            return subpartitionLock;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext getBufferContext(int bufferIndex) {
+            return checkNotNull(bufferIndexToContexts.get(bufferIndex));
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferReleased(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.remove(bufferIndex));
+            bufferContext.onBufferReleased();
+            // remove released buffers from head lazy.
+            while (!finishedBuffers.isEmpty() && finishedBuffers.peekFirst().isReleased) {
+                finishedBuffers.pollFirst();
+            }
+        }
+
+        @GuardedBy("subpartitionLock")
+        private BufferContext onBufferSpill(int bufferIndex, CompletableFuture<Void> spillFuture) {
+            // decrement unSpill buffers when it decided
+            // to spill.
+            numUnSpillBuffers.decrementAndGet();
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpill(spillFuture);
+            return bufferContext;
+        }
+
+        @GuardedBy("subpartitionLock")
+        private void onBufferSpilled(int bufferIndex) {
+            BufferContext bufferContext = checkNotNull(bufferIndexToContexts.get(bufferIndex));
+            bufferContext.onBufferSpilled();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void writeEvent(ByteBuffer event, Buffer.DataType dataType) {
+            checkArgument(dataType.isEvent());
+
+            // each Event must take an exclusive buffer
+            finishCurrentWritingBufferIfNotEmpty();
+
+            // store Events in adhoc heap segments, for network memory efficiency
+            MemorySegment data = MemorySegmentFactory.wrap(event.array());
+            Buffer buffer =
+                    new NetworkBuffer(data, FreeingBufferRecycler.INSTANCE, dataType, data.size());
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void writeRecord(ByteBuffer record, Buffer.DataType dataType)
+                throws InterruptedException {
+            checkArgument(!dataType.isEvent());
+
+            ensureCapacityForRecord(record);
+
+            writeRecord(record);
+        }
+
+        private void ensureCapacityForRecord(ByteBuffer record) throws InterruptedException {
+            final int numRecordBytes = record.remaining();
+            int availableBytes =
+                    Optional.ofNullable(unfinishedBuffers.peek())
+                            .map(
+                                    currentWritingBuffer ->
+                                            currentWritingBuffer.getWritableBytes()
+                                                    + bufferSize * (unfinishedBuffers.size() - 1))
+                            .orElse(0);
+
+            while (availableBytes < numRecordBytes) {
+                // request unfinished buffer.
+                BufferBuilder bufferBuilder = requestUnfinishedBuffer();
+                unfinishedBuffers.add(bufferBuilder);
+                availableBytes += bufferSize;
+            }
+        }
+
+        private void writeRecord(ByteBuffer record) {
+            while (record.hasRemaining()) {
+                BufferBuilder currentWritingBuffer =
+                        checkNotNull(
+                                unfinishedBuffers.peek(), "Expect enough capacity for the record.");
+                currentWritingBuffer.append(record);
+
+                if (currentWritingBuffer.isFull()) {
+                    finishCurrentWritingBuffer();
+                }
+            }
+        }
+
+        private void finishCurrentWritingBufferIfNotEmpty() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.peek();
+            if (currentWritingBuffer == null
+                    || currentWritingBuffer.getWritableBytes() == bufferSize) {
+                return;
+            }
+
+            finishCurrentWritingBuffer();
+        }
+
+        private void finishCurrentWritingBuffer() {
+            BufferBuilder currentWritingBuffer = unfinishedBuffers.poll();
+
+            if (currentWritingBuffer == null) {
+                return;
+            }
+
+            currentWritingBuffer.finish();
+            Buffer buffer = currentWritingBuffer.createBufferConsumerFromBeginning().build();
+            currentWritingBuffer.close();
+            // create buffer consumer will add reference count of internal buffer, decrease it to
+            // keep only one reference count.
+            buffer.recycleBuffer();
+
+            BufferContext bufferContext =
+                    new BufferContext(buffer, finishedBufferIndex, targetChannel);
+            addFinishedBuffer(bufferContext);
+            Optional<Decision> decisionOpt =
+                    spillStrategy.onBufferFinished(numUnSpillBuffers.incrementAndGet());
+            handleDecision(decisionOpt);
+        }
+
+        private void addFinishedBuffer(BufferContext bufferContext) {
+            finishedBufferIndex++;
+            LockUtils.runWithLock(
+                    lock.readLock(),
+                    () -> {
+                        boolean needNotify = false;
+                        synchronized (subpartitionLock) {
+                            finishedBuffers.add(bufferContext);
+                            consumeBuffers.add(bufferContext);
+                            bufferIndexToContexts.put(
+                                    bufferContext.bufferIndexAndChannel.getBufferIndex(),
+                                    bufferContext);
+                            if (consumeBuffers.isEmpty()) {
+                                needNotify = true;
+                            }
+                        }
+                        if (needNotify) {
+                            // TODO notify data available.
+                        }
+                    });
+        }
+
+        @GuardedBy("subpartitionLock")
+        private boolean isBufferSatisfyStatus(
+                BufferContext bufferContext, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+            // released buffer is not needed.
+            if (bufferContext.isReleased) {
+                return false;
+            }
+            boolean isNeeded = true;
+            switch (spillStatus) {
+                case NOT_SPILL:
+                    isNeeded = !bufferContext.isSpill;
+                    break;
+                case SPILL:
+                    isNeeded = bufferContext.isSpill;
+                    break;
+            }
+            switch (consumeStatus) {
+                case NOT_CONSUMED:
+                    isNeeded &= !bufferContext.isConsumed;
+                    break;
+                case CONSUMED:
+                    isNeeded &= bufferContext.isConsumed;
+                    break;
+            }
+            return isNeeded;
+        }
+    }
+
+    /**
+     * This class maintains the buffer information and its status for hybrid shuffle mode.
+     *
+     * <p>Note that: This class itself is not thread-safe, guarded by subpartition lock.
+     */
+    private static class BufferContext {
+        private final Buffer buffer;
+
+        private final BufferIndexAndChannel bufferIndexAndChannel;
+
+        private boolean isReleased;
+
+        private boolean isSpill;
+
+        private boolean isConsumed;
+
+        private CompletableFuture<Void> spilledFuture;
+
+        public BufferContext(Buffer buffer, int bufferIndex, int subpartitionId) {
+            this.bufferIndexAndChannel = new BufferIndexAndChannel(bufferIndex, subpartitionId);
+            this.buffer = buffer;
+        }
+
+        public void onBufferReleased() {
+            isReleased = true;
+            buffer.recycleBuffer();

Review Comment:
   We need to prevent duplicate calls to the state changing methods, by either ignoring extra calls or failing.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924687852


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {

Review Comment:
   Add java doc for all public method.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] reswqa commented on a diff in pull request #20293: [FLINK-27904][runtime] Introduce HsMemoryDataManager

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20293:
URL: https://github.com/apache/flink/pull/20293#discussion_r924610288


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+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.HsFileDataIndex.SpilledBuffer;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsSpillingStrategy.Decision;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** This class is responsible for managing data in memory. */
+public class HsMemoryDataManager implements HsSpillingInfoProvider {
+
+    private final int numSubpartitions;
+
+    private final int bufferSize;
+
+    private final HsSubpartitionDataBuffer[] subpartitionDataBuffers;
+
+    private final HsMemoryDataSpiller spiller;
+
+    private final HsSpillingStrategy spillStrategy;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final BufferPool bufferPool;
+
+    /**
+     * This lock is used for lock the entire resultPartition. As each subpartition has its own lock,
+     * in order to avoid deadlock, the lock must be acquired before subpartition lock.
+     */
+    private final ReentrantReadWriteLock lock;
+
+    private final AtomicInteger numRequestedBuffers;
+
+    private final AtomicInteger numUnSpillBuffers;
+
+    public HsMemoryDataManager(
+            int numSubpartitions,
+            int bufferSize,
+            BufferPool bufferPool,
+            HsSpillingStrategy spillStrategy,
+            FileChannel dataFileChannel) {
+        this.numSubpartitions = numSubpartitions;
+        this.bufferPool = bufferPool;
+        this.bufferSize = bufferSize;
+        this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+        this.spillStrategy = spillStrategy;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.subpartitionDataBuffers = new HsSubpartitionDataBuffer[numSubpartitions];
+        this.lock = new ReentrantReadWriteLock(true);
+        this.numRequestedBuffers = new AtomicInteger(0);
+        this.numUnSpillBuffers = new AtomicInteger(0);
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) {
+            subpartitionDataBuffers[subpartitionId] = new HsSubpartitionDataBuffer(subpartitionId);
+        }
+    }
+
+    // ------------------------------------
+    // For ResultPartition
+    // ------------------------------------
+
+    public void append(ByteBuffer record, int targetChannel, Buffer.DataType dataType)
+            throws IOException {
+        try {
+            getTargetSubpartitionDataBuffer(targetChannel).append(record, dataType);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public HsSubpartitionDataBuffer getTargetSubpartitionDataBuffer(int targetChannel) {
+        return subpartitionDataBuffers[targetChannel];
+    }
+
+    // ------------------------------------
+    //        For Spilling Strategy
+    // ------------------------------------
+
+    @Override
+    public int getPoolSize() {
+        return bufferPool.getNumBuffers();
+    }
+
+    @Override
+    public int getNumSubpartitions() {
+        return numSubpartitions;
+    }
+
+    @Override
+    public int getNumTotalRequestedBuffers() {
+        return numRequestedBuffers.get();
+    }
+
+    @Override
+    public int getNumTotalUnSpillBuffers() {
+        return numUnSpillBuffers.get();
+    }
+
+    @Override
+    public Deque<BufferIndexAndChannel> getBuffersInOrder(
+            int subpartitionId, SpillStatus spillStatus, ConsumeStatus consumeStatus) {
+        HsSubpartitionDataBuffer targetSubpartitionDataBuffer =
+                getTargetSubpartitionDataBuffer(subpartitionId);
+        return targetSubpartitionDataBuffer.getBuffersSatisfyStatus(spillStatus, consumeStatus);
+    }
+
+    @Override
+    public List<Integer> getNextBufferIndexToConsume() {
+        // TODO implements this logical when subpartition view is implemented.
+        return Collections.emptyList();
+    }
+
+    // ------------------------------------
+    // Internal Method
+    // ------------------------------------
+
+    // Do not call this method within the subpartition lock, otherwise deadlock may occur, as
+    // spillBuffers and releaseBuffers maybe acquire other subpartition's lock.
+    private void handleDecision(
+            @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+                    Optional<Decision> decisionOpt) {
+        Decision decision =
+                decisionOpt.orElseGet(
+                        () ->
+                                LockUtils.callWithLock(
+                                        lock.writeLock(),
+                                        () -> spillStrategy.decideActionWithGlobalInfo(this)));
+
+        if (!decision.getBufferToSpill().isEmpty()) {
+            spillBuffers(decision.getBufferToSpill());
+        }
+        if (!decision.getBufferToRelease().isEmpty()) {
+            releaseBuffers(decision.getBufferToRelease());
+        }
+    }
+
+    private BufferBuilder requestUnfinishedBuffer() throws InterruptedException {
+        MemorySegment segment = bufferPool.requestMemorySegmentBlocking();
+        Optional<Decision> decisionOpt =
+                spillStrategy.onMemoryUsageChanged(
+                        numRequestedBuffers.incrementAndGet(), getPoolSize());
+
+        handleDecision(decisionOpt);
+        return new BufferBuilder(segment, this::recycleBuffer);
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void spillBuffers(Map<Integer, List<BufferIndexAndChannel>> toSpill) {
+        CompletableFuture<List<SpilledBuffer>> spillFuture = new CompletableFuture<>();
+        List<BufferWithIdentity> bufferWithIdentities = new ArrayList<>();
+        toSpill.forEach(
+                (subpartitionId, bufferIndexAndChannels) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    bufferWithIdentities.addAll(
+                                            subpartitionId,
+                                            bufferIndexAndChannels.stream()
+                                                    .map(
+                                                            indexAndChannel -> {
+                                                                int bufferIndex =
+                                                                        indexAndChannel
+                                                                                .getBufferIndex();
+                                                                BufferContext bufferContext =
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpill(
+                                                                                        bufferIndex,
+                                                                                        spillFuture
+                                                                                                .thenRun(
+                                                                                                        () -> {}));
+                                                                return new BufferWithIdentity(
+                                                                        bufferContext.buffer,
+                                                                        bufferIndex,
+                                                                        subpartitionId);
+                                                            })
+                                                    .collect(Collectors.toList()));
+                                }
+                            });
+                });
+
+        spiller.spillAsync(bufferWithIdentities, spillFuture);
+
+        spillFuture.thenAccept(
+                // in spilling thread.
+                (spilledBuffers) -> {
+                    // update data index.
+                    dataIndex.addBuffers(spilledBuffers);
+                    spilledBuffers.stream()
+                            .collect(
+                                    Collectors.groupingBy(
+                                            spilledBuffer -> spilledBuffer.subpartitionId))
+                            .forEach(
+                                    (subpartitionId, subpartitionBuffers) -> {
+                                        HsSubpartitionDataBuffer subpartitionDataBuffer =
+                                                getTargetSubpartitionDataBuffer(subpartitionId);
+                                        final Object subpartitionLock =
+                                                subpartitionDataBuffer.getSubpartitionLock();
+                                        LockUtils.runWithLock(
+                                                lock.readLock(),
+                                                () -> {
+                                                    synchronized (subpartitionLock) {
+                                                        subpartitionBuffers.forEach(
+                                                                spilledBuffer ->
+                                                                        subpartitionDataBuffer
+                                                                                .onBufferSpilled(
+                                                                                        spilledBuffer
+                                                                                                .bufferIndex));
+                                                    }
+                                                });
+                                    });
+                });
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    // suppress warning of FieldAccessNotGuarded, as static code analysis cannot correctly identify
+    // lock from getSubpartitionLock.
+    private void releaseBuffers(Map<Integer, List<BufferIndexAndChannel>> toRelease) {
+        toRelease.forEach(
+                (subpartitionId, buffers) -> {
+                    HsSubpartitionDataBuffer subpartitionDataBuffer =
+                            getTargetSubpartitionDataBuffer(subpartitionId);
+                    final Object subpartitionLock = subpartitionDataBuffer.getSubpartitionLock();
+                    LockUtils.runWithLock(
+                            lock.readLock(),
+                            () -> {
+                                synchronized (subpartitionLock) {
+                                    buffers.forEach(
+                                            (indexAndChannel) -> {
+                                                int bufferIndex = indexAndChannel.getBufferIndex();
+                                                BufferContext bufferContext =
+                                                        subpartitionDataBuffer.getBufferContext(
+                                                                bufferIndex);
+                                                // only spill and not consumed buffer needs to be
+                                                // marked as readable.
+                                                if (subpartitionDataBuffer.isBufferSatisfyStatus(
+                                                        bufferContext,
+                                                        SpillStatus.SPILL,
+                                                        ConsumeStatus.NOT_CONSUMED)) {
+                                                    checkNotNull(
+                                                                    bufferContext.spilledFuture,
+                                                                    "Buffer in spill status should already set spill future.")
+                                                            .thenRun(
+                                                                    () ->
+                                                                            dataIndex
+                                                                                    .markBufferReadable(
+                                                                                            subpartitionId,
+                                                                                            bufferIndex));
+                                                }
+                                                subpartitionDataBuffer.onBufferReleased(
+                                                        bufferIndex);
+                                            });
+                                }
+                            });
+                });
+    }
+
+    private void recycleBuffer(MemorySegment buffer) {
+        numRequestedBuffers.decrementAndGet();
+        bufferPool.recycle(buffer);
+    }
+
+    class HsSubpartitionDataBuffer {
+        // this field only accessed by task main thread.
+        private final Queue<BufferBuilder> unfinishedBuffers = new LinkedList<>();
+
+        // this field only accessed by task main thread.
+        private int finishedBufferIndex;
+
+        /**
+         * Lock for subpartition's own field. All buffers status belong to this subpartition guarded
+         * by this lock.
+         */
+        private final Object subpartitionLock = new Object();
+
+        // TODO maybe we should rename this to allBuffers or unReleasedBuffers?
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> finishedBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Deque<BufferContext> consumeBuffers = new LinkedList<>();
+
+        @GuardedBy("subpartitionLock")
+        private final Map<Integer, BufferContext> bufferIndexToContexts;
+
+        private final int targetChannel;
+
+        HsSubpartitionDataBuffer(int targetChannel) {
+            this.targetChannel = targetChannel;
+            this.bufferIndexToContexts = new HashMap<>();
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by Consumer
+        // ------------------------------------------------------------------------
+
+        public Buffer.DataType getFirstDataType(int expectedBufferIndex) {
+            synchronized (subpartitionLock) {
+                while (!consumeBuffers.isEmpty() && consumeBuffers.peekFirst().isReleased) {
+                    consumeBuffers.pollFirst();
+                }

Review Comment:
   done.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org