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

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

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


##########
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:
   fixed.



##########
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 refactored this class.



##########
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:
   By checking the code, I found that there is no need to return a value here. This part of the logic has been changed, you can review it again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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