wsry commented on a change in pull request #13924:
URL: https://github.com/apache/flink/pull/13924#discussion_r602130482



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java
##########
@@ -0,0 +1,395 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.util.FatalExitExceptionHandler;
+import org.apache.flink.util.IOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Data reader for {@link SortMergeResultPartition} which can read data for 
all downstream tasks
+ * consuming the corresponding {@link SortMergeResultPartition}. It always 
tries to read shuffle
+ * data in order of file offset, which maximums the sequential read so can 
improve the blocking
+ * shuffle performance.
+ */
+class SortMergeResultPartitionReadScheduler implements Runnable, 
BufferRecycler {
+
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(SortMergeResultPartitionReadScheduler.class);
+
+    private final FatalExitExceptionHandler fatalErrorHandler = new 
FatalExitExceptionHandler();
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. 
*/
+    private final Object lock;
+
+    /**
+     * A {@link CompletableFuture} to be completed when this read scheduler 
including all resources
+     * is released.
+     */
+    private final CompletableFuture<?> releaseFuture = new 
CompletableFuture<>();
+
+    /** Buffer pool from which to allocate buffers for shuffle data reading. */
+    private final BatchShuffleReadBufferPool bufferPool;
+
+    /** Executor to run the shuffle data reading task. */
+    private final Executor ioExecutor;
+
+    /** Maximum number of buffers can be allocated by this partition reader. */
+    private final int maxRequestedBuffers;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<SortMergeSubpartitionReader> failedReaders = new 
HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<SortMergeSubpartitionReader> allReaders = new 
HashSet<>();
+
+    /** File channel shared by all subpartitions to read data from. */
+    @GuardedBy("lock")
+    private FileChannel dataFileChannel;
+
+    /** File channel shared by all subpartitions to read index from. */
+    @GuardedBy("lock")
+    private FileChannel indexFileChannel;
+
+    /** Number of buffers already allocated and still not recycled by this 
partition reader. */
+    @GuardedBy("lock")
+    private int numRequestedBuffers;
+
+    /**
+     * Whether the data reading task is currently running or not. This flag is 
used when trying to
+     * submit the data reading task.
+     */
+    @GuardedBy("lock")
+    private boolean isRunning;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private boolean isReleased;
+
+    SortMergeResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object 
lock) {
+        this.lock = checkNotNull(lock);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most 32M (the expected buffers 
per request is 8M)
+        // buffers for data read. Currently, it is only an empirical value can 
not be configured
+        this.maxRequestedBuffers = Math.max(1, 4 * 
bufferPool.getNumBuffersPerRequest());
+
+        // initialize the buffer pool eagerly to avoid reporting errors like 
OOM too late
+        bufferPool.initialize();
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<SortMergeSubpartitionReader> availableReaders = 
getAvailableReaders();
+
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<SortMergeSubpartitionReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    private Queue<MemorySegment> allocateBuffers(
+            Queue<SortMergeSubpartitionReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            return new ArrayDeque<>(bufferPool.requestBuffers());
+        } catch (Throwable throwable) {
+            // do not fail the pending subpartitions if the TimeoutException 
occurs and we are
+            // currently holding buffers, otherwise, fail all pending 
subpartitions immediately
+            if (!(throwable instanceof TimeoutException) || 
numRequestedBuffers <= 0) {

Review comment:
       I moved the timeout logic to this method and changed the logic here 
which is easier to understand 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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to