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

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

Reply via email to