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/01 08:17:11 UTC

[GitHub] [flink] xintongsong commented on a diff in pull request #19960: [FLINK-27907][runtime] implement disk read and write logic for hybrid shuffle

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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {
+                // fail all pending subpartition readers immediately if any exception occurs
+                failSubpartitionReaders(availableReaders, throwable);
+                LOG.error("Failed to request buffers for data reading.", throwable);
+            }
+        }
+        int numBuffersAllocated = buffers.size();
+
+        readData(availableReaders, buffers);
+
+        endCurrentRoundOfReading(numBuffersAllocated, buffers);
+    }
+
+    /** This method only called by result partition to create subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewOperation notifier) throws IOException {
+        synchronized (lock) {
+            lazyInitialize();
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId,
+                            dataFileChannel,
+                            notifier,
+                            dataIndex,
+                            hybridShuffleConfiguration.getMaxBuffersReadAhead());
+            allReaders.add(subpartitionReader);
+
+            mayTriggerReading();
+            return subpartitionReader;
+        }
+    }
+
+    /**
+     * Releases this read scheduler and returns a {@link CompletableFuture} which will be completed
+     * when all resources are released.
+     */
+    public synchronized CompletableFuture<?> release() {
+        List<HsSubpartitionFileReader> pendingReaders;
+        synchronized (lock) {
+            if (isReleased) {
+                return releaseFuture;
+            }
+            isReleased = true;
+
+            failedReaders.addAll(allReaders);
+            pendingReaders = new ArrayList<>(allReaders);
+            mayNotifyReleased();
+        }
+
+        failSubpartitionReaders(
+                pendingReaders,
+                new IllegalStateException("Result partition has been already released."));
+        return releaseFuture;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    @SuppressWarnings("GuardedBy")
+    private Queue<MemorySegment> allocateBuffers() throws Exception {
+        long timeoutTime = getBufferRequestTimeoutTime();
+        do {
+            List<MemorySegment> buffers = bufferPool.requestBuffers();
+            if (!buffers.isEmpty()) {
+                return new ArrayDeque<>(buffers);
+            }
+            checkState(!isReleased, "Result partition has been already released.");
+        } while (System.nanoTime() < timeoutTime
+                || System.nanoTime() < (timeoutTime = getBufferRequestTimeoutTime()));
+
+        if (numRequestedBuffers <= 0) {
+            // If the buffer request times out, an exception should be thrown to fail all
+            // subpartitionReaders, otherwise a deadlock may occur (Some operators have consumption
+            // priorities, if low-priority subpartitions occupy the buffer but do not consume, then
+            // high-priority subpartitions can't request buffer.)
+            throw new TimeoutException(
+                    String.format(
+                            "Buffer request timeout, this means there is a fierce contention of"
+                                    + " the batch shuffle read memory, please increase '%s'.",
+                            TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key()));
+        }
+        return new ArrayDeque<>();
+    }
+
+    @GuardedBy("lock")
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() <= maxRequestedBuffers
+                && numRequestedBuffers < bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void mayNotifyReleased() {
+        assert Thread.holdsLock(lock);
+
+        if (isReleased && allReaders.isEmpty()) {
+            releaseFuture.complete(null);
+        }
+    }
+
+    private long getBufferRequestTimeoutTime() {
+        return bufferPool.getLastBufferOperationTimestamp() + bufferRequestTimeout.toNanos();
+    }
+
+    private void releaseBuffers(Queue<MemorySegment> buffers) {
+        if (!buffers.isEmpty()) {
+            try {
+                bufferPool.recycle(buffers);
+                buffers.clear();
+            } catch (Throwable throwable) {
+                // this should never happen so just trigger fatal error
+                FatalExitExceptionHandler.INSTANCE.uncaughtException(
+                        Thread.currentThread(), throwable);
+            }
+        }
+    }
+
+    private Queue<HsSubpartitionFileReader> prepareAndGetAvailableReaders() {
+        synchronized (lock) {
+            if (isReleased) {
+                return new ArrayDeque<>();
+            }
+
+            try {
+                for (HsSubpartitionFileReader reader : allReaders) {
+                    reader.prepareForScheduling();
+                }
+            } catch (Throwable throwable) {
+                failSubpartitionReaders(allReaders, throwable);
+            }
+
+            return new PriorityQueue<>(allReaders);
+        }
+    }
+
+    private void readData(
+            Queue<HsSubpartitionFileReader> availableReaders, Queue<MemorySegment> buffers) {
+        while (!availableReaders.isEmpty() && !buffers.isEmpty()) {
+            HsSubpartitionFileReader subpartitionReader = availableReaders.poll();
+            try {
+                subpartitionReader.readBuffers(buffers, this);
+            } catch (Throwable throwable) {

Review Comment:
   ```suggestion
               } catch (IOException exception) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {

Review Comment:
   Shouldn't we just skip everything is `availableReaders` is empty? Or at least everything before `endCurrentRoundOfReading`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.positionToNextBuffer;
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.readFromByteChannel;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This component is responsible for reading data from disk.
+ *
+ * <p>In order to access the disk as sequentially as possible {@link HsSubpartitionFileReader} need
+ * to be able to compare priorities.
+ */
+public class HsSubpartitionFileReader implements Comparable<HsSubpartitionFileReader> {
+
+    private final ByteBuffer headerBuf = BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewOperation notifier;
+
+    private final CachedRegionManager cachedRegionManager;
+
+    private final BufferIndexManager bufferIndexManager;
+
+    private final Deque<BufferIndexOrError> loadedBuffers = new LinkedBlockingDeque<>();
+
+    private boolean isFailed;
+
+    public HsSubpartitionFileReader(
+            int subpartitionId,
+            FileChannel dataFileChannel,
+            HsSubpartitionViewOperation notifier,
+            HsFileDataIndex dataIndex,
+            int maxBufferReadAhead) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager(maxBufferReadAhead);
+        this.cachedRegionManager = new CachedRegionManager(subpartitionId, dataIndex);
+    }
+
+    /** Refresh downstream consumption progress for another round scheduling of reading. */
+    public void updateConsumptionProgress() {
+        bufferIndexManager.updateLastConsumed(notifier.getConsumingOffset());
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        HsSubpartitionFileReader that = (HsSubpartitionFileReader) o;
+        return subpartitionId == that.subpartitionId;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(subpartitionId);
+    }
+
+    /**
+     * Read subpartition data into buffers.
+     *
+     * <p>This transfers the ownership of used buffers to this class. It's this class'
+     * responsibility to release the buffers using the recycler when no longer needed.
+     *
+     * <p>Calling this method does not always use up all the provided buffers. It's this class'
+     * decision when to stop reading. Currently, it stops reading when: 1) buffers are used up, or
+     * 2) reaches the end of the subpartition data within the region, or 3) enough data have been
+     * read ahead the downstream consuming offset.
+     */
+    public synchronized void readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            throw new IOException("subpartition reader has already failed.");
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return;
+        }
+
+        int numRemainingBuffersInRegion =
+                cachedRegionManager.getRemainingBuffersInRegion(firstBufferToLoad);
+        // If lookup result is empty, it means that one the following things have happened:
+        // 1) The target buffer has not been spilled into disk.
+        // 2) The target buffer has not been released from memory.
+        // So, just skip this round reading.
+        if (numRemainingBuffersInRegion == 0) {
+            return;
+        }
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int indexToLoad;
+        int numLoaded = 0;
+        while (!buffers.isEmpty()
+                && numRemainingBuffersInRegion-- > 0
+                && (indexToLoad = bufferIndexManager.getNextToLoad()) >= 0) {
+            MemorySegment segment = buffers.poll();
+            Buffer buffer;
+            try {
+                if ((buffer = readFromByteChannel(dataFileChannel, headerBuf, segment, recycler))
+                        == null) {
+                    buffers.add(segment);
+                    break;
+                }
+            } catch (Throwable throwable) {
+                buffers.add(segment);
+                throw throwable;
+            }
+
+            loadedBuffers.add(BufferIndexOrError.newBuffer(buffer, indexToLoad));
+            bufferIndexManager.updateLastLoaded(indexToLoad);
+            cachedRegionManager.advance(buffer.readableBytes());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        if (isFailed) {
+            return;
+        }
+        isFailed = true;
+        BufferIndexOrError bufferIndexOrError;
+        // empty from tail, in-case subpartition view consumes concurrently and gets the wrong order
+        while ((bufferIndexOrError = loadedBuffers.pollLast()) != null) {
+            if (bufferIndexOrError.getThrowable().isPresent()) {
+                checkNotNull(bufferIndexOrError.buffer).recycleBuffer();
+            }
+        }
+
+        loadedBuffers.add(BufferIndexOrError.newError(failureCause));
+        notifier.notifyDataAvailableFromDisk();
+    }
+
+    /** Provides priority calculation logic for io scheduler. */
+    @Override
+    public int compareTo(HsSubpartitionFileReader that) {
+        return Long.compare(this.getNextOffsetToLoad(), that.getNextOffsetToLoad());
+    }
+
+    public void prepareForScheduling() {
+        updateConsumptionProgress();
+    }
+
+    // ------------------------------------------------------------------------
+    //  Methods only for testing
+    // ------------------------------------------------------------------------
+
+    @VisibleForTesting
+    Deque<BufferIndexOrError> getLoadedBuffers() {
+        return loadedBuffers;
+    }
+
+    @VisibleForTesting
+    BufferIndexManager getBufferIndexManager() {
+        return bufferIndexManager;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private void moveFileOffsetToBuffer(int bufferIndex) throws IOException {
+        Tuple2<Integer, Long> indexAndOffset =
+                cachedRegionManager.getNumSkipAndFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+    }
+
+    /** Returns Long.MAX_VALUE if shouldn't load. */
+    private long getNextOffsetToLoad() {
+        int bufferIndex = bufferIndexManager.getNextToLoad();
+        if (bufferIndex < 0) {
+            return Long.MAX_VALUE;
+        } else {
+            return cachedRegionManager.getFileOffset(bufferIndex);
+        }
+    }
+
+    /** Indicates a buffer with index or an error. */
+    public static class BufferIndexOrError {
+        @Nullable private final Buffer buffer;
+        private final int index;
+        @Nullable private final Throwable throwable;
+
+        private BufferIndexOrError(
+                @Nullable Buffer buffer, int index, @Nullable Throwable throwable) {
+            this.buffer = buffer;
+            this.index = index;
+            this.throwable = throwable;
+        }
+
+        public Buffer.DataType getDataType() {
+            return buffer == null ? Buffer.DataType.NONE : buffer.getDataType();
+        }
+
+        private static BufferIndexOrError newError(Throwable throwable) {
+            return new BufferIndexOrError(null, -1, checkNotNull(throwable));
+        }
+
+        private static BufferIndexOrError newBuffer(Buffer buffer, int index) {
+            return new BufferIndexOrError(checkNotNull(buffer), index, null);
+        }
+
+        public Optional<Buffer> getBuffer() {
+            return Optional.ofNullable(buffer);
+        }
+
+        public Optional<Throwable> getThrowable() {
+            return Optional.ofNullable(throwable);
+        }
+
+        public int getIndex() {
+            return index;
+        }
+    }
+
+    /** Take care of buffer index consumed by the file reader. */
+    static class BufferIndexManager {
+        private final int maxBuffersReadAhead;
+
+        /** Index of the last buffer that has ever been loaded from file. */
+        private int lastLoaded = -1;
+        /** Index of the last buffer that has been consumed by downstream, to the best knowledge. */
+        private int lastConsumed = -1;
+
+        BufferIndexManager(int maxBuffersReadAhead) {
+            this.maxBuffersReadAhead = maxBuffersReadAhead;
+        }
+
+        private void updateLastLoaded(int lastLoaded) {
+            checkState(this.lastLoaded <= lastLoaded);
+            this.lastLoaded = lastLoaded;
+        }
+
+        private void updateLastConsumed(int lastConsumed) {
+            this.lastConsumed = lastConsumed;
+        }
+
+        /** Returns a negative value if shouldn't load. */
+        int getNextToLoad() {
+            int nextToLoad = Math.max(lastLoaded, lastConsumed) + 1;
+            int maxToLoad = lastConsumed + maxBuffersReadAhead;
+            return nextToLoad <= maxToLoad ? nextToLoad : -1;
+        }
+    }
+
+    private static class CachedRegionManager {
+        private final int subpartitionId;
+        private final HsFileDataIndex dataIndex;
+
+        private int currentBufferIndex;
+        private int numSkip;
+        private int numReadable;
+        private long offset;
+
+        private CachedRegionManager(int subpartitionId, HsFileDataIndex dataIndex) {
+            this.subpartitionId = subpartitionId;
+            this.dataIndex = dataIndex;
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by HsSubpartitionFileReader
+        // ------------------------------------------------------------------------
+
+        /** Return Long.MAX_VALUE if region does not exist to giving the lowest priority. */
+        private long getFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+            return currentBufferIndex == -1 ? Long.MAX_VALUE : offset;
+        }
+
+        private int getRemainingBuffersInRegion(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            return numReadable;
+        }
+
+        /**
+         * Returns index and file offset of a buffer that is:
+         *
+         * <ol>
+         *   <li>In the same region as the requested buffer.
+         *   <li>Index is the largest less than or equal to the requested buffer.
+         *   <li>File offset is known.
+         * </ol>
+         */
+        private Tuple2<Integer, Long> getNumSkipAndFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            checkState(numSkip >= 0, "num skip must be greater than or equal to 0");
+            // Assumption: buffer index is always requested / updated increasingly
+            checkState(currentBufferIndex <= bufferIndex);
+            return new Tuple2<>(numSkip, offset);
+        }
+
+        private void advance(int bufferSize) {

Review Comment:
   ```suggestion
           private void advance(long bufferSize) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.positionToNextBuffer;
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.readFromByteChannel;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This component is responsible for reading data from disk.
+ *
+ * <p>In order to access the disk as sequentially as possible {@link HsSubpartitionFileReader} need
+ * to be able to compare priorities.
+ */
+public class HsSubpartitionFileReader implements Comparable<HsSubpartitionFileReader> {
+
+    private final ByteBuffer headerBuf = BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewOperation notifier;
+
+    private final CachedRegionManager cachedRegionManager;
+
+    private final BufferIndexManager bufferIndexManager;
+
+    private final Deque<BufferIndexOrError> loadedBuffers = new LinkedBlockingDeque<>();
+
+    private boolean isFailed;
+
+    public HsSubpartitionFileReader(
+            int subpartitionId,
+            FileChannel dataFileChannel,
+            HsSubpartitionViewOperation notifier,
+            HsFileDataIndex dataIndex,
+            int maxBufferReadAhead) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager(maxBufferReadAhead);
+        this.cachedRegionManager = new CachedRegionManager(subpartitionId, dataIndex);
+    }
+
+    /** Refresh downstream consumption progress for another round scheduling of reading. */
+    public void updateConsumptionProgress() {
+        bufferIndexManager.updateLastConsumed(notifier.getConsumingOffset());
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        HsSubpartitionFileReader that = (HsSubpartitionFileReader) o;
+        return subpartitionId == that.subpartitionId;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(subpartitionId);
+    }
+
+    /**
+     * Read subpartition data into buffers.
+     *
+     * <p>This transfers the ownership of used buffers to this class. It's this class'
+     * responsibility to release the buffers using the recycler when no longer needed.
+     *
+     * <p>Calling this method does not always use up all the provided buffers. It's this class'
+     * decision when to stop reading. Currently, it stops reading when: 1) buffers are used up, or
+     * 2) reaches the end of the subpartition data within the region, or 3) enough data have been
+     * read ahead the downstream consuming offset.
+     */
+    public synchronized void readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            throw new IOException("subpartition reader has already failed.");
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return;
+        }
+
+        int numRemainingBuffersInRegion =
+                cachedRegionManager.getRemainingBuffersInRegion(firstBufferToLoad);
+        // If lookup result is empty, it means that one the following things have happened:
+        // 1) The target buffer has not been spilled into disk.
+        // 2) The target buffer has not been released from memory.
+        // So, just skip this round reading.
+        if (numRemainingBuffersInRegion == 0) {
+            return;
+        }
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int indexToLoad;
+        int numLoaded = 0;
+        while (!buffers.isEmpty()
+                && numRemainingBuffersInRegion-- > 0
+                && (indexToLoad = bufferIndexManager.getNextToLoad()) >= 0) {
+            MemorySegment segment = buffers.poll();
+            Buffer buffer;
+            try {
+                if ((buffer = readFromByteChannel(dataFileChannel, headerBuf, segment, recycler))
+                        == null) {
+                    buffers.add(segment);
+                    break;
+                }
+            } catch (Throwable throwable) {
+                buffers.add(segment);
+                throw throwable;
+            }
+
+            loadedBuffers.add(BufferIndexOrError.newBuffer(buffer, indexToLoad));
+            bufferIndexManager.updateLastLoaded(indexToLoad);
+            cachedRegionManager.advance(buffer.readableBytes());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        if (isFailed) {
+            return;
+        }
+        isFailed = true;
+        BufferIndexOrError bufferIndexOrError;
+        // empty from tail, in-case subpartition view consumes concurrently and gets the wrong order
+        while ((bufferIndexOrError = loadedBuffers.pollLast()) != null) {
+            if (bufferIndexOrError.getThrowable().isPresent()) {
+                checkNotNull(bufferIndexOrError.buffer).recycleBuffer();
+            }
+        }
+
+        loadedBuffers.add(BufferIndexOrError.newError(failureCause));
+        notifier.notifyDataAvailableFromDisk();
+    }
+
+    /** Provides priority calculation logic for io scheduler. */
+    @Override
+    public int compareTo(HsSubpartitionFileReader that) {
+        return Long.compare(this.getNextOffsetToLoad(), that.getNextOffsetToLoad());
+    }
+
+    public void prepareForScheduling() {
+        updateConsumptionProgress();
+    }
+
+    // ------------------------------------------------------------------------
+    //  Methods only for testing
+    // ------------------------------------------------------------------------
+
+    @VisibleForTesting
+    Deque<BufferIndexOrError> getLoadedBuffers() {
+        return loadedBuffers;
+    }
+
+    @VisibleForTesting
+    BufferIndexManager getBufferIndexManager() {
+        return bufferIndexManager;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private void moveFileOffsetToBuffer(int bufferIndex) throws IOException {
+        Tuple2<Integer, Long> indexAndOffset =
+                cachedRegionManager.getNumSkipAndFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+    }
+
+    /** Returns Long.MAX_VALUE if shouldn't load. */
+    private long getNextOffsetToLoad() {
+        int bufferIndex = bufferIndexManager.getNextToLoad();
+        if (bufferIndex < 0) {
+            return Long.MAX_VALUE;
+        } else {
+            return cachedRegionManager.getFileOffset(bufferIndex);
+        }
+    }
+
+    /** Indicates a buffer with index or an error. */
+    public static class BufferIndexOrError {
+        @Nullable private final Buffer buffer;
+        private final int index;
+        @Nullable private final Throwable throwable;
+
+        private BufferIndexOrError(
+                @Nullable Buffer buffer, int index, @Nullable Throwable throwable) {
+            this.buffer = buffer;
+            this.index = index;
+            this.throwable = throwable;
+        }
+
+        public Buffer.DataType getDataType() {
+            return buffer == null ? Buffer.DataType.NONE : buffer.getDataType();
+        }
+
+        private static BufferIndexOrError newError(Throwable throwable) {
+            return new BufferIndexOrError(null, -1, checkNotNull(throwable));
+        }
+
+        private static BufferIndexOrError newBuffer(Buffer buffer, int index) {
+            return new BufferIndexOrError(checkNotNull(buffer), index, null);
+        }
+
+        public Optional<Buffer> getBuffer() {
+            return Optional.ofNullable(buffer);
+        }
+
+        public Optional<Throwable> getThrowable() {
+            return Optional.ofNullable(throwable);
+        }
+
+        public int getIndex() {
+            return index;
+        }
+    }
+
+    /** Take care of buffer index consumed by the file reader. */
+    static class BufferIndexManager {
+        private final int maxBuffersReadAhead;
+
+        /** Index of the last buffer that has ever been loaded from file. */
+        private int lastLoaded = -1;
+        /** Index of the last buffer that has been consumed by downstream, to the best knowledge. */
+        private int lastConsumed = -1;
+
+        BufferIndexManager(int maxBuffersReadAhead) {
+            this.maxBuffersReadAhead = maxBuffersReadAhead;
+        }
+
+        private void updateLastLoaded(int lastLoaded) {
+            checkState(this.lastLoaded <= lastLoaded);
+            this.lastLoaded = lastLoaded;
+        }
+
+        private void updateLastConsumed(int lastConsumed) {
+            this.lastConsumed = lastConsumed;
+        }
+
+        /** Returns a negative value if shouldn't load. */
+        int getNextToLoad() {
+            int nextToLoad = Math.max(lastLoaded, lastConsumed) + 1;
+            int maxToLoad = lastConsumed + maxBuffersReadAhead;
+            return nextToLoad <= maxToLoad ? nextToLoad : -1;
+        }
+    }
+
+    private static class CachedRegionManager {
+        private final int subpartitionId;
+        private final HsFileDataIndex dataIndex;
+
+        private int currentBufferIndex;
+        private int numSkip;
+        private int numReadable;
+        private long offset;
+
+        private CachedRegionManager(int subpartitionId, HsFileDataIndex dataIndex) {
+            this.subpartitionId = subpartitionId;
+            this.dataIndex = dataIndex;
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by HsSubpartitionFileReader
+        // ------------------------------------------------------------------------
+
+        /** Return Long.MAX_VALUE if region does not exist to giving the lowest priority. */
+        private long getFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+            return currentBufferIndex == -1 ? Long.MAX_VALUE : offset;
+        }
+
+        private int getRemainingBuffersInRegion(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            return numReadable;
+        }
+
+        /**
+         * Returns index and file offset of a buffer that is:
+         *
+         * <ol>
+         *   <li>In the same region as the requested buffer.
+         *   <li>Index is the largest less than or equal to the requested buffer.
+         *   <li>File offset is known.
+         * </ol>
+         */
+        private Tuple2<Integer, Long> getNumSkipAndFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            checkState(numSkip >= 0, "num skip must be greater than or equal to 0");
+            // Assumption: buffer index is always requested / updated increasingly
+            checkState(currentBufferIndex <= bufferIndex);
+            return new Tuple2<>(numSkip, offset);
+        }
+
+        private void advance(int bufferSize) {
+            if (isInCachedRegion(currentBufferIndex + 1)) {
+                numReadable--;
+                currentBufferIndex++;
+                offset += bufferSize;
+            }
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void updateCachedRegionIfNeeded(int bufferIndex) {
+            if (isInCachedRegion(bufferIndex)) {
+                numSkip = bufferIndex - currentBufferIndex;
+                return;

Review Comment:
   Shouldn't we also update `currentBufferIndex` and `numReadable` here? 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {
+                // fail all pending subpartition readers immediately if any exception occurs
+                failSubpartitionReaders(availableReaders, throwable);
+                LOG.error("Failed to request buffers for data reading.", throwable);
+            }
+        }
+        int numBuffersAllocated = buffers.size();
+
+        readData(availableReaders, buffers);
+
+        endCurrentRoundOfReading(numBuffersAllocated, buffers);
+    }
+
+    /** This method only called by result partition to create subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewOperation notifier) throws IOException {
+        synchronized (lock) {
+            lazyInitialize();
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId,
+                            dataFileChannel,
+                            notifier,
+                            dataIndex,
+                            hybridShuffleConfiguration.getMaxBuffersReadAhead());
+            allReaders.add(subpartitionReader);
+
+            mayTriggerReading();
+            return subpartitionReader;
+        }
+    }
+
+    /**
+     * Releases this read scheduler and returns a {@link CompletableFuture} which will be completed
+     * when all resources are released.
+     */
+    public synchronized CompletableFuture<?> release() {
+        List<HsSubpartitionFileReader> pendingReaders;
+        synchronized (lock) {
+            if (isReleased) {
+                return releaseFuture;
+            }
+            isReleased = true;
+
+            failedReaders.addAll(allReaders);
+            pendingReaders = new ArrayList<>(allReaders);
+            mayNotifyReleased();
+        }
+
+        failSubpartitionReaders(
+                pendingReaders,
+                new IllegalStateException("Result partition has been already released."));
+        return releaseFuture;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    @SuppressWarnings("GuardedBy")
+    private Queue<MemorySegment> allocateBuffers() throws Exception {
+        long timeoutTime = getBufferRequestTimeoutTime();
+        do {
+            List<MemorySegment> buffers = bufferPool.requestBuffers();
+            if (!buffers.isEmpty()) {
+                return new ArrayDeque<>(buffers);
+            }
+            checkState(!isReleased, "Result partition has been already released.");
+        } while (System.nanoTime() < timeoutTime
+                || System.nanoTime() < (timeoutTime = getBufferRequestTimeoutTime()));
+
+        if (numRequestedBuffers <= 0) {
+            // If the buffer request times out, an exception should be thrown to fail all
+            // subpartitionReaders, otherwise a deadlock may occur (Some operators have consumption
+            // priorities, if low-priority subpartitions occupy the buffer but do not consume, then
+            // high-priority subpartitions can't request buffer.)
+            throw new TimeoutException(
+                    String.format(
+                            "Buffer request timeout, this means there is a fierce contention of"
+                                    + " the batch shuffle read memory, please increase '%s'.",
+                            TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key()));
+        }
+        return new ArrayDeque<>();
+    }
+
+    @GuardedBy("lock")
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() <= maxRequestedBuffers
+                && numRequestedBuffers < bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void mayNotifyReleased() {
+        assert Thread.holdsLock(lock);
+
+        if (isReleased && allReaders.isEmpty()) {
+            releaseFuture.complete(null);
+        }
+    }
+
+    private long getBufferRequestTimeoutTime() {
+        return bufferPool.getLastBufferOperationTimestamp() + bufferRequestTimeout.toNanos();
+    }
+
+    private void releaseBuffers(Queue<MemorySegment> buffers) {
+        if (!buffers.isEmpty()) {
+            try {
+                bufferPool.recycle(buffers);
+                buffers.clear();
+            } catch (Throwable throwable) {
+                // this should never happen so just trigger fatal error
+                FatalExitExceptionHandler.INSTANCE.uncaughtException(
+                        Thread.currentThread(), throwable);
+            }
+        }
+    }
+
+    private Queue<HsSubpartitionFileReader> prepareAndGetAvailableReaders() {
+        synchronized (lock) {
+            if (isReleased) {
+                return new ArrayDeque<>();
+            }
+
+            try {
+                for (HsSubpartitionFileReader reader : allReaders) {
+                    reader.prepareForScheduling();
+                }
+            } catch (Throwable throwable) {

Review Comment:
   What are we trying to catch here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpiller.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
+import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.SpilledBuffer;
+
+import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * This component is responsible for asynchronously writing in-memory data to disk. Each spilling
+ * operation will write the disk file sequentially.
+ */
+public class HsMemoryDataSpiller implements AutoCloseable {
+    /** One thread to perform spill operation. */
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadScheduledExecutor(
+                    new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
+
+    /** File channel to write data. */
+    private final FileChannel dataFileChannel;
+
+    /** Records the current writing location. */
+    private long totalBytesWritten;
+
+    public HsMemoryDataSpiller(FileChannel dataFileChannel) {
+        this.dataFileChannel = dataFileChannel;
+    }
+
+    /**
+     * Spilling buffers to disk asynchronously.
+     *
+     * @param bufferToSpill buffers need to be spilled, must ensure that it is sorted by
+     *     (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;
+    }
+
+    /** Called in single-threaded ioExecutor. Order is guaranteed. */
+    private void spill(
+            List<BufferWithIdentity> toWrite,
+            CompletableFuture<List<SpilledBuffer>> spilledFuture) {
+        try {
+            List<SpilledBuffer> spilledBuffers = new ArrayList<>();
+            long expectedBytes = createSpilledBuffersAndGetTotalBytes(toWrite, spilledBuffers);
+            // write all buffers to file
+            writeBuffers(toWrite, expectedBytes);
+            // complete spill future when buffers are written to disk successfully.
+            // note that the ownership of these buffers is transferred to the MemoryDataManager,
+            // which controls data's life cycle.
+            // TODO update file data index and handle buffers release in future ticket.
+            spilledFuture.complete(spilledBuffers);
+        } catch (Throwable t) {
+            spilledFuture.completeExceptionally(t);
+        }
+    }
+
+    /**
+     * Compute buffer's file offset and create spilled buffers.
+     *
+     * @param toWrite for create {@link SpilledBuffer}.
+     * @param spilledBuffers receive the created {@link SpilledBuffer} by this method.
+     * @return total bytes(header size + buffer size) of all buffers to write.
+     */
+    private long createSpilledBuffersAndGetTotalBytes(
+            List<BufferWithIdentity> toWrite, List<SpilledBuffer> spilledBuffers) {
+        long expectedBytes = 0;
+        for (int i = 0; i < toWrite.size(); i++) {
+            BufferWithIdentity bufferWithIdentity = toWrite.get(i);

Review Comment:
   ```suggestion
           for (BufferWithIdentity bufferWithIdentity : toWrite) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {
+                // fail all pending subpartition readers immediately if any exception occurs
+                failSubpartitionReaders(availableReaders, throwable);
+                LOG.error("Failed to request buffers for data reading.", throwable);
+            }
+        }
+        int numBuffersAllocated = buffers.size();
+
+        readData(availableReaders, buffers);
+
+        endCurrentRoundOfReading(numBuffersAllocated, buffers);
+    }
+
+    /** This method only called by result partition to create subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewOperation notifier) throws IOException {
+        synchronized (lock) {
+            lazyInitialize();
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId,
+                            dataFileChannel,
+                            notifier,
+                            dataIndex,
+                            hybridShuffleConfiguration.getMaxBuffersReadAhead());
+            allReaders.add(subpartitionReader);
+
+            mayTriggerReading();
+            return subpartitionReader;
+        }
+    }
+
+    /**
+     * Releases this read scheduler and returns a {@link CompletableFuture} which will be completed
+     * when all resources are released.
+     */
+    public synchronized CompletableFuture<?> release() {
+        List<HsSubpartitionFileReader> pendingReaders;
+        synchronized (lock) {
+            if (isReleased) {
+                return releaseFuture;
+            }
+            isReleased = true;
+
+            failedReaders.addAll(allReaders);
+            pendingReaders = new ArrayList<>(allReaders);
+            mayNotifyReleased();
+        }
+
+        failSubpartitionReaders(
+                pendingReaders,
+                new IllegalStateException("Result partition has been already released."));
+        return releaseFuture;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    @SuppressWarnings("GuardedBy")
+    private Queue<MemorySegment> allocateBuffers() throws Exception {
+        long timeoutTime = getBufferRequestTimeoutTime();
+        do {
+            List<MemorySegment> buffers = bufferPool.requestBuffers();
+            if (!buffers.isEmpty()) {
+                return new ArrayDeque<>(buffers);
+            }
+            checkState(!isReleased, "Result partition has been already released.");
+        } while (System.nanoTime() < timeoutTime
+                || System.nanoTime() < (timeoutTime = getBufferRequestTimeoutTime()));
+
+        if (numRequestedBuffers <= 0) {
+            // If the buffer request times out, an exception should be thrown to fail all
+            // subpartitionReaders, otherwise a deadlock may occur (Some operators have consumption
+            // priorities, if low-priority subpartitions occupy the buffer but do not consume, then
+            // high-priority subpartitions can't request buffer.)
+            throw new TimeoutException(
+                    String.format(
+                            "Buffer request timeout, this means there is a fierce contention of"
+                                    + " the batch shuffle read memory, please increase '%s'.",
+                            TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key()));
+        }
+        return new ArrayDeque<>();
+    }
+
+    @GuardedBy("lock")
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() <= maxRequestedBuffers
+                && numRequestedBuffers < bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void mayNotifyReleased() {
+        assert Thread.holdsLock(lock);
+
+        if (isReleased && allReaders.isEmpty()) {
+            releaseFuture.complete(null);
+        }
+    }
+
+    private long getBufferRequestTimeoutTime() {
+        return bufferPool.getLastBufferOperationTimestamp() + bufferRequestTimeout.toNanos();
+    }
+
+    private void releaseBuffers(Queue<MemorySegment> buffers) {
+        if (!buffers.isEmpty()) {
+            try {
+                bufferPool.recycle(buffers);
+                buffers.clear();
+            } catch (Throwable throwable) {
+                // this should never happen so just trigger fatal error
+                FatalExitExceptionHandler.INSTANCE.uncaughtException(
+                        Thread.currentThread(), throwable);
+            }
+        }
+    }
+
+    private Queue<HsSubpartitionFileReader> prepareAndGetAvailableReaders() {
+        synchronized (lock) {
+            if (isReleased) {
+                return new ArrayDeque<>();
+            }
+
+            try {
+                for (HsSubpartitionFileReader reader : allReaders) {
+                    reader.prepareForScheduling();
+                }
+            } catch (Throwable throwable) {
+                failSubpartitionReaders(allReaders, throwable);
+            }
+
+            return new PriorityQueue<>(allReaders);
+        }
+    }
+
+    private void readData(
+            Queue<HsSubpartitionFileReader> availableReaders, Queue<MemorySegment> buffers) {
+        while (!availableReaders.isEmpty() && !buffers.isEmpty()) {
+            HsSubpartitionFileReader subpartitionReader = availableReaders.poll();
+            try {
+                subpartitionReader.readBuffers(buffers, this);
+            } catch (Throwable throwable) {
+                failSubpartitionReaders(Collections.singletonList(subpartitionReader), throwable);
+                LOG.debug("Failed to read shuffle data.", throwable);
+            }
+        }
+    }
+
+    private void failSubpartitionReaders(
+            Collection<HsSubpartitionFileReader> readers, Throwable failureCause) {
+        synchronized (lock) {
+            failedReaders.addAll(readers);
+        }
+
+        for (HsSubpartitionFileReader reader : readers) {
+            try {
+                reader.fail(failureCause);
+            } catch (Throwable throwable) {

Review Comment:
   What are we catching here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.positionToNextBuffer;
+import static org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil.readFromByteChannel;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This component is responsible for reading data from disk.
+ *
+ * <p>In order to access the disk as sequentially as possible {@link HsSubpartitionFileReader} need
+ * to be able to compare priorities.
+ */
+public class HsSubpartitionFileReader implements Comparable<HsSubpartitionFileReader> {
+
+    private final ByteBuffer headerBuf = BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewOperation notifier;
+
+    private final CachedRegionManager cachedRegionManager;
+
+    private final BufferIndexManager bufferIndexManager;
+
+    private final Deque<BufferIndexOrError> loadedBuffers = new LinkedBlockingDeque<>();
+
+    private boolean isFailed;
+
+    public HsSubpartitionFileReader(
+            int subpartitionId,
+            FileChannel dataFileChannel,
+            HsSubpartitionViewOperation notifier,
+            HsFileDataIndex dataIndex,
+            int maxBufferReadAhead) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager(maxBufferReadAhead);
+        this.cachedRegionManager = new CachedRegionManager(subpartitionId, dataIndex);
+    }
+
+    /** Refresh downstream consumption progress for another round scheduling of reading. */
+    public void updateConsumptionProgress() {
+        bufferIndexManager.updateLastConsumed(notifier.getConsumingOffset());
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        HsSubpartitionFileReader that = (HsSubpartitionFileReader) o;
+        return subpartitionId == that.subpartitionId;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(subpartitionId);
+    }
+
+    /**
+     * Read subpartition data into buffers.
+     *
+     * <p>This transfers the ownership of used buffers to this class. It's this class'
+     * responsibility to release the buffers using the recycler when no longer needed.
+     *
+     * <p>Calling this method does not always use up all the provided buffers. It's this class'
+     * decision when to stop reading. Currently, it stops reading when: 1) buffers are used up, or
+     * 2) reaches the end of the subpartition data within the region, or 3) enough data have been
+     * read ahead the downstream consuming offset.
+     */
+    public synchronized void readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            throw new IOException("subpartition reader has already failed.");
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return;
+        }
+
+        int numRemainingBuffersInRegion =
+                cachedRegionManager.getRemainingBuffersInRegion(firstBufferToLoad);
+        // If lookup result is empty, it means that one the following things have happened:
+        // 1) The target buffer has not been spilled into disk.
+        // 2) The target buffer has not been released from memory.
+        // So, just skip this round reading.
+        if (numRemainingBuffersInRegion == 0) {
+            return;
+        }
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int indexToLoad;
+        int numLoaded = 0;
+        while (!buffers.isEmpty()
+                && numRemainingBuffersInRegion-- > 0
+                && (indexToLoad = bufferIndexManager.getNextToLoad()) >= 0) {
+            MemorySegment segment = buffers.poll();
+            Buffer buffer;
+            try {
+                if ((buffer = readFromByteChannel(dataFileChannel, headerBuf, segment, recycler))
+                        == null) {
+                    buffers.add(segment);
+                    break;
+                }
+            } catch (Throwable throwable) {
+                buffers.add(segment);
+                throw throwable;
+            }
+
+            loadedBuffers.add(BufferIndexOrError.newBuffer(buffer, indexToLoad));
+            bufferIndexManager.updateLastLoaded(indexToLoad);
+            cachedRegionManager.advance(buffer.readableBytes());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        if (isFailed) {
+            return;
+        }
+        isFailed = true;
+        BufferIndexOrError bufferIndexOrError;
+        // empty from tail, in-case subpartition view consumes concurrently and gets the wrong order
+        while ((bufferIndexOrError = loadedBuffers.pollLast()) != null) {
+            if (bufferIndexOrError.getThrowable().isPresent()) {
+                checkNotNull(bufferIndexOrError.buffer).recycleBuffer();
+            }
+        }
+
+        loadedBuffers.add(BufferIndexOrError.newError(failureCause));
+        notifier.notifyDataAvailableFromDisk();
+    }
+
+    /** Provides priority calculation logic for io scheduler. */
+    @Override
+    public int compareTo(HsSubpartitionFileReader that) {
+        return Long.compare(this.getNextOffsetToLoad(), that.getNextOffsetToLoad());
+    }
+
+    public void prepareForScheduling() {
+        updateConsumptionProgress();
+    }
+
+    // ------------------------------------------------------------------------
+    //  Methods only for testing
+    // ------------------------------------------------------------------------
+
+    @VisibleForTesting
+    Deque<BufferIndexOrError> getLoadedBuffers() {
+        return loadedBuffers;
+    }
+
+    @VisibleForTesting
+    BufferIndexManager getBufferIndexManager() {
+        return bufferIndexManager;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private void moveFileOffsetToBuffer(int bufferIndex) throws IOException {
+        Tuple2<Integer, Long> indexAndOffset =
+                cachedRegionManager.getNumSkipAndFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+    }
+
+    /** Returns Long.MAX_VALUE if shouldn't load. */
+    private long getNextOffsetToLoad() {
+        int bufferIndex = bufferIndexManager.getNextToLoad();
+        if (bufferIndex < 0) {
+            return Long.MAX_VALUE;
+        } else {
+            return cachedRegionManager.getFileOffset(bufferIndex);
+        }
+    }
+
+    /** Indicates a buffer with index or an error. */
+    public static class BufferIndexOrError {
+        @Nullable private final Buffer buffer;
+        private final int index;
+        @Nullable private final Throwable throwable;
+
+        private BufferIndexOrError(
+                @Nullable Buffer buffer, int index, @Nullable Throwable throwable) {
+            this.buffer = buffer;
+            this.index = index;
+            this.throwable = throwable;
+        }
+
+        public Buffer.DataType getDataType() {
+            return buffer == null ? Buffer.DataType.NONE : buffer.getDataType();
+        }
+
+        private static BufferIndexOrError newError(Throwable throwable) {
+            return new BufferIndexOrError(null, -1, checkNotNull(throwable));
+        }
+
+        private static BufferIndexOrError newBuffer(Buffer buffer, int index) {
+            return new BufferIndexOrError(checkNotNull(buffer), index, null);
+        }
+
+        public Optional<Buffer> getBuffer() {
+            return Optional.ofNullable(buffer);
+        }
+
+        public Optional<Throwable> getThrowable() {
+            return Optional.ofNullable(throwable);
+        }
+
+        public int getIndex() {
+            return index;
+        }
+    }
+
+    /** Take care of buffer index consumed by the file reader. */
+    static class BufferIndexManager {
+        private final int maxBuffersReadAhead;
+
+        /** Index of the last buffer that has ever been loaded from file. */
+        private int lastLoaded = -1;
+        /** Index of the last buffer that has been consumed by downstream, to the best knowledge. */
+        private int lastConsumed = -1;
+
+        BufferIndexManager(int maxBuffersReadAhead) {
+            this.maxBuffersReadAhead = maxBuffersReadAhead;
+        }
+
+        private void updateLastLoaded(int lastLoaded) {
+            checkState(this.lastLoaded <= lastLoaded);
+            this.lastLoaded = lastLoaded;
+        }
+
+        private void updateLastConsumed(int lastConsumed) {
+            this.lastConsumed = lastConsumed;
+        }
+
+        /** Returns a negative value if shouldn't load. */
+        int getNextToLoad() {
+            int nextToLoad = Math.max(lastLoaded, lastConsumed) + 1;
+            int maxToLoad = lastConsumed + maxBuffersReadAhead;
+            return nextToLoad <= maxToLoad ? nextToLoad : -1;
+        }
+    }
+
+    private static class CachedRegionManager {
+        private final int subpartitionId;
+        private final HsFileDataIndex dataIndex;
+
+        private int currentBufferIndex;
+        private int numSkip;
+        private int numReadable;
+        private long offset;
+
+        private CachedRegionManager(int subpartitionId, HsFileDataIndex dataIndex) {
+            this.subpartitionId = subpartitionId;
+            this.dataIndex = dataIndex;
+        }
+
+        // ------------------------------------------------------------------------
+        //  Called by HsSubpartitionFileReader
+        // ------------------------------------------------------------------------
+
+        /** Return Long.MAX_VALUE if region does not exist to giving the lowest priority. */
+        private long getFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+            return currentBufferIndex == -1 ? Long.MAX_VALUE : offset;
+        }
+
+        private int getRemainingBuffersInRegion(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            return numReadable;
+        }
+
+        /**
+         * Returns index and file offset of a buffer that is:
+         *
+         * <ol>
+         *   <li>In the same region as the requested buffer.
+         *   <li>Index is the largest less than or equal to the requested buffer.
+         *   <li>File offset is known.
+         * </ol>
+         */
+        private Tuple2<Integer, Long> getNumSkipAndFileOffset(int bufferIndex) {
+            updateCachedRegionIfNeeded(bufferIndex);
+
+            checkState(numSkip >= 0, "num skip must be greater than or equal to 0");
+            // Assumption: buffer index is always requested / updated increasingly
+            checkState(currentBufferIndex <= bufferIndex);
+            return new Tuple2<>(numSkip, offset);
+        }
+
+        private void advance(int bufferSize) {
+            if (isInCachedRegion(currentBufferIndex + 1)) {
+                numReadable--;
+                currentBufferIndex++;
+                offset += bufferSize;
+            }
+        }
+
+        // ------------------------------------------------------------------------
+        //  Internal Methods
+        // ------------------------------------------------------------------------
+
+        private void updateCachedRegionIfNeeded(int bufferIndex) {
+            if (isInCachedRegion(bufferIndex)) {
+                numSkip = bufferIndex - currentBufferIndex;
+                return;
+            }
+
+            Optional<HsFileDataIndex.ReadableRegion> lookupResultOpt =
+                    dataIndex.getReadableRegion(subpartitionId, bufferIndex);
+            if (!lookupResultOpt.isPresent()) {
+                currentBufferIndex = -1;
+                numReadable = 0;
+                numSkip = 0;
+                offset = -1L;
+            } else {
+                HsFileDataIndex.ReadableRegion cachedRegion = lookupResultOpt.get();
+                currentBufferIndex = bufferIndex;
+                numSkip = cachedRegion.numSkip;
+                numReadable = cachedRegion.numReadable;
+                offset = cachedRegion.offset;
+            }
+        }
+
+        private boolean isInCachedRegion(int bufferIndex) {
+            return bufferIndex < currentBufferIndex + numReadable;

Review Comment:
   ```suggestion
               return bufferIndex < currentBufferIndex + numReadable && bufferIndex >= currentBufferIndex;
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {
+                // fail all pending subpartition readers immediately if any exception occurs
+                failSubpartitionReaders(availableReaders, throwable);
+                LOG.error("Failed to request buffers for data reading.", throwable);
+            }
+        }
+        int numBuffersAllocated = buffers.size();
+
+        readData(availableReaders, buffers);
+
+        endCurrentRoundOfReading(numBuffersAllocated, buffers);
+    }
+
+    /** This method only called by result partition to create subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewOperation notifier) throws IOException {
+        synchronized (lock) {
+            lazyInitialize();
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId,
+                            dataFileChannel,
+                            notifier,
+                            dataIndex,
+                            hybridShuffleConfiguration.getMaxBuffersReadAhead());
+            allReaders.add(subpartitionReader);
+
+            mayTriggerReading();
+            return subpartitionReader;
+        }
+    }
+
+    /**
+     * Releases this read scheduler and returns a {@link CompletableFuture} which will be completed
+     * when all resources are released.
+     */
+    public synchronized CompletableFuture<?> release() {
+        List<HsSubpartitionFileReader> pendingReaders;
+        synchronized (lock) {
+            if (isReleased) {
+                return releaseFuture;
+            }
+            isReleased = true;
+
+            failedReaders.addAll(allReaders);
+            pendingReaders = new ArrayList<>(allReaders);
+            mayNotifyReleased();
+        }
+
+        failSubpartitionReaders(
+                pendingReaders,
+                new IllegalStateException("Result partition has been already released."));
+        return releaseFuture;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    @SuppressWarnings("GuardedBy")
+    private Queue<MemorySegment> allocateBuffers() throws Exception {
+        long timeoutTime = getBufferRequestTimeoutTime();
+        do {
+            List<MemorySegment> buffers = bufferPool.requestBuffers();
+            if (!buffers.isEmpty()) {
+                return new ArrayDeque<>(buffers);
+            }
+            checkState(!isReleased, "Result partition has been already released.");
+        } while (System.nanoTime() < timeoutTime
+                || System.nanoTime() < (timeoutTime = getBufferRequestTimeoutTime()));
+
+        if (numRequestedBuffers <= 0) {
+            // If the buffer request times out, an exception should be thrown to fail all
+            // subpartitionReaders, otherwise a deadlock may occur (Some operators have consumption
+            // priorities, if low-priority subpartitions occupy the buffer but do not consume, then
+            // high-priority subpartitions can't request buffer.)
+            throw new TimeoutException(
+                    String.format(
+                            "Buffer request timeout, this means there is a fierce contention of"
+                                    + " the batch shuffle read memory, please increase '%s'.",
+                            TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key()));
+        }
+        return new ArrayDeque<>();
+    }
+
+    @GuardedBy("lock")
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() <= maxRequestedBuffers
+                && numRequestedBuffers < bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    @GuardedBy("lock")
+    private void mayNotifyReleased() {
+        assert Thread.holdsLock(lock);
+
+        if (isReleased && allReaders.isEmpty()) {
+            releaseFuture.complete(null);
+        }
+    }
+
+    private long getBufferRequestTimeoutTime() {
+        return bufferPool.getLastBufferOperationTimestamp() + bufferRequestTimeout.toNanos();
+    }
+
+    private void releaseBuffers(Queue<MemorySegment> buffers) {
+        if (!buffers.isEmpty()) {
+            try {
+                bufferPool.recycle(buffers);
+                buffers.clear();
+            } catch (Throwable throwable) {
+                // this should never happen so just trigger fatal error
+                FatalExitExceptionHandler.INSTANCE.uncaughtException(
+                        Thread.currentThread(), throwable);
+            }
+        }
+    }
+
+    private Queue<HsSubpartitionFileReader> prepareAndGetAvailableReaders() {
+        synchronized (lock) {
+            if (isReleased) {
+                return new ArrayDeque<>();
+            }
+
+            try {
+                for (HsSubpartitionFileReader reader : allReaders) {
+                    reader.prepareForScheduling();
+                }
+            } catch (Throwable throwable) {
+                failSubpartitionReaders(allReaders, throwable);
+            }
+
+            return new PriorityQueue<>(allReaders);
+        }
+    }
+
+    private void readData(
+            Queue<HsSubpartitionFileReader> availableReaders, Queue<MemorySegment> buffers) {
+        while (!availableReaders.isEmpty() && !buffers.isEmpty()) {
+            HsSubpartitionFileReader subpartitionReader = availableReaders.poll();
+            try {
+                subpartitionReader.readBuffers(buffers, this);
+            } catch (Throwable throwable) {
+                failSubpartitionReaders(Collections.singletonList(subpartitionReader), throwable);
+                LOG.debug("Failed to read shuffle data.", throwable);
+            }
+        }
+    }
+
+    private void failSubpartitionReaders(
+            Collection<HsSubpartitionFileReader> readers, Throwable failureCause) {
+        synchronized (lock) {
+            failedReaders.addAll(readers);
+        }
+
+        for (HsSubpartitionFileReader reader : readers) {
+            try {
+                reader.fail(failureCause);
+            } catch (Throwable throwable) {
+                // this should never happen so just trigger fatal error
+                FatalExitExceptionHandler.INSTANCE.uncaughtException(
+                        Thread.currentThread(), throwable);
+            }
+        }
+    }
+
+    private void endCurrentRoundOfReading(int numBuffersAllocated, Queue<MemorySegment> buffers) {
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);

Review Comment:
   I would do these in `run()`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {
+                // fail all pending subpartition readers immediately if any exception occurs
+                failSubpartitionReaders(availableReaders, throwable);
+                LOG.error("Failed to request buffers for data reading.", throwable);
+            }
+        }
+        int numBuffersAllocated = buffers.size();
+
+        readData(availableReaders, buffers);
+
+        endCurrentRoundOfReading(numBuffersAllocated, buffers);
+    }
+
+    /** This method only called by result partition to create subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewOperation notifier) throws IOException {
+        synchronized (lock) {
+            lazyInitialize();
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId,
+                            dataFileChannel,
+                            notifier,
+                            dataIndex,
+                            hybridShuffleConfiguration.getMaxBuffersReadAhead());
+            allReaders.add(subpartitionReader);
+
+            mayTriggerReading();
+            return subpartitionReader;
+        }
+    }
+
+    /**
+     * Releases this read scheduler and returns a {@link CompletableFuture} which will be completed
+     * when all resources are released.
+     */
+    public synchronized CompletableFuture<?> release() {
+        List<HsSubpartitionFileReader> pendingReaders;
+        synchronized (lock) {
+            if (isReleased) {
+                return releaseFuture;
+            }
+            isReleased = true;
+
+            failedReaders.addAll(allReaders);
+            pendingReaders = new ArrayList<>(allReaders);
+            mayNotifyReleased();
+        }
+
+        failSubpartitionReaders(
+                pendingReaders,
+                new IllegalStateException("Result partition has been already released."));
+        return releaseFuture;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    @SuppressWarnings("GuardedBy")

Review Comment:
   ```suggestion
       @SuppressWarnings("FieldAccessNotGuarded")
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,408 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+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.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.time.Duration;
+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 HsResultPartition which can schedule {@link HsSubpartitionFileReader} and read
+ * data for all downstream tasks.
+ */
+public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler {
+    private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class);
+
+    /** 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;
+
+    /**
+     * Maximum time to wait when requesting read buffers from the buffer pool before throwing an
+     * exception.
+     */
+    private final Duration bufferRequestTimeout;
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. */
+    private final Object lock = new Object();
+
+    /**
+     * 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;
+
+    private final Path dataFilePath;
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> failedReaders = new HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<HsSubpartitionFileReader> allReaders = new HashSet<>();
+
+    /**
+     * 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;
+
+    /** Number of buffers already allocated and still not recycled by this partition reader. */
+    @GuardedBy("lock")
+    private volatile int numRequestedBuffers;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private volatile boolean isReleased;
+
+    private FileChannel dataFileChannel;
+
+    public HsResultPartitionReadScheduler(
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            HybridShuffleConfiguration hybridShuffleConfiguration) {
+        this.hybridShuffleConfiguration = checkNotNull(hybridShuffleConfiguration);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        this.maxRequestedBuffers = hybridShuffleConfiguration.getMaxRequestedBuffers();
+        this.bufferRequestTimeout =
+                checkNotNull(hybridShuffleConfiguration.getBufferRequestTimeout());
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = prepareAndGetAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers = allocateBuffers();
+            } catch (Throwable throwable) {

Review Comment:
   ```suggestion
               } catch (Exception exception) {
   ```



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