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


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

Review Comment:
   fixed.



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

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

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

Reply via email to