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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);

Review Comment:
   I'd suggest to introduce a `HybridShuffleConfiguration` that contains all 
the possible parameters. All hybrid shuffle components should read the 
parameters from it. We can later decide which parameters should be exposed as 
configuration options and which should be constants, and only the codes for 
creating this `HybridShuffleConfiguration` needs to be touched.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            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) {

Review Comment:
   `numRequestedBuffers`: accessed outside of the declared guards



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            long timeoutTime = getBufferRequestTimeoutTime();
+            do {
+                List<MemorySegment> buffers = bufferPool.requestBuffers();
+                if (!buffers.isEmpty()) {
+                    return new ArrayDeque<>(buffers);
+                }
+                checkState(!isReleased, "Result partition has been already 
released.");

Review Comment:
   `isReleased`: accessed outside of the declared guards



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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> {
+    /**
+     * Prevent other consumers from starving. This default value needs to be 
tuning experimentally
+     * or introduced a configuration option.
+     */
+    public static final int MAX_BUFFERS_READ_AHEAD = 5;

Review Comment:
   Same here for the parameter.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {

Review Comment:
   Why does this method need the argument `availableReaders`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            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) {
+                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()));
+            }
+        } 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);
+        }
+        return new ArrayDeque<>();
+    }
+
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() 
<= maxRequestedBuffers
+                && numRequestedBuffers < 
bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    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> getAvailableReaders() {
+        synchronized (lock) {
+            if (isReleased) {
+                return new ArrayDeque<>();
+            }
+
+            try {
+                for (HsSubpartitionFileReader reader : allReaders) {
+                    reader.prepareForScheduling();
+                }

Review Comment:
   It is too implicit preparing the readers here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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> {
+    /**
+     * Prevent other consumers from starving. This default value needs to be 
tuning experimentally
+     * or introduced a configuration option.
+     */
+    public static final int MAX_BUFFERS_READ_AHEAD = 5;
+
+    private final ByteBuffer headerBuf = 
BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewNotifier 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,
+            HsSubpartitionViewNotifier notifier,
+            HsFileDataIndex dataIndex) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager();
+        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 boolean readBuffers(Queue<MemorySegment> buffers, 
BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            return false;
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return true;
+        }
+
+        // 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 
(!cachedRegionManager.updateCachedRegionIfNeeded(firstBufferToLoad).isPresent())
 {
+            return true;
+        }
+
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int numRemainingBuffersInRegion =
+                
cachedRegionManager.getRemainingBuffersInRegion(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.updateBufferAndFileOffset(indexToLoad, 
dataFileChannel.position());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+
+        return true;
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        checkState(!isFailed, "subpartition file reader has already failed.");
+        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.isError()) {
+                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.tryGetFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < bufferIndex - indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+        dataFileChannel.position(indexAndOffset.f1);
+    }
+
+    /** 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 public final Buffer buffer;
+        public final int index;
+        public final Throwable throwable;

Review Comment:
   I think we should make these fields private and wrap them with getters that 
returns `Optional`, to force the caller to do the null check.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewNotifier.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+/** A notifier used to send messages from {@link HsSubpartitionFileReader} to 
HsSubpartitionView. */
+public interface HsSubpartitionViewNotifier {
+
+    /**
+     * Used to notify subpartitionView that data on disk is ready.
+     *
+     * <p>Attention: This method only called by IO scheduling thread.
+     */
+    void notifyDataAvailableFromDisk();
+
+    /**
+     * Used to get subpartitionView's newest consuming progress. *
+     *
+     * <p>Attention: This method only called by IO scheduling thread.
+     */
+    int getConsumingOffset();

Review Comment:
   This does not fit the interface name `HsSubpartitionViewNotifier`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java:
##########
@@ -162,8 +162,19 @@ static Buffer readFileRegionFromByteChannel(FileChannel 
channel, ByteBuffer head
         return new FileRegionBuffer(channel, position, size, dataType, 
isCompressed);
     }
 
+    public static int setBufferWithHeader(
+            Buffer buffer, ByteBuffer[] bufferWithHeaders, int index) {
+        ByteBuffer header = BufferReaderWriterUtil.allocatedHeaderBuffer();
+        BufferReaderWriterUtil.setByteChannelBufferHeader(buffer, header);
+
+        bufferWithHeaders[index] = header;
+        bufferWithHeaders[index + 1] = buffer.getNioBufferReadable();
+
+        return header.remaining() + buffer.readableBytes();
+    }

Review Comment:
   I find this method very confusing. It's hard to understand what does it do 
and what the arguments and return value mean.
   
   After looking into its implementation and how it is used on the caller side, 
I find it does three things.
   1. Generate a header for the buffer.
   2. Put the header and the buffer into an array at a specific position.
   3. Return the sum of the length of the buffer and its header.
   
   I think only 1) should be included as a common util. 2) can be done on the 
caller side. 3) is never used.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java:
##########
@@ -200,12 +211,32 @@ static Buffer readFromByteChannel(
         return new NetworkBuffer(memorySegment, bufferRecycler, dataType, 
isCompressed, size);
     }
 
-    static ByteBuffer allocatedHeaderBuffer() {
+    public static ByteBuffer allocatedHeaderBuffer() {
         ByteBuffer bb = ByteBuffer.allocateDirect(HEADER_LENGTH);
         configureByteBuffer(bb);
         return bb;
     }
 
+    public static void positionToNextBuffer(FileChannel channel, ByteBuffer 
headerBuffer)

Review Comment:
   This should be tested.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);

Review Comment:
   Same here for this parameter.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);

