gaoyunhaii commented on a change in pull request #13595: URL: https://github.com/apache/flink/pull/13595#discussion_r510851102
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileReader.java ########## @@ -0,0 +1,153 @@ +/* + * 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.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Reader which can read all data of the target subpartition from a {@link PartitionedFile}. + */ +public class PartitionedFileReader implements AutoCloseable { + + /** Used to read buffers from file channel. */ + private final ByteBuffer headerBuf = BufferReaderWriterUtil.allocatedHeaderBuffer(); + + /** Used to read index entry from index file. */ + private final ByteBuffer indexEntryBuf; + + /** Target {@link PartitionedFile} to read. */ + private final PartitionedFile partitionedFile; + + /** Target subpartition to read. */ + private final int targetSubpartition; + + /** Data file channel of the target {@link PartitionedFile}. */ + private FileChannel dataFileChannel; + + /** Index file channel of the target {@link PartitionedFile}. */ + private FileChannel indexFileChannel; + + /** Next data region to be read. */ + private int nextRegionToRead; + + /** Number of remaining buffers in the current data region read. */ + private int currentRegionRemainingBuffers; + + /** Whether this partitioned file reader is closed. */ + private boolean isClosed; + + public PartitionedFileReader(PartitionedFile partitionedFile, int targetSubpartition) { + this.partitionedFile = checkNotNull(partitionedFile); + this.targetSubpartition = targetSubpartition; + + this.indexEntryBuf = ByteBuffer.allocate(PartitionedFile.INDEX_ENTRY_SIZE); + indexEntryBuf.order(PartitionedFile.DEFAULT_BYTE_ORDER); + } + + /** + * Opens the given {@link PartitionedFile} and moves read position to the starting offset of the + * target subpartition. + */ + public void open() throws IOException { + checkState(dataFileChannel == null && indexFileChannel == null, "File reader is already opened."); + checkState(!isClosed, "File reader is already closed."); + + dataFileChannel = FileChannel.open(partitionedFile.getDataFilePath(), StandardOpenOption.READ); + indexFileChannel = FileChannel.open(partitionedFile.getIndexFilePath(), StandardOpenOption.READ); + + moveToNextReadableRegion(); + } + + private boolean moveToNextReadableRegion() throws IOException { + while (currentRegionRemainingBuffers == 0 && nextRegionToRead < partitionedFile.getNumRegions()) { + partitionedFile.getIndexEntry(indexFileChannel, indexEntryBuf, nextRegionToRead, targetSubpartition); + long dataOffset = indexEntryBuf.getLong(); + currentRegionRemainingBuffers = indexEntryBuf.getInt(); + + ++nextRegionToRead; + dataFileChannel.position(dataOffset); + } + + return currentRegionRemainingBuffers > 0; + } + + /** + * Reads a buffer from the {@link PartitionedFile} and moves the read position forward. + * + * <p>Note: The caller is responsible for recycling the target buffer if any exception occurs. + */ + @Nullable + public Buffer readBuffer(MemorySegment target, BufferRecycler recycler) throws IOException { + checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first."); + checkState(!isClosed, "File reader is already closed."); + + if (moveToNextReadableRegion()) { + --currentRegionRemainingBuffers; + return BufferReaderWriterUtil.readFromByteChannel(dataFileChannel, headerBuf, target, recycler); + } + + return null; + } + + public boolean hasRemaining() throws IOException { Review comment: @VisibleForTesting ? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBufferTest.java ########## @@ -0,0 +1,291 @@ +/* + * 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.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.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; + +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link PartitionSortedBuffer}. + */ +public class PartitionSortedBufferTest { + + @Test + public void testWriteAndReadSortBuffer() throws Exception { + int numSubpartitions = 10; + int bufferSize = 1024; + int bufferPoolSize = 1000; + Random random = new Random(); Review comment: I think we might set the random seed explicitly so that the test is deterministic, then it would be easy to track if there is problems. ########## File path: flink-tests/src/test/java/org/apache/flink/test/runtime/BlockingShuffleITCase.java ########## @@ -0,0 +1,100 @@ +/* + * 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.test.runtime; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.graph.GlobalDataExchangeMode; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; + +import org.junit.Test; + +/** + * Tests for blocking shuffle. + */ +public class BlockingShuffleITCase { + + private static final String RECORD = "hello, world!"; + + private final int numTaskManagers = 2; + + private final int numSlotsPerTaskManager = 4; + + @Test + public void testBoundedBlockingShuffle() throws Exception { + JobGraph jobGraph = createJobGraph(); + JobGraphRunningUtil.execute(jobGraph, new Configuration(), numTaskManagers, numSlotsPerTaskManager); + } + + @Test + public void testSortMergeBlockingShuffle() throws Exception { + Configuration configuration = new Configuration(); + configuration.setInteger(NettyShuffleEnvironmentOptions.NETWORK_SORT_MERGE_SHUFFLE_MIN_PARALLELISM, 1); + + JobGraph jobGraph = createJobGraph(); + JobGraphRunningUtil.execute(jobGraph, configuration, numTaskManagers, numSlotsPerTaskManager); + } + + private JobGraph createJobGraph() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(numTaskManagers * numSlotsPerTaskManager); + DataStream<String> source = env.addSource(new StringSource()); + source + .rebalance().map((MapFunction<String, String>) value -> value) + .broadcast().addSink(new VerifySink()); + + StreamGraph streamGraph = env.getStreamGraph(); + streamGraph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_BLOCKING); + streamGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES); + return StreamingJobGraphGenerator.createJobGraph(streamGraph); + } + + private static class StringSource implements ParallelSourceFunction<String> { + private volatile boolean isRunning = true; + private int numRecordsToSend = 1000000; + + @Override + public void run(SourceContext<String> ctx) throws Exception { + while (isRunning && numRecordsToSend-- > 0) { + ctx.collect(RECORD); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class VerifySink implements SinkFunction<String> { + + @Override + public void invoke(String value) throws Exception { + assert value.equals(RECORD); Review comment: might change to `assertEquals` ? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBufferTest.java ########## @@ -0,0 +1,291 @@ +/* + * 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.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.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; + +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link PartitionSortedBuffer}. + */ +public class PartitionSortedBufferTest { + + @Test + public void testWriteAndReadSortBuffer() throws Exception { + int numSubpartitions = 10; + int bufferSize = 1024; + int bufferPoolSize = 1000; + Random random = new Random(); + + // used to store data written to and read from sort buffer for correctness check + Queue<DataAndType>[] dataWritten = new Queue[numSubpartitions]; + Queue<Buffer>[] buffersRead = new Queue[numSubpartitions]; + for (int i = 0; i < numSubpartitions; ++i) { + dataWritten[i] = new ArrayDeque<>(); + buffersRead[i] = new ArrayDeque<>(); + } + + int[] numBytesWritten = new int[numSubpartitions]; + int[] numBytesRead = new int[numSubpartitions]; + Arrays.fill(numBytesWritten, 0); + Arrays.fill(numBytesRead, 0); + + // fill the sort buffer with randomly generated data + int totalBytesWritten = 0; + SortBuffer sortBuffer = createSortBuffer(bufferPoolSize, bufferSize, numSubpartitions); Review comment: Do we need to finally cleanup the sortbuffer, local buffer pool and global buffer pool ? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriteReadTest.java ########## @@ -0,0 +1,249 @@ +/* + * 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.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.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +/** + * Tests for writing and reading {@link PartitionedFile} with {@link PartitionedFileWriter} + * and {@link PartitionedFileReader}. + */ +public class PartitionedFileWriteReadTest { + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void testWriteAndReadPartitionedFile() throws Exception { + int numSubpartitions = 10; + int bufferSize = 1024; + int numBuffers = 1000; + int numRegions = 10; + Random random = new Random(); Review comment: Similarly might set seed. ---------------------------------------------------------------- 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]
