zhuzhurk commented on a change in pull request #13924: URL: https://github.com/apache/flink/pull/13924#discussion_r601217982
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java ########## @@ -0,0 +1,365 @@ +/* + * 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; + +import org.apache.flink.annotation.VisibleForTesting; +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.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.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.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 {@link SortMergeResultPartition} which can read data for all downstream tasks + * consuming the corresponding {@link SortMergeResultPartition}. It always tries to read shuffle + * data in order of file offset, which maximums the sequential read so can improve the blocking + * shuffle performance. + */ +public class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler { + + private static final Logger LOG = + LoggerFactory.getLogger(SortMergeResultPartitionReadScheduler.class); + + /** Lock used to synchronize multi-thread access to thread-unsafe fields. */ + private final Object lock; + + /** Corresponding {@link SortMergeResultPartition} to read data from. */ + private final SortMergeResultPartition partition; + + /** Buffer pool from which to allocate buffers for shuffle data reading. */ + private final BatchShuffleReadBufferPool bufferPool; + + /** 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; + + /** All failed subpartition readers to be released. */ + @GuardedBy("lock") + private final Set<SortMergeSubpartitionReader> failedReaders = new HashSet<>(); + + /** All readers waiting to read data of different subpartitions. */ + @GuardedBy("lock") + private final Set<SortMergeSubpartitionReader> allReaders = new HashSet<>(); + + /** File channel shared by all subpartitions to read data from. */ + @GuardedBy("lock") + private FileChannel dataFileChannel; + + /** File channel shared by all subpartitions to read index from. */ + @GuardedBy("lock") + private FileChannel indexFileChannel; + + /** Number of buffers already allocated and still not recycled by this partition reader. */ + @GuardedBy("lock") + private int numRequestedBuffers; + + /** + * 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; + + /** Whether this reader has been released or not. */ + @GuardedBy("lock") + private boolean isReleased; + + public SortMergeResultPartitionReadScheduler( + SortMergeResultPartition partition, Review comment: It's better to just pass in a release listener/function. This makes it possible to test this scheduler without creating a `SortMergeResultPartition` and to get rid of the `SortMergeResultPartition#getReadScheduler()`. Besides that, I think we need some unit tests for this class. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java ########## @@ -19,154 +19,182 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; 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.ResultSubpartition.BufferAndBacklog; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.IOUtils; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayDeque; import java.util.Queue; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** Subpartition data reader for {@link SortMergeResultPartition}. */ -public class SortMergeSubpartitionReader implements ResultSubpartitionView, BufferRecycler { +public class SortMergeSubpartitionReader + implements ResultSubpartitionView, Comparable<SortMergeSubpartitionReader> { - private static final int NUM_READ_BUFFERS = 2; + private final Object lock = new Object(); - /** Target {@link SortMergeResultPartition} to read data from. */ - private final SortMergeResultPartition partition; + /** + * Corresponding {@link SortMergeResultPartitionReadScheduler} of the target result partition. + */ + private final SortMergeResultPartitionReadScheduler readScheduler; /** Listener to notify when data is available. */ private final BufferAvailabilityListener availabilityListener; - /** Unmanaged memory used as read buffers. */ - private final Queue<MemorySegment> readBuffers = new ArrayDeque<>(); - - /** Buffers read by the file reader. */ + /** Buffers already read which can be consumed by netty thread. */ + @GuardedBy("lock") private final Queue<Buffer> buffersRead = new ArrayDeque<>(); /** File reader used to read buffer from. */ private final PartitionedFileReader fileReader; - /** Number of remaining non-event buffers to read. */ + /** Number of remaining non-event buffers in the buffer queue. */ + @GuardedBy("lock") private int dataBufferBacklog; /** Whether this reader is released or not. */ + @GuardedBy("lock") private boolean isReleased; + /** Cause of failure which should be propagated to the consumer. */ + @GuardedBy("lock") + private Throwable failureCause; + /** Sequence number of the next buffer to be sent to the consumer. */ private int sequenceNumber; public SortMergeSubpartitionReader( Review comment: This class and its constructor can be package private. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java ########## @@ -0,0 +1,365 @@ +/* + * 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; + +import org.apache.flink.annotation.VisibleForTesting; +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.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.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.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 {@link SortMergeResultPartition} which can read data for all downstream tasks + * consuming the corresponding {@link SortMergeResultPartition}. It always tries to read shuffle + * data in order of file offset, which maximums the sequential read so can improve the blocking + * shuffle performance. + */ +public class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler { + + private static final Logger LOG = + LoggerFactory.getLogger(SortMergeResultPartitionReadScheduler.class); + + /** Lock used to synchronize multi-thread access to thread-unsafe fields. */ + private final Object lock; + + /** Corresponding {@link SortMergeResultPartition} to read data from. */ + private final SortMergeResultPartition partition; + + /** Buffer pool from which to allocate buffers for shuffle data reading. */ + private final BatchShuffleReadBufferPool bufferPool; + + /** 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; + + /** All failed subpartition readers to be released. */ + @GuardedBy("lock") + private final Set<SortMergeSubpartitionReader> failedReaders = new HashSet<>(); + + /** All readers waiting to read data of different subpartitions. */ + @GuardedBy("lock") + private final Set<SortMergeSubpartitionReader> allReaders = new HashSet<>(); + + /** File channel shared by all subpartitions to read data from. */ + @GuardedBy("lock") + private FileChannel dataFileChannel; + + /** File channel shared by all subpartitions to read index from. */ + @GuardedBy("lock") + private FileChannel indexFileChannel; + + /** Number of buffers already allocated and still not recycled by this partition reader. */ + @GuardedBy("lock") + private int numRequestedBuffers; + + /** + * 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; + + /** Whether this reader has been released or not. */ + @GuardedBy("lock") + private boolean isReleased; + + public SortMergeResultPartitionReadScheduler( Review comment: This class and its constructor can be package private. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java ########## @@ -19,154 +19,182 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; 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.ResultSubpartition.BufferAndBacklog; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.IOUtils; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayDeque; import java.util.Queue; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** Subpartition data reader for {@link SortMergeResultPartition}. */ -public class SortMergeSubpartitionReader implements ResultSubpartitionView, BufferRecycler { +public class SortMergeSubpartitionReader + implements ResultSubpartitionView, Comparable<SortMergeSubpartitionReader> { - private static final int NUM_READ_BUFFERS = 2; + private final Object lock = new Object(); - /** Target {@link SortMergeResultPartition} to read data from. */ - private final SortMergeResultPartition partition; + /** + * Corresponding {@link SortMergeResultPartitionReadScheduler} of the target result partition. + */ + private final SortMergeResultPartitionReadScheduler readScheduler; /** Listener to notify when data is available. */ private final BufferAvailabilityListener availabilityListener; - /** Unmanaged memory used as read buffers. */ - private final Queue<MemorySegment> readBuffers = new ArrayDeque<>(); - - /** Buffers read by the file reader. */ + /** Buffers already read which can be consumed by netty thread. */ + @GuardedBy("lock") private final Queue<Buffer> buffersRead = new ArrayDeque<>(); /** File reader used to read buffer from. */ private final PartitionedFileReader fileReader; - /** Number of remaining non-event buffers to read. */ + /** Number of remaining non-event buffers in the buffer queue. */ + @GuardedBy("lock") private int dataBufferBacklog; /** Whether this reader is released or not. */ + @GuardedBy("lock") private boolean isReleased; + /** Cause of failure which should be propagated to the consumer. */ + @GuardedBy("lock") + private Throwable failureCause; + /** Sequence number of the next buffer to be sent to the consumer. */ private int sequenceNumber; public SortMergeSubpartitionReader( - int subpartitionIndex, - int dataBufferBacklog, - int bufferSize, - SortMergeResultPartition partition, + SortMergeResultPartitionReadScheduler readScheduler, Review comment: It's better to just pass in a release listener/function. This makes it possible to test this reader without creating a `SortMergeResultPartitionReadScheduler `. I also think we need some unit tests for this class. -- 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. For queries about this service, please contact Infrastructure at: [email protected]
