xintongsong commented on code in PR #19960: URL: https://github.com/apache/flink/pull/19960#discussion_r913578661
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadScheduler.java: ########## @@ -0,0 +1,396 @@ +/* + * 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.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 javax.annotation.concurrent.ThreadSafe; + +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; + +/** + * IO scheduler for HsResultPartition, which schedules {@link HsSubpartitionFileReader} for loading + * data w.r.t. their offset in the file. + */ +@ThreadSafe +public class HsResultPartitionReadScheduler implements Runnable, BufferRecycler { + private static final Logger LOG = LoggerFactory.getLogger(HsResultPartitionReadScheduler.class); + + /** Executor to run the shuffle data reading task. */ + private final Executor ioExecutor; + + /** Maximum number of buffers can be allocated by this partition reader. */ + private final int maxRequestedBuffers; + + /** + * Maximum time to wait when requesting read buffers from the buffer pool before throwing an + * exception. + */ + private final Duration bufferRequestTimeout; + + /** Lock used to synchronize multi-thread access to thread-unsafe fields. */ + private final Object lock = new Object(); + + /** + * A {@link CompletableFuture} to be completed when this read scheduler including all resources + * is released. + */ + private final CompletableFuture<?> releaseFuture = new CompletableFuture<>(); Review Comment: ```suggestion @GuardedBy("lock") private final CompletableFuture<?> releaseFuture = new CompletableFuture<>(); ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java: ########## @@ -0,0 +1,390 @@ +/* + * 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.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 Comparable<HsSubpartitionFileReader> { + + 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 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. + */ + public 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() + BufferReaderWriterUtil.HEADER_LENGTH); + ++numLoaded; + } + + if (loadedBuffers.size() <= numLoaded) { + operations.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)); + operations.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(); + } + + 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.numSkip--; + } + cachedRegionManager.offset = dataFileChannel.position(); Review Comment: Should not access `numSkip` and `offset` outside `cachedRegionManager`. We can introduce a method `CachedRegionManager#skipAll(long offset)`. ########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReader.java: ########## @@ -0,0 +1,390 @@ +/* + * 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.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 Comparable<HsSubpartitionFileReader> { + + 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 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. + */ + public 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) { Review Comment: ```suggestion while (!buffers.isEmpty() && (indexToLoad = bufferIndexManager.getNextToLoad()) >= 0 && cachedRegionManager.getRemainingBuffersInRegion(indexToLoad) > 0) { ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadSchedulerTest.java: ########## @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition.hybrid; + +import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.Deque; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link HsResultPartitionReadScheduler}. */ +@ExtendWith(TestLoggerExtension.class) +@Timeout(60) +class HsResultPartitionReadSchedulerTest { + private static final int BUFFER_SIZE = 1024; + + private static final int NUM_THREADS = 4; + + private static final int NUM_SUBPARTITIONS = 10; + + private final byte[] dataBytes = new byte[BUFFER_SIZE]; + + private BatchShuffleReadBufferPool bufferPool; + + private ExecutorService executor; + + private FileChannel dataFileChannel; + + private Path dataFilePath; + + private HsResultPartitionReadScheduler readScheduler; + + private TestingSubpartitionViewInternalOperation subpartitionViewOperation; + + @BeforeEach + void before(@TempDir Path tempDir) throws IOException { + Random random = new Random(); + random.nextBytes(dataBytes); + bufferPool = new BatchShuffleReadBufferPool(BUFFER_SIZE, BUFFER_SIZE); + executor = Executors.newFixedThreadPool(NUM_THREADS); + dataFilePath = Files.createFile(tempDir.resolve(".data")); + dataFileChannel = openFileChannel(dataFilePath); + readScheduler = + new HsResultPartitionReadScheduler( + bufferPool, + executor, + new TestingDataIndex(), + dataFilePath, + HybridShuffleConfiguration.createConfiguration( + bufferPool.getNumBuffersPerRequest(), NUM_SUBPARTITIONS)); + subpartitionViewOperation = new TestingSubpartitionViewInternalOperation(); + } + + @AfterEach + void after() throws Exception { + bufferPool.destroy(); + executor.shutdown(); + if (dataFileChannel != null) { + dataFileChannel.close(); + } + } + + @Test + void testOnReadBufferRequestError() throws Exception { + HsSubpartitionFileReader fileReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + bufferPool.destroy(); Review Comment: Why destroy the buffer pool? ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadSchedulerTest.java: ########## @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition.hybrid; + +import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.Deque; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link HsResultPartitionReadScheduler}. */ +@ExtendWith(TestLoggerExtension.class) +@Timeout(60) +class HsResultPartitionReadSchedulerTest { + private static final int BUFFER_SIZE = 1024; + + private static final int NUM_THREADS = 4; + + private static final int NUM_SUBPARTITIONS = 10; + + private final byte[] dataBytes = new byte[BUFFER_SIZE]; + + private BatchShuffleReadBufferPool bufferPool; + + private ExecutorService executor; + + private FileChannel dataFileChannel; + + private Path dataFilePath; + + private HsResultPartitionReadScheduler readScheduler; + + private TestingSubpartitionViewInternalOperation subpartitionViewOperation; + + @BeforeEach + void before(@TempDir Path tempDir) throws IOException { + Random random = new Random(); + random.nextBytes(dataBytes); + bufferPool = new BatchShuffleReadBufferPool(BUFFER_SIZE, BUFFER_SIZE); + executor = Executors.newFixedThreadPool(NUM_THREADS); + dataFilePath = Files.createFile(tempDir.resolve(".data")); + dataFileChannel = openFileChannel(dataFilePath); + readScheduler = + new HsResultPartitionReadScheduler( + bufferPool, + executor, + new TestingDataIndex(), + dataFilePath, + HybridShuffleConfiguration.createConfiguration( + bufferPool.getNumBuffersPerRequest(), NUM_SUBPARTITIONS)); + subpartitionViewOperation = new TestingSubpartitionViewInternalOperation(); + } + + @AfterEach + void after() throws Exception { + bufferPool.destroy(); + executor.shutdown(); + if (dataFileChannel != null) { + dataFileChannel.close(); + } + } + + @Test + void testOnReadBufferRequestError() throws Exception { + HsSubpartitionFileReader fileReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + bufferPool.destroy(); + + waitReaderGetErrorAndCheck(fileReader, RuntimeException.class); + } + + @Test + void testReadSchedulerReleaseWhileReading() throws Exception { + HsSubpartitionFileReader fileReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + Thread.sleep(1000); Review Comment: Why sleep? ########## flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionReadSchedulerTest.java: ########## @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition.hybrid; + +import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.Deque; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link HsResultPartitionReadScheduler}. */ +@ExtendWith(TestLoggerExtension.class) +@Timeout(60) +class HsResultPartitionReadSchedulerTest { + private static final int BUFFER_SIZE = 1024; + + private static final int NUM_THREADS = 4; + + private static final int NUM_SUBPARTITIONS = 10; + + private final byte[] dataBytes = new byte[BUFFER_SIZE]; + + private BatchShuffleReadBufferPool bufferPool; + + private ExecutorService executor; + + private FileChannel dataFileChannel; + + private Path dataFilePath; + + private HsResultPartitionReadScheduler readScheduler; + + private TestingSubpartitionViewInternalOperation subpartitionViewOperation; + + @BeforeEach + void before(@TempDir Path tempDir) throws IOException { + Random random = new Random(); + random.nextBytes(dataBytes); + bufferPool = new BatchShuffleReadBufferPool(BUFFER_SIZE, BUFFER_SIZE); + executor = Executors.newFixedThreadPool(NUM_THREADS); + dataFilePath = Files.createFile(tempDir.resolve(".data")); + dataFileChannel = openFileChannel(dataFilePath); + readScheduler = + new HsResultPartitionReadScheduler( + bufferPool, + executor, + new TestingDataIndex(), + dataFilePath, + HybridShuffleConfiguration.createConfiguration( + bufferPool.getNumBuffersPerRequest(), NUM_SUBPARTITIONS)); + subpartitionViewOperation = new TestingSubpartitionViewInternalOperation(); + } + + @AfterEach + void after() throws Exception { + bufferPool.destroy(); + executor.shutdown(); + if (dataFileChannel != null) { + dataFileChannel.close(); + } + } + + @Test + void testOnReadBufferRequestError() throws Exception { + HsSubpartitionFileReader fileReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + bufferPool.destroy(); + + waitReaderGetErrorAndCheck(fileReader, RuntimeException.class); + } + + @Test + void testReadSchedulerReleaseWhileReading() throws Exception { + HsSubpartitionFileReader fileReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + Thread.sleep(1000); + readScheduler.release(); + + waitReaderGetErrorAndCheck(fileReader, IllegalStateException.class); + } + + @Test + void testRegisterSubpartitionReaderAfterSchedulerReleased() { + readScheduler.release(); + assertThatThrownBy( + () -> readScheduler.registerNewSubpartition(0, subpartitionViewOperation)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("HsResultPartitionReadScheduler is already released."); + } + + @Test + void testRequestBufferTimeoutAndFailed() throws Exception { + Duration bufferRequestTimeout = Duration.ofSeconds(3); + bufferPool.requestBuffers(); + readScheduler = + new HsResultPartitionReadScheduler( + bufferPool, + executor, + new TestingDataIndex(), + dataFilePath, + HybridShuffleConfiguration.createConfiguration( + bufferPool.getNumBuffersPerRequest(), + NUM_SUBPARTITIONS, + bufferRequestTimeout)); + + HsSubpartitionFileReader subpartitionReader = + readScheduler.registerNewSubpartition(0, subpartitionViewOperation); + + waitReaderGetErrorAndCheck(subpartitionReader, TimeoutException.class); + } + + private void waitReaderGetErrorAndCheck( + HsSubpartitionFileReader reader, Class<?> expectedException) throws Exception { + Deque<HsSubpartitionFileReader.BufferIndexOrError> loadedBuffers = + reader.getLoadedBuffers(); + while (loadedBuffers.size() == 0) { + Thread.sleep(100); + } + + assertThat(loadedBuffers).hasSize(1); + assertThat(loadedBuffers.poll()) + .satisfies( + bufferIndexOrError -> + assertThat(bufferIndexOrError.getThrowable()) + .hasValueSatisfying( + throwable -> + assertThat(throwable) + .isInstanceOf(expectedException))); + } Review Comment: ```suggestion private void expectError( HsSubpartitionFileReader reader, Class<?> expectedException, long timeoutMS) throws Exception { Deque<HsSubpartitionFileReader.BufferIndexOrError> loadedBuffers = reader.getLoadedBuffers(); long waited = 0L; long interval = Math.max(1L, Math.min(10L, timeoutMS / 10)); while (waited < timeoutMS) { HsSubpartitionFileReader.BufferIndexOrError bufferIndexOrError = loadedBuffers.poll(); if (bufferIndexOrError != null && bufferIndexOrError.getThrowable().isPresent()) { assertThat(bufferIndexOrError.getThrowable()) .hasValueSatisfying( throwable -> assertThat(throwable).isInstanceOf(expectedException)); return; } Thread.sleep(interval); waited += interval; } fail(); } ``` -- 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]