Review Comment:
   We can put all the lazy initialization logics into one method.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,

Review Comment:
   Why is the lock needs to be passed in as an argument?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            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) {
+                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()));
+            }
+        } 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);
+        }
+        return new ArrayDeque<>();
+    }
+
+    private void mayTriggerReading() {
+        assert Thread.holdsLock(lock);
+
+        if (!isRunning
+                && !allReaders.isEmpty()
+                && numRequestedBuffers + bufferPool.getNumBuffersPerRequest() 
<= maxRequestedBuffers
+                && numRequestedBuffers < 
bufferPool.getAverageBuffersPerRequester()) {
+            isRunning = true;
+            ioExecutor.execute(this);
+        }
+    }
+
+    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> getAvailableReaders() {
+        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 Set<HsSubpartitionFileReader> readData(
+            Queue<HsSubpartitionFileReader> availableReaders, 
Queue<MemorySegment> buffers) {
+        Set<HsSubpartitionFileReader> finishedReaders = new HashSet<>();
+
+        while (!availableReaders.isEmpty() && !buffers.isEmpty()) {
+            HsSubpartitionFileReader subpartitionReader = 
availableReaders.poll();
+            try {
+                if (!subpartitionReader.readBuffers(buffers, this)) {
+                    // there is no resource to release for finished readers 
currently
+                    finishedReaders.add(subpartitionReader);
+                }
+            } catch (Throwable throwable) {
+                
failSubpartitionReaders(Collections.singletonList(subpartitionReader), 
throwable);
+                LOG.debug("Failed to read shuffle data.", throwable);
+            }
+        }
+        return finishedReaders;
+    }
+
+    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 removeFinishedAndFailedReaders(
+            int numBuffersRead, Set<HsSubpartitionFileReader> finishedReaders) 
{
+        synchronized (lock) {
+            for (HsSubpartitionFileReader reader : finishedReaders) {
+                allReaders.remove(reader);
+            }
+            finishedReaders.clear();
+
+            for (HsSubpartitionFileReader reader : failedReaders) {
+                bufferPool.unregisterRequester(this);
+                allReaders.remove(reader);
+            }
+            failedReaders.clear();
+
+            if (allReaders.isEmpty()) {
+                closeFileChannel();
+            }
+
+            numRequestedBuffers += numBuffersRead;
+            isRunning = false;
+            mayTriggerReading();
+            mayNotifyReleased();

Review Comment:
   Why are these logic in this method? They have nothing to do with *remove 
finished and failed readers*.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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> {
+    /**
+     * Prevent other consumers from starving. This default value needs to be 
tuning experimentally
+     * or introduced a configuration option.
+     */
+    public static final int MAX_BUFFERS_READ_AHEAD = 5;
+
+    private final ByteBuffer headerBuf = 
BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewNotifier 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,
+            HsSubpartitionViewNotifier notifier,
+            HsFileDataIndex dataIndex) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager();
+        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 boolean readBuffers(Queue<MemorySegment> buffers, 
BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            return false;
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return true;
+        }
+
+        // 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 
(!cachedRegionManager.updateCachedRegionIfNeeded(firstBufferToLoad).isPresent())
 {
+            return true;
+        }
+
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int numRemainingBuffersInRegion =
+                
cachedRegionManager.getRemainingBuffersInRegion(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.updateBufferAndFileOffset(indexToLoad, 
dataFileChannel.position());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+
+        return true;
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        checkState(!isFailed, "subpartition file reader has already failed.");
+        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.isError()) {
+                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.tryGetFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < bufferIndex - indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+        dataFileChannel.position(indexAndOffset.f1);
+    }
+
+    /** 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 public final Buffer buffer;
+        public final int index;
+        public final Throwable throwable;
+
+        private BufferIndexOrError(@Nullable Buffer buffer, int index, 
Throwable throwable) {
+            this.buffer = buffer;
+            this.index = index;
+            this.throwable = throwable;
+        }
+
+        public boolean isError() {
+            return buffer == null;
+        }
+
+        public Buffer.DataType getDataType() {
+            return buffer == null ? Buffer.DataType.NONE : 
buffer.getDataType();
+        }
+
+        private static BufferIndexOrError newError(Throwable throwable) {
+            return new BufferIndexOrError(null, -1, throwable);
+        }
+
+        private static BufferIndexOrError newBuffer(Buffer buffer, int index) {
+            return new BufferIndexOrError(checkNotNull(buffer), index, null);
+        }
+    }
+
+    /** Take care of buffer index consumed by the file reader. */
+    static class BufferIndexManager {
+        /** 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;
+
+        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 + MAX_BUFFERS_READ_AHEAD;
+            return nextToLoad <= maxToLoad ? nextToLoad : -1;
+        }
+    }
+
+    private static class CachedRegionManager {

Review Comment:
   I think currently the class is complex because we are maintaining cursors 
for both first and current buffer. It can be simplified by only maintain a set 
of cursors that points out the offset of the current buffer, and introduce a 
set of reset / move methods.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java:
##########
@@ -200,12 +211,32 @@ static Buffer readFromByteChannel(
         return new NetworkBuffer(memorySegment, bufferRecycler, dataType, 
isCompressed, size);
     }
 
-    static ByteBuffer allocatedHeaderBuffer() {
+    public static ByteBuffer allocatedHeaderBuffer() {
         ByteBuffer bb = ByteBuffer.allocateDirect(HEADER_LENGTH);
         configureByteBuffer(bb);
         return bb;
     }
 
+    public static void positionToNextBuffer(FileChannel channel, ByteBuffer 
headerBuffer)

Review Comment:
   Same for `setBufferWithHeader`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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> {
+    /**
+     * Prevent other consumers from starving. This default value needs to be 
tuning experimentally
+     * or introduced a configuration option.
+     */
+    public static final int MAX_BUFFERS_READ_AHEAD = 5;
+
+    private final ByteBuffer headerBuf = 
BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewNotifier 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,
+            HsSubpartitionViewNotifier notifier,
+            HsFileDataIndex dataIndex) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager();
+        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 boolean readBuffers(Queue<MemorySegment> buffers, 
BufferRecycler recycler)
+            throws IOException {
+        if (isFailed) {
+            return false;
+        }
+        int firstBufferToLoad = bufferIndexManager.getNextToLoad();
+        if (firstBufferToLoad < 0) {
+            return true;
+        }
+
+        // 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 
(!cachedRegionManager.updateCachedRegionIfNeeded(firstBufferToLoad).isPresent())
 {
+            return true;
+        }
+
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int numRemainingBuffersInRegion =
+                
cachedRegionManager.getRemainingBuffersInRegion(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.updateBufferAndFileOffset(indexToLoad, 
dataFileChannel.position());
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            notifier.notifyDataAvailableFromDisk();
+        }
+
+        return true;
+    }
+
+    public synchronized void fail(Throwable failureCause) {
+        checkState(!isFailed, "subpartition file reader has already failed.");
+        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.isError()) {
+                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.tryGetFileOffset(bufferIndex);
+        dataFileChannel.position(indexAndOffset.f1);
+        for (int i = 0; i < bufferIndex - indexAndOffset.f0; ++i) {
+            positionToNextBuffer(dataFileChannel, headerBuf);
+        }
+        dataFileChannel.position(indexAndOffset.f1);
+    }
+
+    /** 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 public final Buffer buffer;
+        public final int index;
+        public final Throwable throwable;

Review Comment:
   This should also be nullable



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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);
+
+    /**
+     * Default maximum time (5min) to wait when requesting read buffers from 
the buffer pool before
+     * throwing an exception.
+     */
+    private static final Duration DEFAULT_BUFFER_REQUEST_TIMEOUT = 
Duration.ofMinutes(5);
+
+    /** 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;
+
+    /**
+     * 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;
+
+    /** 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(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Path dataFilePath,
+            HsFileDataIndex dataIndex,
+            Object lock) {
+        this(
+                numSubpartitions,
+                bufferPool,
+                ioExecutor,
+                lock,
+                dataIndex,
+                dataFilePath,
+                DEFAULT_BUFFER_REQUEST_TIMEOUT);
+    }
+
+    public HsResultPartitionReadScheduler(
+            int numSubpartitions,
+            BatchShuffleReadBufferPool bufferPool,
+            Executor ioExecutor,
+            Object lock,
+            HsFileDataIndex dataIndex,
+            Path dataFilePath,
+            Duration bufferRequestTimeout) {
+        this.lock = checkNotNull(lock);
+        this.dataIndex = checkNotNull(dataIndex);
+        this.dataFilePath = checkNotNull(dataFilePath);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most Math.max(16M, 
numSubpartitions) (the expected
+        // buffers per request is 8M) buffers for data read, which means 
larger parallelism, more
+        // buffers. Currently, it is only an empirical strategy which can not 
be configured.
+        this.maxRequestedBuffers =
+                Math.max(2 * bufferPool.getNumBuffersPerRequest(), 
numSubpartitions);
+        this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout);
+    }
+
+    @Override
+    public synchronized void run() {
+        Queue<HsSubpartitionFileReader> availableReaders = 
getAvailableReaders();
+        Queue<MemorySegment> buffers = allocateBuffers(availableReaders);
+        int numBuffersAllocated = buffers.size();
+
+        Set<HsSubpartitionFileReader> finishedReaders = 
readData(availableReaders, buffers);
+
+        int numBuffersRead = numBuffersAllocated - buffers.size();
+        releaseBuffers(buffers);
+
+        removeFinishedAndFailedReaders(numBuffersRead, finishedReaders);
+    }
+
+    /** This method only called by result partition to create 
subpartitionFileReader. */
+    public HsSubpartitionFileReader registerNewSubpartition(
+            int subpartitionId, HsSubpartitionViewNotifier notifier) throws 
IOException {
+        synchronized (lock) {
+            try {
+                if (allReaders.isEmpty()) {
+                    dataFileChannel = openFileChannel(dataFilePath);
+                }
+            } catch (Throwable throwable) {
+                if (allReaders.isEmpty()) {
+                    closeFileChannel();
+                }
+                throw throwable;
+            }
+
+            HsSubpartitionFileReader subpartitionReader =
+                    new HsSubpartitionFileReader(
+                            subpartitionId, dataFileChannel, notifier, 
dataIndex);
+            if (allReaders.isEmpty()) {
+                bufferPool.registerRequester(this);
+            }
+            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
+    // ------------------------------------------------------------------------
+
+    private Queue<MemorySegment> 
allocateBuffers(Queue<HsSubpartitionFileReader> availableReaders) {
+        if (availableReaders.isEmpty()) {
+            return new ArrayDeque<>();
+        }
+
+        try {
+            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) {

Review Comment:
   There're quite some similar issues. I'm not pointing out all of them.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,379 @@
+/*
+ * 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> {
+    /**
+     * Prevent other consumers from starving. This default value needs to be 
tuning experimentally
+     * or introduced a configuration option.
+     */
+    public static final int MAX_BUFFERS_READ_AHEAD = 5;
+
+    private final ByteBuffer headerBuf = 
BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewNotifier 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,
+            HsSubpartitionViewNotifier notifier,
+            HsFileDataIndex dataIndex) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.notifier = notifier;
+        this.bufferIndexManager = new BufferIndexManager();
+        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 boolean readBuffers(Queue<MemorySegment> buffers, 
BufferRecycler recycler)

Review Comment:
   What does the return value mean?



-- 
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: [email protected]

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

Reply via email to