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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.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.checkArgument;
+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 for a specific 
subpartition.
+ *
+ * <p>In order to access the disk as sequentially as possible {@link 
HsSubpartitionFileReader} need
+ * to be able to compare priorities.
+ *
+ * <p>Note: This class is not thread safe.
+ */
+public class HsSubpartitionFileReader implements SubpartitionFileReader {
+
+    private final ByteBuffer headerBuf = 
BufferReaderWriterUtil.allocatedHeaderBuffer();
+
+    private final int subpartitionId;
+
+    private final FileChannel dataFileChannel;
+
+    private final HsSubpartitionViewInternalOperations operations;
+
+    private final CachedRegionManager cachedRegionManager;
+
+    private final BufferIndexManager bufferIndexManager;
+
+    private final Deque<BufferIndexOrError> loadedBuffers = new 
LinkedBlockingDeque<>();
+
+    private volatile boolean isFailed;
+
+    public HsSubpartitionFileReader(
+            int subpartitionId,
+            FileChannel dataFileChannel,
+            HsSubpartitionViewInternalOperations operations,
+            HsFileDataIndex dataIndex,
+            int maxBufferReadAhead) {
+        this.subpartitionId = subpartitionId;
+        this.dataFileChannel = dataFileChannel;
+        this.operations = operations;
+        this.bufferIndexManager = new BufferIndexManager(maxBufferReadAhead);
+        this.cachedRegionManager = new CachedRegionManager(subpartitionId, 
dataIndex);
+    }
+
+    @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.
+     */
+    @Override
+    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;
+        }
+
+        // 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.
+        int numRemainingBuffer = 
cachedRegionManager.getRemainingBuffersInRegion(firstBufferToLoad);
+        if (numRemainingBuffer == 0) {
+            return;
+        }
+        moveFileOffsetToBuffer(firstBufferToLoad);
+
+        int indexToLoad;
+        int numLoaded = 0;
+        while (!buffers.isEmpty()
+                && (indexToLoad = bufferIndexManager.getNextToLoad()) >= 0
+                && numRemainingBuffer-- > 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() + 
BufferReaderWriterUtil.HEADER_LENGTH);
+            ++numLoaded;
+        }
+
+        if (loadedBuffers.size() <= numLoaded) {
+            operations.notifyDataAvailableFromDisk();
+        }
+    }
+
+    @Override
+    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.getBuffer().isPresent()) {
+                checkNotNull(bufferIndexOrError.buffer).recycleBuffer();
+            }
+        }
+
+        loadedBuffers.add(BufferIndexOrError.newError(failureCause));
+        operations.notifyDataAvailableFromDisk();
+    }
+
+    @Override
+    public void prepareForScheduling() {
+        updateConsumptionProgress();
+    }
+
+    public Deque<BufferIndexOrError> getLoadedBuffers() {
+        return loadedBuffers;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    /** Refresh downstream consumption progress for another round scheduling 
of reading. */
+    private void updateConsumptionProgress() {
+        bufferIndexManager.updateLastConsumed(operations.getConsumingOffset());
+    }
+
+    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);
+        }
+        cachedRegionManager.skipAll(dataFileChannel.position());
+    }
+
+    /** Returns Long.MAX_VALUE if it shouldn't load. */
+    private long getNextOffsetToLoad() {
+        int bufferIndex = bufferIndexManager.getNextToLoad();
+        if (bufferIndex < 0) {
+            return Long.MAX_VALUE;
+        } else {
+            return cachedRegionManager.getFileOffset(bufferIndex);
+        }
+    }
+
+    /** Provides priority calculation logic for io scheduler. */
+    @Override
+    public int compareTo(SubpartitionFileReader that) {

Review Comment:
   good catch.



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