You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/12 14:19:09 UTC

[GitHub] [flink] wsry opened a new pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

wsry opened a new pull request #13595:
URL: https://github.com/apache/flink/pull/13595


   ## What is the purpose of the change
   
   Hash-based blocking shuffle and sort-merge based blocking shuffle are two main blocking shuffle implementations wildly adopted by existing distributed data processing frameworks. Hash-based implementation writes data sent to different reducer tasks into separate files concurrently while sort-merge based approach writes those data together into a single file and merges those small files into bigger ones. Compared to sort-merge based approach, hash-based approach has several weak points when it comes to running large scale batch jobs. By introducing the sort-merge based approach to Flink, we can improve Flinkā€™s capability of running large scale batch jobs.
   
   
   ## Brief change log
   
     - Introduce SortBuffer and its implementation PartitionSortedBuffer for sort-merge based blocking shuffle
     - Introduce PartitionedFile and the corresponding writer/reader for sort-merge based blocking shuffle
     - Introduce sort-merge based result partition SortMergeResultPartition and the corresponding subpartition reader
     - Introduce new config options to enable sort-merge based blocking shuffle
     - Introduce shuffle data compression to sort-merge based blocking shuffle
   
   
   ## Verifying this change
   
   Several new tests are added to verify this change, including PartitionSortedBufferTest, PartitionedFileWriteReadTest, SortMergeResultPartitionTest and BlockingShuffleITCase.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / **docs** / **JavaDocs** / not documented)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-718141948


   Looks good form my side!
   
   Let's wait for @zhijiangW and @gaoyunhaii and CI to give their ok as well.
   
   We can start looking at documentation in the meantime. I think we need a quick description of this and how to configure it in the memory configuration section: 
     - https://ci.apache.org/projects/flink/flink-docs-master  ==> Deployment & Operations ==> Memory Configuration
     - Or add it to the tuning page: https://ci.apache.org/projects/flink/flink-docs-master/ops/memory/mem_tuning.html


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511595610



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFile.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.util.IOUtils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link PartitionedFile} is the persistent file type of sort-merge blocking shuffle. Each {@link PartitionedFile}
+ * contains two files: one is data file and the other is index file. Both the data file and index file have multiple
+ * regions. Each data region store the shuffle data in subpartition index order and the corresponding index region
+ * contains index entries of all subpartitions. Each index entry is a (long, integer) tuple of which the long value
+ * is the file offset of the corresponding subpartition and the integer value is the number of buffers.
+ */
+public class PartitionedFile {
+
+	public static final String DATA_FILE_SUFFIX = ".shuffle.data";
+
+	public static final String INDEX_FILE_SUFFIX = ".shuffle.index";
+
+	public static final ByteOrder DEFAULT_BYTE_ORDER = ByteOrder.BIG_ENDIAN;
+
+	/** Size of each index entry in the index file. 8 bytes for offset and 4 bytes for number of buffers. */
+	public static final int INDEX_ENTRY_SIZE = 8 + 4;
+
+	/** Number of data regions in this {@link PartitionedFile}. */
+	private final int numRegions;
+
+	/** Number of subpartitions of this {@link PartitionedFile}. */
+	private final int numSubpartitions;
+
+	/** Path of the data file which stores all data in this {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Path of the index file which stores indexes of all regions in this {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Used to accelerate index data access. */
+	private final ByteBuffer indexDataCache;
+
+	public PartitionedFile(
+			int numRegions,
+			int numSubpartitions,
+			Path dataFilePath,
+			Path indexFilePath,
+			ByteBuffer indexDataCache) {
+		checkArgument(numRegions >= 0, "Illegal number of data regions.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numRegions = numRegions;
+		this.numSubpartitions = numSubpartitions;
+		this.dataFilePath = checkNotNull(dataFilePath);
+		this.indexFilePath = checkNotNull(indexFilePath);
+		this.indexDataCache = indexDataCache;
+	}
+
+	public Path getDataFilePath() {
+		return dataFilePath;
+	}
+
+	public Path getIndexFilePath() {
+		return indexFilePath;
+	}
+
+	public int getNumRegions() {
+		return numRegions;
+	}
+
+	/**
+	 * Returns the index entry offset of the target region and subpartition in the index file. Both region index
+	 * and subpartition index start from 0.
+	 */
+	private long getIndexEntryOffset(int region, int subpartition) {

Review comment:
       We need long when reading from file




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r512634338



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter implements AutoCloseable {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data is cached in memory or not. */
+	private boolean allIndexDataCached = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions, int indexBufferSize) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+		checkArgument(indexBufferSize > 0, "Illegal index buffer size.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(indexBufferSize * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					allIndexDataCached = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		indexBuffer.flip();
+		if (indexBuffer.limit() > 0) {
+			BufferReaderWriterUtil.writeBuffer(indexFileChannel, indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		isFinished = true;
+
+		writeRegionIndex();
+		flushIndexBuffer();
+		indexBuffer.rewind();
+
+		close();
+		ByteBuffer indexDataCache = allIndexDataCached ? indexBuffer : null;
+		return new PartitionedFile(numRegions, numSubpartitions, dataFilePath, indexFilePath, indexDataCache);
+	}
+
+	/**
+	 * Used to close and delete the failed {@link PartitionedFile} when any exception occurs.
+	 */
+	public void releaseQuietly() {
+		close();
+		IOUtils.deleteFileQuietly(dataFilePath);
+		IOUtils.deleteFileQuietly(indexFilePath);
+	}
+
+	@Override
+	public void close() {
+		Throwable exception = null;
+
+		try {
+			dataFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = throwable;
+		}
+
+		try {
+			dataFileChannel.close();

Review comment:
       Looks like there is a copy/paste error here, and this should be `indexFileChannel`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f5e7c69522f61d5d992b85371340f232b59f3cdd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950) 
   * 0ea0d823be70a718ca9ac3c7684e8b46552ad634 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035) 
   * d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511621407



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());

Review comment:
       It guarantees that the result partitioned file will not be produced if the result partition is already released.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510814191



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					canCacheAllIndexData = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		if (indexBuffer.position() > 0) {
+			indexBuffer.flip();
+			indexFileChannel.write(indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");

Review comment:
       The `dataFileChannel` and `indexFileChannel` are created via data output driven. I am wondering that does it exist the case of empty output? Maybe the operator filters all the input data without any output, then finished finally.
   
   If so, we should remove above check to avoid unnecessary failure.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510737752



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFile.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.util.IOUtils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link PartitionedFile} is the persistent file type of sort-merge blocking shuffle. Each {@link PartitionedFile}
+ * contains two files: one is data file and the other is index file. Both the data file and index file have multiple
+ * regions. Each data region store the shuffle data in subpartition index order and the corresponding index region
+ * contains index entries of all subpartitions. Each index entry is a (long, integer) tuple of which the long value
+ * is the file offset of the corresponding subpartition and the integer value is the number of buffers.
+ */
+public class PartitionedFile {
+
+	public static final String DATA_FILE_SUFFIX = ".shuffle.data";
+
+	public static final String INDEX_FILE_SUFFIX = ".shuffle.index";
+
+	public static final ByteOrder DEFAULT_BYTE_ORDER = ByteOrder.BIG_ENDIAN;
+
+	/** Size of each index entry in the index file. 8 bytes for offset and 4 bytes for number of buffers. */
+	public static final int INDEX_ENTRY_SIZE = 8 + 4;
+
+	/** Number of data regions in this {@link PartitionedFile}. */
+	private final int numRegions;
+
+	/** Number of subpartitions of this {@link PartitionedFile}. */
+	private final int numSubpartitions;
+
+	/** Path of the data file which stores all data in this {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Path of the index file which stores indexes of all regions in this {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Used to accelerate index data access. */
+	private final ByteBuffer indexDataCache;
+
+	public PartitionedFile(
+			int numRegions,
+			int numSubpartitions,
+			Path dataFilePath,
+			Path indexFilePath,
+			ByteBuffer indexDataCache) {
+		checkArgument(numRegions >= 0, "Illegal number of data regions.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numRegions = numRegions;
+		this.numSubpartitions = numSubpartitions;
+		this.dataFilePath = checkNotNull(dataFilePath);
+		this.indexFilePath = checkNotNull(indexFilePath);
+		this.indexDataCache = indexDataCache;
+	}
+
+	public Path getDataFilePath() {
+		return dataFilePath;
+	}
+
+	public Path getIndexFilePath() {
+		return indexFilePath;
+	}
+
+	public int getNumRegions() {
+		return numRegions;
+	}
+
+	/**
+	 * Returns the index entry offset of the target region and subpartition in the index file. Both region index
+	 * and subpartition index start from 0.
+	 */
+	private long getIndexEntryOffset(int region, int subpartition) {

Review comment:
       I think it does not make much difference here to return long since we always first cast it to integer when using the result, we may change the return type to int and use `MathUtils.checkedCastDown` in return statement.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510053495



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();

Review comment:
       it would be safe to recycle the buffer in `finally` region in case of any exceptions in above processes. If so we might not reuse the same variable `buffer` as the return value of `bufferCompressor.compressToIntermediateBuffer`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510055491



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);

Review comment:
       Make `buffer = EventSerializer.toBuffer(event, isPriorityEvent);` inside `try` region to avoid buffer leak once exception.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716182352


   > @wsry , I am not quite sure whether we ever had the case of empty data for some subpartitions in any form of tests.
   > If not, we should verify such scenarios to guarantee no potential bugs in this new supplement partition. As based on my previous experience, it is easy to trigger some corner case in logics if some subpartitions without any data until job finishes.
   
   I will add one for blocking shuffle.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511596210



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {

Review comment:
       At reading side, we use (subpartition index) Ɨ (index entry size) locate an index entry for the given subpartition in a data region. And you are right, it's not very performance sensitive.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f5e7c69522f61d5d992b85371340f232b59f3cdd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950) 
   * 0ea0d823be70a718ca9ac3c7684e8b46552ad634 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717350591


   Thanks for explaining, @wsry 
   
   Maybe we can find a good combination of "min-parallelism" and "min-buffers" to make the switch to sort-based shuffle work well for users. Because then the very low parallelism cases would use the partition-based shuffle and the higher-parallelism cases the sort-based shuffle.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 690ec79d32dbd8e2a9c62556c0548a56c6a52521 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529",
       "triggerID" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4121561bd0e53433d56f19ebe8f8ab35b9755e34 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717204918






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-715286077


   @wsry , I am not quite sure whether we ever had the case of empty data for some subpartitions in any form of tests. 
   If not, we should verify such scenarios to guarantee no potential bugs in this new supplement partition.  As based on my previous experience, it is easy to trigger some corner case in logics if some subpartitions without any data until job finishes.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717204918


   @wsry I have two more questions on the minimum buffers logic and confi
   
   (1)
   
   I previously thought that this is to ensure that the sort buffers are not too small. But I saw now that they can actually be smaller, because the minimum is `Math.min(minBuffersPerSortMergePartition , numSubpartitions + 1)`. So if we have a low parallelism, then the sort shuffle might run with something like 5 buffers (assuming a user turns it on for all shuffles). Is that on purpose?
   
   Does it make sense to just use the `minBuffersPerSortMergePartition` value and not lower it below that?
   
   BTW: I think I now understand why you called it "max buffers" before - because it is the maximum to which the minimum requirement can grow.
   
   (2)
   
   The current value for `taskmanager.network.sort-shuffle.min-buffers` may be high for mini clusters. At least if we end up doing what is proposed in (1) and not lower it for lower parallelism.
   
   The current min is 512 buffers (16 MB) and the mini cluster has only 64 MB network memory by default. So as soon as there are four shuffles, the mini cluster will break, which is pretty soon.
   We should probably set a different min on the mini cluster, like 64 buffers or so.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r509989145



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster

Review comment:
       It seems not proper to misuse the variable of `isFinished` here. If we want to fail writing faster, we can make use of `isReleased` tag since it was also tagged true from `ResultPartition` before this execution. 
   
   In other words, if the `releaseInternal()` is triggered during view consumption, the data writing should already be finished before. If it is triggered during writing data, then it should be executed by task thread, that means the writing process should already be exited before.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510025704



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {

Review comment:
       The condition of `currentSortBuffer == null` should not trigger the `releaseCurrentSortBuffer`. Maybe we can split them as below:
   
   ```
   if (currentSortBuffer == null) {
        return;
   }
   
   if (currentSortBuffer.hasRemaining()) {
         .........
   }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510005482



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);

Review comment:
       nit: `updateStatistics` could be covered inside `writeCompressedBufferIfPossible` since both `#writeLargeRecord` and `#flushCurrentSortBuffer`  have the same usages, then it can deduplicate the code a bit.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717347640


   > @wsry I have two more questions on the minimum buffers logic and confi
   > 
   > (1)
   > 
   > I previously thought that this is to ensure that the sort buffers are not too small. But I saw now that they can actually be smaller, because the minimum is `Math.min(minBuffersPerSortMergePartition , numSubpartitions + 1)`. So if we have a low parallelism, then the sort shuffle might run with something like 5 buffers (assuming a user turns it on for all shuffles). Is that on purpose?
   > 
   > Does it make sense to just use the `minBuffersPerSortMergePartition` value and not lower it below that?
   > 
   > BTW: I think I now understand why you called it "max buffers" before - because it is the maximum to which the minimum requirement can grow.
   > 
   > (2)
   > 
   > The current value for `taskmanager.network.sort-shuffle.min-buffers` may be high for mini clusters. At least if we end up doing what is proposed in (1) and not lower it for lower parallelism.
   > 
   > The current min is 512 buffers (16 MB) and the mini cluster has only 64 MB network memory by default. So as soon as there are four shuffles, the mini cluster will break, which is pretty soon.
   > We should probably set a different min on the mini cluster, like 64 buffers or so. This does not require a new config key, it you can just set this directly in the mini cluster configuration.
   
   Using `taskmanager.network.sort-shuffle.min-buffers` as the min required number of buffers directly is also a reasonable choice. Previously, I was thinking that users can switch to sort-merge shuffle directly without changing the config value (may need to increase network buffers for small parallelism jobs). However, to achieve a better performance, the user may have to tune these config values, if so, leaving the config values unchanged seems meaningless.
   
   I will update the PR soon as suggested.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707151351


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 96fc01e7f199429da15dbcae89762c9314e6780a (Mon Oct 12 14:21:58 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-19582).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ā“ 1. The [description] looks good.
   * ā“ 2. There is [consensus] that the contribution should go into to Flink.
   * ā“ 3. Needs [attention] from.
   * ā“ 4. The change fits into the overall [architecture].
   * ā“ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511625760



##########
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:
       I think there is no need to do that, because these resources will be cleared after test finishes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510816451



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFile.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.util.IOUtils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link PartitionedFile} is the persistent file type of sort-merge blocking shuffle. Each {@link PartitionedFile}
+ * contains two files: one is data file and the other is index file. Both the data file and index file have multiple
+ * regions. Each data region store the shuffle data in subpartition index order and the corresponding index region
+ * contains index entries of all subpartitions. Each index entry is a (long, integer) tuple of which the long value
+ * is the file offset of the corresponding subpartition and the integer value is the number of buffers.
+ */
+public class PartitionedFile {
+
+	public static final String DATA_FILE_SUFFIX = ".shuffle.data";
+
+	public static final String INDEX_FILE_SUFFIX = ".shuffle.index";
+
+	public static final ByteOrder DEFAULT_BYTE_ORDER = ByteOrder.BIG_ENDIAN;
+
+	/** Size of each index entry in the index file. 8 bytes for offset and 4 bytes for number of buffers. */
+	public static final int INDEX_ENTRY_SIZE = 8 + 4;
+
+	/** Number of data regions in this {@link PartitionedFile}. */
+	private final int numRegions;
+
+	/** Number of subpartitions of this {@link PartitionedFile}. */
+	private final int numSubpartitions;
+
+	/** Path of the data file which stores all data in this {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Path of the index file which stores indexes of all regions in this {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Used to accelerate index data access. */
+	private final ByteBuffer indexDataCache;

Review comment:
       nit: better to add annotation `@Nullable`, also for the respective constructor argument.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510010071



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());

Review comment:
       is this check really necessary? If so, i guess it should be placed onto `super#finish` and done firstly. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r514988635



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());
+
+			resultFile = fileWriter.finish();
+			fileWriter = null;
+
+			LOG.info("New partitioned file produced: {}.", resultFile);
+		}
+
+		super.finish();
+	}
+
+	@Override
+	public void close() {
+		releaseCurrentSortBuffer();

Review comment:
       If the variable `currentSortBuffer` might be touched concurrently by two threads (task main thread and canceler thread), it might still has the potential race condition if one thread reset it as `null` to cause misleading NPE for another thread.
   
   The ideal condition is only allowing main thread access this variable and release it no matter with active or passive exit. Of course we can also resort to lock to avoid it if the logic is not easy to refactored. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510014166



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());
+
+			resultFile = fileWriter.finish();
+			fileWriter = null;
+
+			LOG.info("New partitioned file produced: {}.", resultFile);
+		}
+
+		super.finish();
+	}
+
+	@Override
+	public void close() {
+		releaseCurrentSortBuffer();
+
+		if (fileWriter != null) {
+			fileWriter.releaseQuietly();

Review comment:
       It also has race condition for `fileWriter` as the similar case above. We can resolve this issue by defining it as final variable as commented https://github.com/apache/flink/pull/13595/files#r509983867




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716450042


   The updated logic looks good. But the name `maxBuffersPerSortMergePartition` seems confusing. I think we should rename this to `minBuffersPerSortMergePartition` and also rename the config key to `taskmanager.network.sort-shuffle.min-buffers`.
   
   What is a good default for the minimum number of buffers? Maybe something like 512?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-720663470






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r509983937



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortBuffer.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Data of different channels can be appended to a {@link SortBuffer} and after the {@link SortBuffer} is
+ * finished, the appended data can be copied from it in channel index order.
+ */
+public interface SortBuffer {
+
+	/**
+	 * Appends data of the specified channel to this {@link SortBuffer} and returns true if all bytes of
+	 * the source buffer is copied to this {@link SortBuffer} successfully, otherwise if returns false,
+	 * nothing will be copied.
+	 */
+	boolean append(ByteBuffer source, int targetChannel, Buffer.DataType dataType) throws IOException;
+
+	/**
+	 * Copies data in this {@link SortBuffer} to the target {@link MemorySegment} in channel index order
+	 * and returns {@link BufferWithChannel} which contains the copied data and the corresponding channel
+	 * index.
+	 */
+	BufferWithChannel copyData(MemorySegment target);

Review comment:
       nit: rename as `copyIntoSegment` for better readable, otherwise I might be a bit confused whether `copyDataTo` or `copyDataFrom` in internal processes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r509983669



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,

Review comment:
       The `fileWriter` is not necessary to provide as an argument since it can be got directly from `getPartitionedFileWriter()`, then it can simplify a bit the callers.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);

Review comment:
       I think the below logic is not very easily to understand and also has some overheads in practice. 
   
   ```
   if (!sortBuffer.hasRemaining()) {
   	// the record can not be appended to the free sort buffer because it is too large
   	releaseCurrentSortBuffer();
           writeLargeRecord(record, targetSubpartition, dataType);
   	return;
   }
   
   flushCurrentSortBuffer();
   emit(record, targetSubpartition, dataType);
   ```
   
   Here we have to flush current buffer and call `emit` again to try out whether the unfilled sort buffer can hold the large record or not. During `sortBuffer.append` it would request all the segments from `LocalBufferPool` until exhausted before return `false` to decide flush large record.
   
   I think of two options for improvement here.
   
   - Make `SortMergeResultPartition` more light-weight component to only dispatch record/events, then the `SortBuffer` can decide itself when to append records in cache, when to flush them and when to flush large records directly internally. No need to expose many interfaces to interact with `SortMergeResultPartition`.
   
   - `SortBuffer` provide another method to expose the cache threshold, it can be easily got from the internal `LocalBufferPool`. Then `SortMergeResultPartition` can decide whether to flush the large record directly to avoid unnecessary requesting all the segments. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();

Review comment:
       I guess this call is not really necessary. Because the current sort buffer can still be used in the followup small data after flushing the large record to file directly.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;

Review comment:
       I guess this variable could be defined as final and setup in the constructor after removing `null` setting in below `finish()` method. Then we do not need to consider the condition of `fileWriter == null` is various usages.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r514990038



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();

Review comment:
       Yeah, same thought as https://github.com/apache/flink/pull/13595#discussion_r514989760




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b97362104410740de393cbbfb1b27ca564dadc0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253) 
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511591782



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;
+
+	/** A buffer pool to request memory segments from. */
+	private final BufferPool bufferPool;
+
+	/** A segment list as a joint buffer which stores all records and index entries. */
+	private final ArrayList<MemorySegment> buffers = new ArrayList<>();
+
+	/** Addresses of the first record's index entry for each subpartition. */
+	private final long[] firstIndexEntryAddresses;
+
+	/** Addresses of the last record's index entry for each subpartition. */
+	private final long[] lastIndexEntryAddresses;
+
+	/** Size of buffers requested from buffer pool. All buffers must be of the same size. */
+	private final int bufferSize;
+
+	// ----------------------------------------------------------------------------------------------
+	// Statistics and states
+	// ----------------------------------------------------------------------------------------------
+
+	/** Total number of bytes already appended to this sort buffer. */
+	private long numTotalBytes;
+
+	/** Total number of records already appended to this sort buffer. */
+	private long numTotalRecords;
+
+	/** Total number of bytes already read from this sort buffer. */
+	private long numTotalBytesRead;
+
+	/** Whether this sort buffer is finished. One can only read a finished sort buffer. */
+	private boolean isFinished;
+
+	/** Whether this sort buffer is released. A released sort buffer can not be used. */
+	private boolean isReleased;
+
+	// ----------------------------------------------------------------------------------------------
+	// For writing
+	// ----------------------------------------------------------------------------------------------
+
+	/** Array index in the segment list of the current available buffer for writing. */
+	private int writeSegmentIndex;
+
+	/** Next position in the current available buffer for writing. */
+	private int writeSegmentOffset;
+
+	// ----------------------------------------------------------------------------------------------
+	// For reading
+	// ----------------------------------------------------------------------------------------------
+
+	/** Index entry address of the current record or event to be read. */
+	private long readIndexEntryAddress;
+
+	/** Record bytes remaining after last copy, which must be read first in next copy. */
+	private int recordRemainingBytes;
+
+	/** Current available channel to read data from. */
+	private int readChannelIndex = -1;
+
+	public PartitionSortedBuffer(BufferPool bufferPool, int numSubpartitions, int bufferSize) {
+		checkArgument(bufferSize > INDEX_ENTRY_SIZE, "Buffer size is too small.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+		this.bufferSize = bufferSize;
+		this.firstIndexEntryAddresses = new long[numSubpartitions];
+		this.lastIndexEntryAddresses = new long[numSubpartitions];
+
+		// initialized with -1 means the corresponding channel has no data
+		Arrays.fill(firstIndexEntryAddresses, -1L);
+		Arrays.fill(lastIndexEntryAddresses, -1L);
+	}
+
+	@Override
+	public boolean append(ByteBuffer source, int targetChannel, DataType dataType) throws IOException {
+		checkState(!isFinished, "Sort buffer is already finished.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int totalBytes = source.remaining();
+		if (totalBytes == 0) {

Review comment:
       Empty record is impossible here, I will add a check here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r515974702



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter implements AutoCloseable {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data is cached in memory or not. */
+	private boolean allIndexDataCached = true;
+
+	/** Whether this file writer is finished or not. */
+	private boolean isFinished;
+
+	/** Whether this file writer is closed or not. */
+	private boolean isClosed;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions, int indexBufferSize) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+		checkArgument(indexBufferSize > 0, "Illegal index buffer size.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(indexBufferSize * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					allIndexDataCached = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		indexBuffer.flip();
+		if (indexBuffer.limit() > 0) {
+			BufferReaderWriterUtil.writeBuffer(indexFileChannel, indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		isFinished = true;
+
+		writeRegionIndex();
+		flushIndexBuffer();
+		indexBuffer.rewind();
+
+		close();
+		ByteBuffer indexDataCache = allIndexDataCached ? indexBuffer : null;
+		return new PartitionedFile(numRegions, numSubpartitions, dataFilePath, indexFilePath, indexDataCache);
+	}
+
+	/**
+	 * Used to close and delete the failed {@link PartitionedFile} when any exception occurs.
+	 */
+	public void releaseQuietly() {
+		IOUtils.closeQuietly(this);
+		IOUtils.deleteFileQuietly(dataFilePath);
+		IOUtils.deleteFileQuietly(indexFilePath);
+	}
+
+	@Override
+	public void close() {
+		if (isClosed) {
+			return;
+		}
+		isClosed = true;
+
+		Throwable exception = null;
+		try {
+			dataFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = throwable;
+		}
+
+		try {
+			indexFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = ExceptionUtils.firstOrSuppressed(throwable, exception);
+		}
+
+		if (exception != null) {
+			ExceptionUtils.rethrow(exception);

Review comment:
       In this case it does a redundant wrapping into RuntimeException. If you add the exception signature you avoid that.
   Exception wrapping should be avoided unless you add actual information in the new exception. Otherwise it just complicates the stack traces for debugging.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96fc01e7f199429da15dbcae89762c9314e6780a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459) 
   * 72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510750829



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+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 {
+
+	private static final int NUM_READ_BUFFERS = 2;
+
+	/** Target {@link SortMergeResultPartition} to read data from. */
+	private final SortMergeResultPartition partition;
+
+	/** Listener to notify when data is available. */
+	private final BufferAvailabilityListener availabilityListener;
+
+	/** Result {@link PartitionedFile} to read. */
+	private final PartitionedFile partitionedFile;
+
+	/** Unmanaged memory used as read buffers. */
+	private final Queue<MemorySegment> readBuffers = new ArrayDeque<>();
+
+	/** Buffers read by the file reader. */
+	private final Queue<Buffer> buffersRead = new ArrayDeque<>();
+
+	/** Target subpartition to read. */
+	private final int subpartitionIndex;

Review comment:
       This variable only used in constructor




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96fc01e7f199429da15dbcae89762c9314e6780a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b97362104410740de393cbbfb1b27ca564dadc0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510200867



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;
+
+	/** A buffer pool to request memory segments from. */
+	private final BufferPool bufferPool;
+
+	/** A segment list as a joint buffer which stores all records and index entries. */
+	private final ArrayList<MemorySegment> buffers = new ArrayList<>();
+
+	/** Addresses of the first record's index entry for each subpartition. */
+	private final long[] firstIndexEntryAddresses;
+
+	/** Addresses of the last record's index entry for each subpartition. */
+	private final long[] lastIndexEntryAddresses;
+
+	/** Size of buffers requested from buffer pool. All buffers must be of the same size. */
+	private final int bufferSize;
+
+	// ----------------------------------------------------------------------------------------------
+	// Statistics and states
+	// ----------------------------------------------------------------------------------------------
+
+	/** Total number of bytes already appended to this sort buffer. */
+	private long numTotalBytes;
+
+	/** Total number of records already appended to this sort buffer. */
+	private long numTotalRecords;
+
+	/** Total number of bytes already read from this sort buffer. */
+	private long numTotalBytesRead;
+
+	/** Whether this sort buffer is finished. One can only read a finished sort buffer. */
+	private boolean isFinished;
+
+	/** Whether this sort buffer is released. A released sort buffer can not be used. */
+	private boolean isReleased;
+
+	// ----------------------------------------------------------------------------------------------
+	// For writing
+	// ----------------------------------------------------------------------------------------------
+
+	/** Array index in the segment list of the current available buffer for writing. */
+	private int writeSegmentIndex;
+
+	/** Next position in the current available buffer for writing. */
+	private int writeSegmentOffset;
+
+	// ----------------------------------------------------------------------------------------------
+	// For reading
+	// ----------------------------------------------------------------------------------------------
+
+	/** Index entry address of the current record or event to be read. */
+	private long readIndexEntryAddress;
+
+	/** Record bytes remaining after last copy, which must be read first in next copy. */
+	private int recordRemainingBytes;
+
+	/** Current available channel to read data from. */
+	private int readChannelIndex = -1;
+
+	public PartitionSortedBuffer(BufferPool bufferPool, int numSubpartitions, int bufferSize) {
+		checkArgument(bufferSize > INDEX_ENTRY_SIZE, "Buffer size is too small.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+		this.bufferSize = bufferSize;
+		this.firstIndexEntryAddresses = new long[numSubpartitions];
+		this.lastIndexEntryAddresses = new long[numSubpartitions];
+
+		// initialized with -1 means the corresponding channel has no data
+		Arrays.fill(firstIndexEntryAddresses, -1L);
+		Arrays.fill(lastIndexEntryAddresses, -1L);
+	}
+
+	@Override
+	public boolean append(ByteBuffer source, int targetChannel, DataType dataType) throws IOException {
+		checkState(!isFinished, "Sort buffer is already finished.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int totalBytes = source.remaining();
+		if (totalBytes == 0) {
+			return true;
+		}
+
+		// return false directly if it can not allocate enough buffers for the given record
+		if (!allocateBuffersForRecord(totalBytes)) {
+			return false;
+		}
+
+		// write the index entry and record or event data
+		writeIndex(targetChannel, totalBytes, dataType);
+		writeRecord(source);
+
+		++numTotalRecords;
+		numTotalBytes += totalBytes;
+
+		return true;
+	}
+
+	private void writeIndex(int channelIndex, int numRecordBytes, Buffer.DataType dataType) {
+		MemorySegment segment = buffers.get(writeSegmentIndex);
+
+		// record length takes the high 32 bits and data type takes the low 32 bits
+		segment.putLong(writeSegmentOffset, ((long) numRecordBytes << 32) | dataType.ordinal());
+
+		// segment index takes the high 32 bits and segment offset takes the low 32 bits
+		long indexEntryAddress = ((long) writeSegmentIndex << 32) | writeSegmentOffset;
+
+		long lastIndexEntryAddress =  lastIndexEntryAddresses[channelIndex];
+		lastIndexEntryAddresses[channelIndex] = indexEntryAddress;
+
+		if (lastIndexEntryAddress >= 0) {
+			// link the previous index entry of the given channel to the new index entry
+			segment = buffers.get(getHigh32BitsFromLongAsInteger(lastIndexEntryAddress));
+			segment.putLong(getLow32BitsFromLongAsInteger(lastIndexEntryAddress) + 8, indexEntryAddress);
+		} else {
+			firstIndexEntryAddresses[channelIndex] = indexEntryAddress;
+		}
+
+		// move the write position forward so as to write the corresponding record
+		updateWriteSegmentIndexAndOffset(INDEX_ENTRY_SIZE);
+	}
+
+	private void writeRecord(ByteBuffer source) {
+		while (source.hasRemaining()) {
+			MemorySegment segment = buffers.get(writeSegmentIndex);
+			int toCopy = Math.min(bufferSize - writeSegmentOffset, source.remaining());
+			segment.put(writeSegmentOffset, source, toCopy);
+
+			// move the write position forward so as to write the remaining bytes or next record
+			updateWriteSegmentIndexAndOffset(toCopy);
+		}
+	}
+
+	private boolean allocateBuffersForRecord(int numRecordBytes) throws IOException {
+		int numBytesRequired = INDEX_ENTRY_SIZE + numRecordBytes;
+		int availableBytes = writeSegmentIndex == buffers.size() ? 0 : bufferSize - writeSegmentOffset;
+
+		// return directly if current available bytes is adequate
+		if (availableBytes >= numBytesRequired) {
+			return true;
+		}
+
+		// skip the remaining free space if the available bytes is not enough for an index entry
+		if (availableBytes < INDEX_ENTRY_SIZE) {
+			updateWriteSegmentIndexAndOffset(availableBytes);
+			availableBytes = 0;
+		}
+
+		// allocate exactly enough buffers for the appended record
+		do {
+			MemorySegment segment = requestBufferFromPool();
+			if (segment == null) {
+				// return false if we can not allocate enough buffers for the appended record
+				return false;
+			}
+
+			assert segment.size() == bufferSize;
+			availableBytes += bufferSize;
+			buffers.add(segment);
+		} while (availableBytes < numBytesRequired);
+
+		return true;
+	}
+
+	private MemorySegment requestBufferFromPool() throws IOException {
+		try {
+			// blocking request buffers if there is still guaranteed memory
+			if (buffers.size() < bufferPool.getNumberOfRequiredMemorySegments()) {
+				return bufferPool.requestBufferBuilderBlocking().getMemorySegment();
+			}
+		} catch (InterruptedException e) {
+			throw new IOException("Interrupted while requesting buffer.");
+		}
+
+		BufferBuilder buffer = bufferPool.requestBufferBuilder();
+		return buffer != null ? buffer.getMemorySegment() : null;
+	}
+
+	private void updateWriteSegmentIndexAndOffset(int numBytes) {
+		writeSegmentOffset += numBytes;
+
+		// using the next available free buffer if the current is full
+		if (writeSegmentOffset == bufferSize) {
+			++writeSegmentIndex;
+			writeSegmentOffset = 0;
+		}
+	}
+
+	@Override
+	public BufferWithChannel copyData(MemorySegment target) {
+		checkState(hasRemaining(), "No data remaining.");
+		checkState(isFinished, "Should finish the sort buffer first before coping any data.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int numBytesCopied = 0;
+		DataType bufferDataType = DataType.DATA_BUFFER;
+		int channelIndex = readChannelIndex;
+
+		do {
+			int sourceSegmentIndex = getHigh32BitsFromLongAsInteger(readIndexEntryAddress);
+			int sourceSegmentOffset = getLow32BitsFromLongAsInteger(readIndexEntryAddress);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+
+			long lengthAndDataType = sourceSegment.getLong(sourceSegmentOffset);
+			int length = getHigh32BitsFromLongAsInteger(lengthAndDataType);
+			DataType dataType = DataType.values()[getLow32BitsFromLongAsInteger(lengthAndDataType)];
+
+			// return the data read directly if the next to read is an event
+			if (dataType.isEvent() && numBytesCopied > 0) {
+				break;
+			}
+			bufferDataType = dataType;
+
+			// get the next index entry address and move the read position forward
+			long nextReadIndexEntryAddress = sourceSegment.getLong(sourceSegmentOffset + 8);
+			sourceSegmentOffset += INDEX_ENTRY_SIZE;
+
+			// allocate a temp buffer for the event if the target buffer is not big enough
+			if (bufferDataType.isEvent() && target.size() < length) {
+				target = MemorySegmentFactory.allocateUnpooledSegment(length);
+			}
+
+			numBytesCopied += copyRecordOrEvent(
+				target, numBytesCopied, sourceSegmentIndex, sourceSegmentOffset, length);
+
+			if (recordRemainingBytes == 0) {
+				// move to next channel if the current channel has been finished
+				if (readIndexEntryAddress == lastIndexEntryAddresses[channelIndex]) {
+					updateReadChannelAndIndexEntryAddress();
+					break;
+				}
+				readIndexEntryAddress = nextReadIndexEntryAddress;
+			}
+		} while (numBytesCopied < target.size() && bufferDataType.isBuffer());
+
+		numTotalBytesRead += numBytesCopied;
+		Buffer buffer = new NetworkBuffer(target, (buf) -> {}, bufferDataType, numBytesCopied);
+		return new BufferWithChannel(buffer, channelIndex);
+	}
+
+	private int copyRecordOrEvent(
+			MemorySegment targetSegment,
+			int targetSegmentOffset,
+			int sourceSegmentIndex,
+			int sourceSegmentOffset,
+			int recordLength) {
+		if (recordRemainingBytes > 0) {
+			// skip the data already read if there is remaining partial record after the previous copy
+			long position = (long) sourceSegmentOffset + (recordLength - recordRemainingBytes);
+			sourceSegmentIndex += (position / bufferSize);
+			sourceSegmentOffset = (int) (position % bufferSize);
+		} else {
+			recordRemainingBytes = recordLength;
+		}
+
+		int targetSegmentSize = targetSegment.size();
+		int numBytesToCopy = Math.min(targetSegmentSize - targetSegmentOffset, recordRemainingBytes);
+		do {
+			// move to next data buffer if all data of the current buffer has been copied
+			if (sourceSegmentOffset == bufferSize) {
+				++sourceSegmentIndex;
+				sourceSegmentOffset = 0;
+			}
+
+			int sourceRemainingBytes = Math.min(bufferSize - sourceSegmentOffset, recordRemainingBytes);
+			int numBytes = Math.min(targetSegmentSize - targetSegmentOffset, sourceRemainingBytes);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+			sourceSegment.copyTo(sourceSegmentOffset, targetSegment, targetSegmentOffset, numBytes);
+
+			recordRemainingBytes -= numBytes;
+			targetSegmentOffset += numBytes;
+			sourceSegmentOffset += numBytes;
+		} while ((recordRemainingBytes > 0 && targetSegmentOffset < targetSegmentSize));
+
+		return numBytesToCopy;
+	}
+
+	private void updateReadChannelAndIndexEntryAddress() {
+		// skip the channels without any data
+		while (++readChannelIndex < firstIndexEntryAddresses.length) {
+			if ((readIndexEntryAddress = firstIndexEntryAddresses[readChannelIndex]) >= 0) {
+				break;
+			}
+		}
+	}
+
+	private int getHigh32BitsFromLongAsInteger(long value) {
+		return (int) (value >>> 32);
+	}
+
+	private int getLow32BitsFromLongAsInteger(long value) {
+		return (int) (value & 0xffff);

Review comment:
       This only gets the low 16 bits, not the low 32 bits.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510805311



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					canCacheAllIndexData = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		if (indexBuffer.position() > 0) {
+			indexBuffer.flip();
+			indexFileChannel.write(indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);

Review comment:
       I am a bit confused why we need to maintain this variable for above `checkArgument(targetSubpartition >= currentSubpartition...`. I do not see the necessary from it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r514994161



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {

Review comment:
       Ok, let's further drive into it if really necessary future. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300) 
   * 761eb39aefc6863bc0cc7c35c78d5a1a9d016997 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522) 
   * 4121561bd0e53433d56f19ebe8f8ab35b9755e34 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511597510



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					canCacheAllIndexData = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		if (indexBuffer.position() > 0) {
+			indexBuffer.flip();
+			indexFileChannel.write(indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");

Review comment:
       We always have an EndOfPartition event




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510025704



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {

Review comment:
       The condition of `currentSortBuffer == null` should not trigger the `releaseCurrentSortBuffer`. Maybe we can split them as below:
   
   ```
   if (currentSortBuffer == null) {
        return;
   }
   
   if (currentSortBuffer.hasRemaining()) {
         // actual flush 
   }
   
   releaseCurrentSortBuffer();
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511601728



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);

Review comment:
       For the first option, the change is a little big and I would like to treat SortBuffer a purely in-memory data structure.
   For the second option, the current size of buffer pool can be changed if new buffer pool created or old buffer pool is destroyed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511628189



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java
##########
@@ -136,26 +144,41 @@ public ResultPartition create(
 
 			partition = pipelinedPartition;
 		}
-		else if (type == ResultPartitionType.BLOCKING || type == ResultPartitionType.BLOCKING_PERSISTENT) {
-			final BoundedBlockingResultPartition blockingPartition = new BoundedBlockingResultPartition(
-				taskNameWithSubtaskAndId,
-				partitionIndex,
-				id,
-				type,
-				subpartitions,
-				maxParallelism,
-				partitionManager,
-				bufferCompressor,
-				bufferPoolFactory);
-
-			initializeBoundedBlockingPartitions(
-				subpartitions,
-				blockingPartition,
-				blockingSubpartitionType,
-				networkBufferSize,
-				channelManager);
-
-			partition = blockingPartition;
+		else if (isBlockingShuffle(type)) {
+			if (isSortMergeBlockingShuffle(type, numberOfSubpartitions)) {
+				partition = new SortMergeResultPartition(
+					taskNameWithSubtaskAndId,
+					partitionIndex,
+					id,
+					type,
+					subpartitions.length,
+					maxParallelism,
+					networkBufferSize,
+					partitionManager,
+					channelManager,
+					bufferCompressor,
+					bufferPoolFactory);
+			} else {
+				final BoundedBlockingResultPartition blockingPartition = new BoundedBlockingResultPartition(

Review comment:
       We need the type of BoundedBlockingResultPartition when calling initializeBoundedBlockingPartitions.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-720980140






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r514989760



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);

Review comment:
       Thanks for the explanation. I might overlook the case of dynamic buffer pool for the second option.
   I am not quite sure whether it is worth improving the overhead or simplifying the logic ATM. Maybe we can later decide it if necessary future.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 690ec79d32dbd8e2a9c62556c0548a56c6a52521 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248) 
   * 9b97362104410740de393cbbfb1b27ca564dadc0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511606587



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());
+
+			resultFile = fileWriter.finish();
+			fileWriter = null;
+
+			LOG.info("New partitioned file produced: {}.", resultFile);
+		}
+
+		super.finish();
+	}
+
+	@Override
+	public void close() {
+		releaseCurrentSortBuffer();

Review comment:
       You are right, I ignored this.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717622807


   @StephanEwen 
   It is a really good point. If the "min-parallelism" is larger than the "min-buffers", the network buffers will be always satisfied.
   Then what do you think if we give a smaller default value for "min-buffers", for example 64. If so, we do not need to change the default mini-cluster configuration, besides, if there are more available buffers in the global network buffer pool, we can also use them, because the max number of buffer can be used by the local buffer pool is Integer.MAX.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529",
       "triggerID" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "914604bac395df4e65538eeadfa1d00091dbec5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8724",
       "triggerID" : "914604bac395df4e65538eeadfa1d00091dbec5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ced5f9f6f482ea96e69baf9af095e81640b4a28d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8749",
       "triggerID" : "ced5f9f6f482ea96e69baf9af095e81640b4a28d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d2e33683fd05f4eb64bbdb42b7fb55bcf5d5a989",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d2e33683fd05f4eb64bbdb42b7fb55bcf5d5a989",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aeceebe4b9f9fff8a515e0e95ef47b678b3d752",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8819",
       "triggerID" : "3aeceebe4b9f9fff8a515e0e95ef47b678b3d752",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b2de494d153e4253c297bac8af73ebc2da51f3d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8866",
       "triggerID" : "9b2de494d153e4253c297bac8af73ebc2da51f3d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d2e33683fd05f4eb64bbdb42b7fb55bcf5d5a989 UNKNOWN
   * 9b2de494d153e4253c297bac8af73ebc2da51f3d Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8866) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 690ec79d32dbd8e2a9c62556c0548a56c6a52521 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248) 
   * 9b97362104410740de393cbbfb1b27ca564dadc0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716519500


   > The updated logic looks good. But the name `maxBuffersPerSortMergePartition` seems confusing. I think we should rename this to `minBuffersPerSortMergePartition` and also rename the config key to `taskmanager.network.sort-shuffle.min-buffers`.
   > 
   > What is a good default for the minimum number of buffers? Maybe something like 512?
   
   @StephanEwen These are good suggestions. `taskmanager.network.sort-shuffle.min-buffers` is easier to understand from the perspective of user and set the default value to 512 is also reasonable. I have updated the PR accordingly. 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510256114



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;
+
+	/** A buffer pool to request memory segments from. */
+	private final BufferPool bufferPool;
+
+	/** A segment list as a joint buffer which stores all records and index entries. */
+	private final ArrayList<MemorySegment> buffers = new ArrayList<>();
+
+	/** Addresses of the first record's index entry for each subpartition. */
+	private final long[] firstIndexEntryAddresses;
+
+	/** Addresses of the last record's index entry for each subpartition. */
+	private final long[] lastIndexEntryAddresses;
+
+	/** Size of buffers requested from buffer pool. All buffers must be of the same size. */
+	private final int bufferSize;
+
+	// ----------------------------------------------------------------------------------------------
+	// Statistics and states
+	// ----------------------------------------------------------------------------------------------
+
+	/** Total number of bytes already appended to this sort buffer. */
+	private long numTotalBytes;
+
+	/** Total number of records already appended to this sort buffer. */
+	private long numTotalRecords;
+
+	/** Total number of bytes already read from this sort buffer. */
+	private long numTotalBytesRead;
+
+	/** Whether this sort buffer is finished. One can only read a finished sort buffer. */
+	private boolean isFinished;
+
+	/** Whether this sort buffer is released. A released sort buffer can not be used. */
+	private boolean isReleased;
+
+	// ----------------------------------------------------------------------------------------------
+	// For writing
+	// ----------------------------------------------------------------------------------------------
+
+	/** Array index in the segment list of the current available buffer for writing. */
+	private int writeSegmentIndex;
+
+	/** Next position in the current available buffer for writing. */
+	private int writeSegmentOffset;
+
+	// ----------------------------------------------------------------------------------------------
+	// For reading
+	// ----------------------------------------------------------------------------------------------
+
+	/** Index entry address of the current record or event to be read. */
+	private long readIndexEntryAddress;
+
+	/** Record bytes remaining after last copy, which must be read first in next copy. */
+	private int recordRemainingBytes;
+
+	/** Current available channel to read data from. */
+	private int readChannelIndex = -1;
+
+	public PartitionSortedBuffer(BufferPool bufferPool, int numSubpartitions, int bufferSize) {
+		checkArgument(bufferSize > INDEX_ENTRY_SIZE, "Buffer size is too small.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+		this.bufferSize = bufferSize;
+		this.firstIndexEntryAddresses = new long[numSubpartitions];
+		this.lastIndexEntryAddresses = new long[numSubpartitions];
+
+		// initialized with -1 means the corresponding channel has no data
+		Arrays.fill(firstIndexEntryAddresses, -1L);
+		Arrays.fill(lastIndexEntryAddresses, -1L);
+	}
+
+	@Override
+	public boolean append(ByteBuffer source, int targetChannel, DataType dataType) throws IOException {
+		checkState(!isFinished, "Sort buffer is already finished.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int totalBytes = source.remaining();
+		if (totalBytes == 0) {

Review comment:
       Is empty record possible ? I think if it is possible we should also write index for it, otherwise we should throw exceptions for illegal state, but it seems not reasonable to ignore it silently.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-718023765


   @StephanEwen I have updated the PR accordingly.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-717865867


   @wsry Your suggestion sounds good to me!


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r509983577



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());
+
+			resultFile = fileWriter.finish();
+			fileWriter = null;
+
+			LOG.info("New partitioned file produced: {}.", resultFile);
+		}
+
+		super.finish();
+	}
+
+	@Override
+	public void close() {
+		releaseCurrentSortBuffer();

Review comment:
       `#close` could be called by canceller thread if I remembered correctly, then it might cause race condition here.
   Maybe we can only release `fileWriter` as below to interrupt the task thread while writing data. For the other resource cleanup we can put them into `#releaseInternal()` before task exits.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510581883



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;

Review comment:
       As a whole, I think the index entry size could be reduced into 12 bytes by 4 bytes record length + 4 bytes for segment index + 3 bytes for segment offset + 1 bytes for data type. This would help increase the number of records in each region, especially when the record size is smaller. But we might also leave it as a future optimization (if we think it is ok)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500) 
   * 62395a70a02dfd79375f51134f1c195824eeb12e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510808989



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {

Review comment:
       I guess the current way assumes the data redistributed in almost all the subpartitions for every flush call. If in the data skew scenario like only 10 of 1000 subpartitions have actual data for some flush, then the index buffer will be written many empty holes to waste space and execution time. 
   
   I am curious that is it by design to have to write all the subpartitions in the same sequence since I have not touched the detail logics on view reader side? Or it is not very performance sensitive for easy implementation ATM?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510007151



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void writeCompressedBufferIfPossible(
+			Buffer buffer,
+			PartitionedFileWriter fileWriter,
+			int targetSubpartition) throws IOException {
+		if (canBeCompressed(buffer)) {
+			buffer = bufferCompressor.compressToIntermediateBuffer(buffer);
+		}
+		fileWriter.writeBuffer(buffer, targetSubpartition);
+		buffer.recycleBuffer();
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+
+			writeCompressedBufferIfPossible(buffer, fileWriter, targetSubpartition);
+			updateStatistics(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();

Review comment:
       nit: it is redundant here since already done in `super#finish`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] asfgit closed pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #13595:
URL: https://github.com/apache/flink/pull/13595


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511600050



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);

Review comment:
       If it throws an exception, the buffer must be null. we have nothing to release.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300) 
   * 761eb39aefc6863bc0cc7c35c78d5a1a9d016997 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b97362104410740de393cbbfb1b27ca564dadc0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253) 
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510248527



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtil.java
##########
@@ -201,7 +201,7 @@ private static boolean tryReadByteBuffer(FileChannel channel, ByteBuffer b) thro
 		}
 	}
 
-	private static void readByteBufferFully(FileChannel channel, ByteBuffer b) throws IOException {
+	public static void readByteBufferFully(FileChannel channel, ByteBuffer b) throws IOException {

Review comment:
       Make this package-private instead, to be consistent with the visibility of the class.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;
+
+	/** A buffer pool to request memory segments from. */
+	private final BufferPool bufferPool;
+
+	/** A segment list as a joint buffer which stores all records and index entries. */
+	private final ArrayList<MemorySegment> buffers = new ArrayList<>();
+
+	/** Addresses of the first record's index entry for each subpartition. */
+	private final long[] firstIndexEntryAddresses;
+
+	/** Addresses of the last record's index entry for each subpartition. */
+	private final long[] lastIndexEntryAddresses;
+
+	/** Size of buffers requested from buffer pool. All buffers must be of the same size. */
+	private final int bufferSize;
+
+	// ----------------------------------------------------------------------------------------------
+	// Statistics and states
+	// ----------------------------------------------------------------------------------------------
+
+	/** Total number of bytes already appended to this sort buffer. */
+	private long numTotalBytes;
+
+	/** Total number of records already appended to this sort buffer. */
+	private long numTotalRecords;
+
+	/** Total number of bytes already read from this sort buffer. */
+	private long numTotalBytesRead;
+
+	/** Whether this sort buffer is finished. One can only read a finished sort buffer. */
+	private boolean isFinished;
+
+	/** Whether this sort buffer is released. A released sort buffer can not be used. */
+	private boolean isReleased;
+
+	// ----------------------------------------------------------------------------------------------
+	// For writing
+	// ----------------------------------------------------------------------------------------------
+
+	/** Array index in the segment list of the current available buffer for writing. */
+	private int writeSegmentIndex;
+
+	/** Next position in the current available buffer for writing. */
+	private int writeSegmentOffset;
+
+	// ----------------------------------------------------------------------------------------------
+	// For reading
+	// ----------------------------------------------------------------------------------------------
+
+	/** Index entry address of the current record or event to be read. */
+	private long readIndexEntryAddress;
+
+	/** Record bytes remaining after last copy, which must be read first in next copy. */
+	private int recordRemainingBytes;
+
+	/** Current available channel to read data from. */
+	private int readChannelIndex = -1;
+
+	public PartitionSortedBuffer(BufferPool bufferPool, int numSubpartitions, int bufferSize) {
+		checkArgument(bufferSize > INDEX_ENTRY_SIZE, "Buffer size is too small.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+		this.bufferSize = bufferSize;
+		this.firstIndexEntryAddresses = new long[numSubpartitions];
+		this.lastIndexEntryAddresses = new long[numSubpartitions];
+
+		// initialized with -1 means the corresponding channel has no data
+		Arrays.fill(firstIndexEntryAddresses, -1L);
+		Arrays.fill(lastIndexEntryAddresses, -1L);
+	}
+
+	@Override
+	public boolean append(ByteBuffer source, int targetChannel, DataType dataType) throws IOException {
+		checkState(!isFinished, "Sort buffer is already finished.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int totalBytes = source.remaining();
+		if (totalBytes == 0) {
+			return true;
+		}
+
+		// return false directly if it can not allocate enough buffers for the given record
+		if (!allocateBuffersForRecord(totalBytes)) {
+			return false;
+		}
+
+		// write the index entry and record or event data
+		writeIndex(targetChannel, totalBytes, dataType);
+		writeRecord(source);
+
+		++numTotalRecords;
+		numTotalBytes += totalBytes;
+
+		return true;
+	}
+
+	private void writeIndex(int channelIndex, int numRecordBytes, Buffer.DataType dataType) {
+		MemorySegment segment = buffers.get(writeSegmentIndex);
+
+		// record length takes the high 32 bits and data type takes the low 32 bits
+		segment.putLong(writeSegmentOffset, ((long) numRecordBytes << 32) | dataType.ordinal());
+
+		// segment index takes the high 32 bits and segment offset takes the low 32 bits
+		long indexEntryAddress = ((long) writeSegmentIndex << 32) | writeSegmentOffset;
+
+		long lastIndexEntryAddress =  lastIndexEntryAddresses[channelIndex];
+		lastIndexEntryAddresses[channelIndex] = indexEntryAddress;
+
+		if (lastIndexEntryAddress >= 0) {
+			// link the previous index entry of the given channel to the new index entry
+			segment = buffers.get(getHigh32BitsFromLongAsInteger(lastIndexEntryAddress));
+			segment.putLong(getLow32BitsFromLongAsInteger(lastIndexEntryAddress) + 8, indexEntryAddress);
+		} else {
+			firstIndexEntryAddresses[channelIndex] = indexEntryAddress;
+		}
+
+		// move the write position forward so as to write the corresponding record
+		updateWriteSegmentIndexAndOffset(INDEX_ENTRY_SIZE);
+	}
+
+	private void writeRecord(ByteBuffer source) {
+		while (source.hasRemaining()) {
+			MemorySegment segment = buffers.get(writeSegmentIndex);
+			int toCopy = Math.min(bufferSize - writeSegmentOffset, source.remaining());
+			segment.put(writeSegmentOffset, source, toCopy);
+
+			// move the write position forward so as to write the remaining bytes or next record
+			updateWriteSegmentIndexAndOffset(toCopy);
+		}
+	}
+
+	private boolean allocateBuffersForRecord(int numRecordBytes) throws IOException {
+		int numBytesRequired = INDEX_ENTRY_SIZE + numRecordBytes;
+		int availableBytes = writeSegmentIndex == buffers.size() ? 0 : bufferSize - writeSegmentOffset;
+
+		// return directly if current available bytes is adequate
+		if (availableBytes >= numBytesRequired) {
+			return true;
+		}
+
+		// skip the remaining free space if the available bytes is not enough for an index entry
+		if (availableBytes < INDEX_ENTRY_SIZE) {
+			updateWriteSegmentIndexAndOffset(availableBytes);
+			availableBytes = 0;
+		}
+
+		// allocate exactly enough buffers for the appended record
+		do {
+			MemorySegment segment = requestBufferFromPool();
+			if (segment == null) {
+				// return false if we can not allocate enough buffers for the appended record
+				return false;
+			}
+
+			assert segment.size() == bufferSize;
+			availableBytes += bufferSize;
+			buffers.add(segment);
+		} while (availableBytes < numBytesRequired);
+
+		return true;
+	}
+
+	private MemorySegment requestBufferFromPool() throws IOException {
+		try {
+			// blocking request buffers if there is still guaranteed memory
+			if (buffers.size() < bufferPool.getNumberOfRequiredMemorySegments()) {
+				return bufferPool.requestBufferBuilderBlocking().getMemorySegment();
+			}
+		} catch (InterruptedException e) {
+			throw new IOException("Interrupted while requesting buffer.");
+		}
+
+		BufferBuilder buffer = bufferPool.requestBufferBuilder();
+		return buffer != null ? buffer.getMemorySegment() : null;
+	}
+
+	private void updateWriteSegmentIndexAndOffset(int numBytes) {
+		writeSegmentOffset += numBytes;
+
+		// using the next available free buffer if the current is full
+		if (writeSegmentOffset == bufferSize) {
+			++writeSegmentIndex;
+			writeSegmentOffset = 0;
+		}
+	}
+
+	@Override
+	public BufferWithChannel copyData(MemorySegment target) {
+		checkState(hasRemaining(), "No data remaining.");
+		checkState(isFinished, "Should finish the sort buffer first before coping any data.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int numBytesCopied = 0;
+		DataType bufferDataType = DataType.DATA_BUFFER;
+		int channelIndex = readChannelIndex;
+
+		do {
+			int sourceSegmentIndex = getHigh32BitsFromLongAsInteger(readIndexEntryAddress);
+			int sourceSegmentOffset = getLow32BitsFromLongAsInteger(readIndexEntryAddress);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+
+			long lengthAndDataType = sourceSegment.getLong(sourceSegmentOffset);
+			int length = getHigh32BitsFromLongAsInteger(lengthAndDataType);
+			DataType dataType = DataType.values()[getLow32BitsFromLongAsInteger(lengthAndDataType)];
+
+			// return the data read directly if the next to read is an event
+			if (dataType.isEvent() && numBytesCopied > 0) {
+				break;
+			}
+			bufferDataType = dataType;
+
+			// get the next index entry address and move the read position forward
+			long nextReadIndexEntryAddress = sourceSegment.getLong(sourceSegmentOffset + 8);
+			sourceSegmentOffset += INDEX_ENTRY_SIZE;
+
+			// allocate a temp buffer for the event if the target buffer is not big enough
+			if (bufferDataType.isEvent() && target.size() < length) {
+				target = MemorySegmentFactory.allocateUnpooledSegment(length);
+			}
+
+			numBytesCopied += copyRecordOrEvent(
+				target, numBytesCopied, sourceSegmentIndex, sourceSegmentOffset, length);
+
+			if (recordRemainingBytes == 0) {
+				// move to next channel if the current channel has been finished
+				if (readIndexEntryAddress == lastIndexEntryAddresses[channelIndex]) {
+					updateReadChannelAndIndexEntryAddress();
+					break;
+				}
+				readIndexEntryAddress = nextReadIndexEntryAddress;
+			}
+		} while (numBytesCopied < target.size() && bufferDataType.isBuffer());
+
+		numTotalBytesRead += numBytesCopied;
+		Buffer buffer = new NetworkBuffer(target, (buf) -> {}, bufferDataType, numBytesCopied);
+		return new BufferWithChannel(buffer, channelIndex);
+	}
+
+	private int copyRecordOrEvent(
+			MemorySegment targetSegment,
+			int targetSegmentOffset,
+			int sourceSegmentIndex,
+			int sourceSegmentOffset,
+			int recordLength) {
+		if (recordRemainingBytes > 0) {
+			// skip the data already read if there is remaining partial record after the previous copy
+			long position = (long) sourceSegmentOffset + (recordLength - recordRemainingBytes);
+			sourceSegmentIndex += (position / bufferSize);
+			sourceSegmentOffset = (int) (position % bufferSize);
+		} else {
+			recordRemainingBytes = recordLength;
+		}
+
+		int targetSegmentSize = targetSegment.size();
+		int numBytesToCopy = Math.min(targetSegmentSize - targetSegmentOffset, recordRemainingBytes);
+		do {
+			// move to next data buffer if all data of the current buffer has been copied
+			if (sourceSegmentOffset == bufferSize) {
+				++sourceSegmentIndex;
+				sourceSegmentOffset = 0;
+			}
+
+			int sourceRemainingBytes = Math.min(bufferSize - sourceSegmentOffset, recordRemainingBytes);
+			int numBytes = Math.min(targetSegmentSize - targetSegmentOffset, sourceRemainingBytes);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+			sourceSegment.copyTo(sourceSegmentOffset, targetSegment, targetSegmentOffset, numBytes);
+
+			recordRemainingBytes -= numBytes;
+			targetSegmentOffset += numBytes;
+			sourceSegmentOffset += numBytes;
+		} while ((recordRemainingBytes > 0 && targetSegmentOffset < targetSegmentSize));
+
+		return numBytesToCopy;
+	}
+
+	private void updateReadChannelAndIndexEntryAddress() {
+		// skip the channels without any data
+		while (++readChannelIndex < firstIndexEntryAddresses.length) {
+			if ((readIndexEntryAddress = firstIndexEntryAddresses[readChannelIndex]) >= 0) {
+				break;
+			}
+		}
+	}
+
+	private int getHigh32BitsFromLongAsInteger(long value) {

Review comment:
       Maybe rename this to `getSegmentIndexFromPointer` ?

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
##########
@@ -173,6 +173,28 @@
 				" help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be" +
 				" increased in case of higher round trip times between nodes and/or larger number of machines in the cluster.");
 
+	/**
+	 * Maximum number of network buffers can be used per sort-merge blocking result partition.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_MAX_BUFFERS_PER_SORT_MERGE_PARTITION =
+		key("taskmanager.network.sort-merge-blocking-shuffle.max-buffers-per-partition")
+			.defaultValue(2048)
+			.withDescription("Maximum number of network buffers can be used per sort-merge blocking result partition. " +
+				"This value is only an upper bound limit and does not mean that the sort-merge blocking result partition" +
+				" will use as many network buffers.");
+
+	/**
+	 * Parallelism threshold to switch between sort-merge based blocking shuffle and the default hash-based blocking shuffle.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_SORT_MERGE_SHUFFLE_MIN_PARALLELISM =
+		key("taskmanager.network.sort-merge-blocking-shuffle.min-parallelism")

Review comment:
       Similar, maybe use `taskmanager.network.sort-shuffle.min-parallelism` here.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
##########
@@ -173,6 +173,28 @@
 				" help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be" +
 				" increased in case of higher round trip times between nodes and/or larger number of machines in the cluster.");
 
+	/**
+	 * Maximum number of network buffers can be used per sort-merge blocking result partition.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_MAX_BUFFERS_PER_SORT_MERGE_PARTITION =
+		key("taskmanager.network.sort-merge-blocking-shuffle.max-buffers-per-partition")
+			.defaultValue(2048)
+			.withDescription("Maximum number of network buffers can be used per sort-merge blocking result partition. " +

Review comment:
       Please add the `intType()` here to make it runtime type safe (and not have the deprecation warnings).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,348 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {
+			releaseCurrentSortBuffer();
+			return;
+		}
+
+		currentSortBuffer.finish();
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (currentSortBuffer.hasRemaining()) {
+			BufferWithChannel bufferWithChannel = currentSortBuffer.copyData(writeBuffer);
+			Buffer buffer = bufferWithChannel.getBuffer();
+			int subpartitionIndex = bufferWithChannel.getChannelIndex();
+
+			fileWriter.writeBuffer(buffer, subpartitionIndex);
+			updateStatistics(buffer, subpartitionIndex);
+		}
+
+		releaseCurrentSortBuffer();
+	}
+
+	private PartitionedFileWriter getPartitionedFileWriter() throws IOException {
+		if (fileWriter == null) {
+			String basePath = channelManager.createChannel().getPath();
+			fileWriter = new PartitionedFileWriter(basePath, numSubpartitions);
+			fileWriter.open();
+		}
+
+		fileWriter.startNewRegion();
+		return fileWriter;
+	}
+
+	private void updateStatistics(Buffer buffer, int subpartitionIndex) {
+		numBuffersOut.inc();
+		numBytesOut.inc(buffer.readableBytes());
+		if (buffer.isBuffer()) {
+			++numDataBuffers[subpartitionIndex];
+		}
+	}
+
+	/**
+	 * Spills the large record into the target {@link PartitionedFile} as a separate data region.
+	 */
+	private void writeLargeRecord(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		PartitionedFileWriter fileWriter = getPartitionedFileWriter();
+
+		while (record.hasRemaining()) {
+			int toCopy = Math.min(record.remaining(), writeBuffer.size());
+			writeBuffer.put(0, record, toCopy);
+
+			NetworkBuffer buffer = new NetworkBuffer(writeBuffer, (buf) -> {}, dataType, toCopy);
+			fileWriter.writeBuffer(buffer, targetSubpartition);
+		}
+	}
+
+	void releaseReader(SortMergeSubpartitionReader reader) {
+		synchronized (lock) {
+			readers.remove(reader);
+
+			// release the result partition if it has been marked as released
+			if (readers.isEmpty() && isReleased()) {
+				releaseInternal();
+			}
+		}
+	}
+
+	@Override
+	public void finish() throws IOException {
+		checkInProduceState();
+
+		broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+		flushCurrentSortBuffer();
+
+		synchronized (lock) {
+			checkState(!isReleased());
+
+			resultFile = fileWriter.finish();
+			fileWriter = null;
+
+			LOG.info("New partitioned file produced: {}.", resultFile);
+		}
+
+		super.finish();
+	}
+
+	@Override
+	public void close() {
+		releaseCurrentSortBuffer();
+
+		if (fileWriter != null) {
+			fileWriter.releaseQuietly();
+		}
+
+		super.close();
+	}
+
+	@Override
+	public ResultSubpartitionView createSubpartitionView(
+			int subpartitionIndex,
+			BufferAvailabilityListener availabilityListener) throws IOException {
+		synchronized (lock) {
+			checkElementIndex(subpartitionIndex, numSubpartitions, "Subpartition not found.");
+			checkState(!isReleased(), "Partition released.");
+			checkState(isFinished(), "Trying to read unfinished blocking partition.");
+
+			SortMergeSubpartitionReader reader = new SortMergeSubpartitionReader(
+				this,
+				availabilityListener,
+				subpartitionIndex,
+				numDataBuffers[subpartitionIndex],
+				networkBufferSize);
+			readers.add(reader);
+			availabilityListener.notifyDataAvailable();
+
+			return reader;
+		}
+	}
+
+	@Override
+	public void flushAll() {
+		try {
+			flushCurrentSortBuffer();
+		} catch (IOException e) {
+			LOG.error("Failed to flush the current sort buffer.", e);
+		}
+	}
+
+	@Override
+	public void flush(int subpartitionIndex) {
+		try {
+			flushCurrentSortBuffer();
+		} catch (IOException e) {
+			LOG.error("Failed to flush the current sort buffer.", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<?> getAvailableFuture() {
+		return AVAILABLE;
+	}
+
+	@Override
+	public int getNumberOfQueuedBuffers() {
+		return 0;
+	}
+
+	@Override
+	public int getNumberOfQueuedBuffers(int targetSubpartition) {
+		return 0;
+	}
+
+	public PartitionedFile getResultFile() {

Review comment:
       This method publicly exposes a field that is otherwise lock-guarded. Would be good to avoid that, or at least not in the production scope.
   
   You can do the following:
     - pass this directly to the constructor of `SortMergeSubpartitionReader`
     - reduce visibility to package-private and annotate it as `@VisibleForTesting`.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
##########
@@ -173,6 +173,28 @@
 				" help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be" +
 				" increased in case of higher round trip times between nodes and/or larger number of machines in the cluster.");
 
+	/**
+	 * Maximum number of network buffers can be used per sort-merge blocking result partition.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_MAX_BUFFERS_PER_SORT_MERGE_PARTITION =
+		key("taskmanager.network.sort-merge-blocking-shuffle.max-buffers-per-partition")
+			.defaultValue(2048)
+			.withDescription("Maximum number of network buffers can be used per sort-merge blocking result partition. " +
+				"This value is only an upper bound limit and does not mean that the sort-merge blocking result partition" +
+				" will use as many network buffers.");
+
+	/**
+	 * Parallelism threshold to switch between sort-merge based blocking shuffle and the default hash-based blocking shuffle.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_SORT_MERGE_SHUFFLE_MIN_PARALLELISM =
+		key("taskmanager.network.sort-merge-blocking-shuffle.min-parallelism")
+			.defaultValue(Integer.MAX_VALUE)

Review comment:
       Please add the `intType()` here to make it runtime type safe (and not have the deprecation warnings).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;
+
+	/** A buffer pool to request memory segments from. */
+	private final BufferPool bufferPool;
+
+	/** A segment list as a joint buffer which stores all records and index entries. */
+	private final ArrayList<MemorySegment> buffers = new ArrayList<>();
+
+	/** Addresses of the first record's index entry for each subpartition. */
+	private final long[] firstIndexEntryAddresses;
+
+	/** Addresses of the last record's index entry for each subpartition. */
+	private final long[] lastIndexEntryAddresses;
+
+	/** Size of buffers requested from buffer pool. All buffers must be of the same size. */
+	private final int bufferSize;
+
+	// ----------------------------------------------------------------------------------------------
+	// Statistics and states
+	// ----------------------------------------------------------------------------------------------
+
+	/** Total number of bytes already appended to this sort buffer. */
+	private long numTotalBytes;
+
+	/** Total number of records already appended to this sort buffer. */
+	private long numTotalRecords;
+
+	/** Total number of bytes already read from this sort buffer. */
+	private long numTotalBytesRead;
+
+	/** Whether this sort buffer is finished. One can only read a finished sort buffer. */
+	private boolean isFinished;
+
+	/** Whether this sort buffer is released. A released sort buffer can not be used. */
+	private boolean isReleased;
+
+	// ----------------------------------------------------------------------------------------------
+	// For writing
+	// ----------------------------------------------------------------------------------------------
+
+	/** Array index in the segment list of the current available buffer for writing. */
+	private int writeSegmentIndex;
+
+	/** Next position in the current available buffer for writing. */
+	private int writeSegmentOffset;
+
+	// ----------------------------------------------------------------------------------------------
+	// For reading
+	// ----------------------------------------------------------------------------------------------
+
+	/** Index entry address of the current record or event to be read. */
+	private long readIndexEntryAddress;
+
+	/** Record bytes remaining after last copy, which must be read first in next copy. */
+	private int recordRemainingBytes;
+
+	/** Current available channel to read data from. */
+	private int readChannelIndex = -1;
+
+	public PartitionSortedBuffer(BufferPool bufferPool, int numSubpartitions, int bufferSize) {
+		checkArgument(bufferSize > INDEX_ENTRY_SIZE, "Buffer size is too small.");
+
+		this.bufferPool = checkNotNull(bufferPool);
+		this.bufferSize = bufferSize;
+		this.firstIndexEntryAddresses = new long[numSubpartitions];
+		this.lastIndexEntryAddresses = new long[numSubpartitions];
+
+		// initialized with -1 means the corresponding channel has no data
+		Arrays.fill(firstIndexEntryAddresses, -1L);
+		Arrays.fill(lastIndexEntryAddresses, -1L);
+	}
+
+	@Override
+	public boolean append(ByteBuffer source, int targetChannel, DataType dataType) throws IOException {
+		checkState(!isFinished, "Sort buffer is already finished.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int totalBytes = source.remaining();
+		if (totalBytes == 0) {
+			return true;
+		}
+
+		// return false directly if it can not allocate enough buffers for the given record
+		if (!allocateBuffersForRecord(totalBytes)) {
+			return false;
+		}
+
+		// write the index entry and record or event data
+		writeIndex(targetChannel, totalBytes, dataType);
+		writeRecord(source);
+
+		++numTotalRecords;
+		numTotalBytes += totalBytes;
+
+		return true;
+	}
+
+	private void writeIndex(int channelIndex, int numRecordBytes, Buffer.DataType dataType) {
+		MemorySegment segment = buffers.get(writeSegmentIndex);
+
+		// record length takes the high 32 bits and data type takes the low 32 bits
+		segment.putLong(writeSegmentOffset, ((long) numRecordBytes << 32) | dataType.ordinal());
+
+		// segment index takes the high 32 bits and segment offset takes the low 32 bits
+		long indexEntryAddress = ((long) writeSegmentIndex << 32) | writeSegmentOffset;
+
+		long lastIndexEntryAddress =  lastIndexEntryAddresses[channelIndex];
+		lastIndexEntryAddresses[channelIndex] = indexEntryAddress;
+
+		if (lastIndexEntryAddress >= 0) {
+			// link the previous index entry of the given channel to the new index entry
+			segment = buffers.get(getHigh32BitsFromLongAsInteger(lastIndexEntryAddress));
+			segment.putLong(getLow32BitsFromLongAsInteger(lastIndexEntryAddress) + 8, indexEntryAddress);
+		} else {
+			firstIndexEntryAddresses[channelIndex] = indexEntryAddress;
+		}
+
+		// move the write position forward so as to write the corresponding record
+		updateWriteSegmentIndexAndOffset(INDEX_ENTRY_SIZE);
+	}
+
+	private void writeRecord(ByteBuffer source) {
+		while (source.hasRemaining()) {
+			MemorySegment segment = buffers.get(writeSegmentIndex);
+			int toCopy = Math.min(bufferSize - writeSegmentOffset, source.remaining());
+			segment.put(writeSegmentOffset, source, toCopy);
+
+			// move the write position forward so as to write the remaining bytes or next record
+			updateWriteSegmentIndexAndOffset(toCopy);
+		}
+	}
+
+	private boolean allocateBuffersForRecord(int numRecordBytes) throws IOException {
+		int numBytesRequired = INDEX_ENTRY_SIZE + numRecordBytes;
+		int availableBytes = writeSegmentIndex == buffers.size() ? 0 : bufferSize - writeSegmentOffset;
+
+		// return directly if current available bytes is adequate
+		if (availableBytes >= numBytesRequired) {
+			return true;
+		}
+
+		// skip the remaining free space if the available bytes is not enough for an index entry
+		if (availableBytes < INDEX_ENTRY_SIZE) {
+			updateWriteSegmentIndexAndOffset(availableBytes);
+			availableBytes = 0;
+		}
+
+		// allocate exactly enough buffers for the appended record
+		do {
+			MemorySegment segment = requestBufferFromPool();
+			if (segment == null) {
+				// return false if we can not allocate enough buffers for the appended record
+				return false;
+			}
+
+			assert segment.size() == bufferSize;
+			availableBytes += bufferSize;
+			buffers.add(segment);
+		} while (availableBytes < numBytesRequired);
+
+		return true;
+	}
+
+	private MemorySegment requestBufferFromPool() throws IOException {
+		try {
+			// blocking request buffers if there is still guaranteed memory
+			if (buffers.size() < bufferPool.getNumberOfRequiredMemorySegments()) {
+				return bufferPool.requestBufferBuilderBlocking().getMemorySegment();
+			}
+		} catch (InterruptedException e) {
+			throw new IOException("Interrupted while requesting buffer.");
+		}
+
+		BufferBuilder buffer = bufferPool.requestBufferBuilder();
+		return buffer != null ? buffer.getMemorySegment() : null;
+	}
+
+	private void updateWriteSegmentIndexAndOffset(int numBytes) {
+		writeSegmentOffset += numBytes;
+
+		// using the next available free buffer if the current is full
+		if (writeSegmentOffset == bufferSize) {
+			++writeSegmentIndex;
+			writeSegmentOffset = 0;
+		}
+	}
+
+	@Override
+	public BufferWithChannel copyData(MemorySegment target) {
+		checkState(hasRemaining(), "No data remaining.");
+		checkState(isFinished, "Should finish the sort buffer first before coping any data.");
+		checkState(!isReleased, "Sort buffer is already released.");
+
+		int numBytesCopied = 0;
+		DataType bufferDataType = DataType.DATA_BUFFER;
+		int channelIndex = readChannelIndex;
+
+		do {
+			int sourceSegmentIndex = getHigh32BitsFromLongAsInteger(readIndexEntryAddress);
+			int sourceSegmentOffset = getLow32BitsFromLongAsInteger(readIndexEntryAddress);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+
+			long lengthAndDataType = sourceSegment.getLong(sourceSegmentOffset);
+			int length = getHigh32BitsFromLongAsInteger(lengthAndDataType);
+			DataType dataType = DataType.values()[getLow32BitsFromLongAsInteger(lengthAndDataType)];
+
+			// return the data read directly if the next to read is an event
+			if (dataType.isEvent() && numBytesCopied > 0) {
+				break;
+			}
+			bufferDataType = dataType;
+
+			// get the next index entry address and move the read position forward
+			long nextReadIndexEntryAddress = sourceSegment.getLong(sourceSegmentOffset + 8);
+			sourceSegmentOffset += INDEX_ENTRY_SIZE;
+
+			// allocate a temp buffer for the event if the target buffer is not big enough
+			if (bufferDataType.isEvent() && target.size() < length) {
+				target = MemorySegmentFactory.allocateUnpooledSegment(length);
+			}
+
+			numBytesCopied += copyRecordOrEvent(
+				target, numBytesCopied, sourceSegmentIndex, sourceSegmentOffset, length);
+
+			if (recordRemainingBytes == 0) {
+				// move to next channel if the current channel has been finished
+				if (readIndexEntryAddress == lastIndexEntryAddresses[channelIndex]) {
+					updateReadChannelAndIndexEntryAddress();
+					break;
+				}
+				readIndexEntryAddress = nextReadIndexEntryAddress;
+			}
+		} while (numBytesCopied < target.size() && bufferDataType.isBuffer());
+
+		numTotalBytesRead += numBytesCopied;
+		Buffer buffer = new NetworkBuffer(target, (buf) -> {}, bufferDataType, numBytesCopied);
+		return new BufferWithChannel(buffer, channelIndex);
+	}
+
+	private int copyRecordOrEvent(
+			MemorySegment targetSegment,
+			int targetSegmentOffset,
+			int sourceSegmentIndex,
+			int sourceSegmentOffset,
+			int recordLength) {
+		if (recordRemainingBytes > 0) {
+			// skip the data already read if there is remaining partial record after the previous copy
+			long position = (long) sourceSegmentOffset + (recordLength - recordRemainingBytes);
+			sourceSegmentIndex += (position / bufferSize);
+			sourceSegmentOffset = (int) (position % bufferSize);
+		} else {
+			recordRemainingBytes = recordLength;
+		}
+
+		int targetSegmentSize = targetSegment.size();
+		int numBytesToCopy = Math.min(targetSegmentSize - targetSegmentOffset, recordRemainingBytes);
+		do {
+			// move to next data buffer if all data of the current buffer has been copied
+			if (sourceSegmentOffset == bufferSize) {
+				++sourceSegmentIndex;
+				sourceSegmentOffset = 0;
+			}
+
+			int sourceRemainingBytes = Math.min(bufferSize - sourceSegmentOffset, recordRemainingBytes);
+			int numBytes = Math.min(targetSegmentSize - targetSegmentOffset, sourceRemainingBytes);
+			MemorySegment sourceSegment = buffers.get(sourceSegmentIndex);
+			sourceSegment.copyTo(sourceSegmentOffset, targetSegment, targetSegmentOffset, numBytes);
+
+			recordRemainingBytes -= numBytes;
+			targetSegmentOffset += numBytes;
+			sourceSegmentOffset += numBytes;
+		} while ((recordRemainingBytes > 0 && targetSegmentOffset < targetSegmentSize));
+
+		return numBytesToCopy;
+	}
+
+	private void updateReadChannelAndIndexEntryAddress() {
+		// skip the channels without any data
+		while (++readChannelIndex < firstIndexEntryAddresses.length) {
+			if ((readIndexEntryAddress = firstIndexEntryAddresses[readChannelIndex]) >= 0) {
+				break;
+			}
+		}
+	}
+
+	private int getHigh32BitsFromLongAsInteger(long value) {
+		return (int) (value >>> 32);
+	}
+
+	private int getLow32BitsFromLongAsInteger(long value) {

Review comment:
       Maybe rename this to `getSegmentOffsetFromPointer` ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java
##########
@@ -166,6 +189,14 @@ else if (type == ResultPartitionType.BLOCKING || type == ResultPartitionType.BLO
 		return partition;
 	}
 
+	private boolean isBlockingShuffle(ResultPartitionType type) {

Review comment:
       You don't need this, you can just say `!type.isPipelined()`.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
##########
@@ -173,6 +173,28 @@
 				" help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be" +
 				" increased in case of higher round trip times between nodes and/or larger number of machines in the cluster.");
 
+	/**
+	 * Maximum number of network buffers can be used per sort-merge blocking result partition.
+	 */
+	@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+	public static final ConfigOption<Integer> NETWORK_MAX_BUFFERS_PER_SORT_MERGE_PARTITION =
+		key("taskmanager.network.sort-merge-blocking-shuffle.max-buffers-per-partition")

Review comment:
       Can we make the keys a bit shorter? For example use `taskmanager.network.sort-shuffle.max-buffers`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java
##########
@@ -215,10 +246,19 @@ private static void releasePartitionsQuietly(ResultSubpartition[] partitions, in
 		return () -> {
 			int maxNumberOfMemorySegments = type.isBounded() ?
 				numberOfSubpartitions * networkBuffersPerChannel + floatingNetworkBuffersPerGate : Integer.MAX_VALUE;
+			int numRequiredBuffers = numberOfSubpartitions + 1;
+
+			if (isSortMergeBlockingShuffle(type, numberOfSubpartitions)) {

Review comment:
       Maybe rewrite this to not compute it twice in the sort shuffle mode (once for non-sort shuffle, then for sort shuffle).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-718444548


   Very thanks for @wsry  opening the PR and @StephanEwen @zhijiangW for the reviewing! The PR also looks good to me in general from my side and thus +1 for merging. A small issue is that if available, I still tend to that we add some tests covering the case some subpartitions are empty in the SortBuffer or in one region, as @zhijiangW pointed out before, since we have some logic related to skip the empty sub-partitions. Of course if not available we may also add the tests in the separate PR. 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-719467007


   Thanks for the updates @wsry ! 
   
   I think most of my previous comments were addressed except for https://github.com/apache/flink/pull/13595#discussion_r514988635. And as @gaoyunhaii also mentioned above, it is better to supplement some tests for covering empty subpartition case.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r512634744



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter implements AutoCloseable {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data is cached in memory or not. */
+	private boolean allIndexDataCached = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions, int indexBufferSize) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+		checkArgument(indexBufferSize > 0, "Illegal index buffer size.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(indexBufferSize * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					allIndexDataCached = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		indexBuffer.flip();
+		if (indexBuffer.limit() > 0) {
+			BufferReaderWriterUtil.writeBuffer(indexFileChannel, indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		isFinished = true;
+
+		writeRegionIndex();
+		flushIndexBuffer();
+		indexBuffer.rewind();
+
+		close();
+		ByteBuffer indexDataCache = allIndexDataCached ? indexBuffer : null;
+		return new PartitionedFile(numRegions, numSubpartitions, dataFilePath, indexFilePath, indexDataCache);
+	}
+
+	/**
+	 * Used to close and delete the failed {@link PartitionedFile} when any exception occurs.
+	 */
+	public void releaseQuietly() {
+		close();
+		IOUtils.deleteFileQuietly(dataFilePath);
+		IOUtils.deleteFileQuietly(indexFilePath);
+	}
+
+	@Override
+	public void close() {
+		Throwable exception = null;
+
+		try {
+			dataFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = throwable;
+		}
+
+		try {
+			dataFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = throwable;

Review comment:
       This way, you'll lose the first exception. I would either keep the first exception or use the `ExceptionUtils.firstOrSuppressed()` pattern here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716321736


   @StephanEwen @zhijiangW @gaoyunhaii Thanks for the review and comments, I have updated the PR according to the review comments.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529",
       "triggerID" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 761eb39aefc6863bc0cc7c35c78d5a1a9d016997 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522) 
   * 4121561bd0e53433d56f19ebe8f8ab35b9755e34 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r513669487



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter implements AutoCloseable {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data is cached in memory or not. */
+	private boolean allIndexDataCached = true;
+
+	/** Whether this file writer is finished or not. */
+	private boolean isFinished;
+
+	/** Whether this file writer is closed or not. */
+	private boolean isClosed;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions, int indexBufferSize) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+		checkArgument(indexBufferSize > 0, "Illegal index buffer size.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(indexBufferSize * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					allIndexDataCached = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		indexBuffer.flip();
+		if (indexBuffer.limit() > 0) {
+			BufferReaderWriterUtil.writeBuffer(indexFileChannel, indexBuffer);
+		}
+	}
+
+	/**
+	 * Writes a {@link Buffer} of the given subpartition to the this {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for recycling the target buffer and releasing the failed
+	 * {@link PartitionedFile} if any exception occurs.
+	 */
+	public void writeBuffer(Buffer target, int targetSubpartition) throws IOException {
+		checkArgument(targetSubpartition >= currentSubpartition, "Must write in subpartition index order.");
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		currentSubpartition = Math.max(currentSubpartition, targetSubpartition);
+		long numBytes = BufferReaderWriterUtil.writeToByteChannel(dataFileChannel, target, header);
+
+		++subpartitionBuffers[targetSubpartition];
+		subpartitionBytes[targetSubpartition] += numBytes;
+	}
+
+	/**
+	 * Finishes writing which closes the file channel and returns the corresponding {@link PartitionedFile}.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public PartitionedFile finish() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(!isClosed, "File writer is already closed.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		isFinished = true;
+
+		writeRegionIndex();
+		flushIndexBuffer();
+		indexBuffer.rewind();
+
+		close();
+		ByteBuffer indexDataCache = allIndexDataCached ? indexBuffer : null;
+		return new PartitionedFile(numRegions, numSubpartitions, dataFilePath, indexFilePath, indexDataCache);
+	}
+
+	/**
+	 * Used to close and delete the failed {@link PartitionedFile} when any exception occurs.
+	 */
+	public void releaseQuietly() {
+		IOUtils.closeQuietly(this);
+		IOUtils.deleteFileQuietly(dataFilePath);
+		IOUtils.deleteFileQuietly(indexFilePath);
+	}
+
+	@Override
+	public void close() {
+		if (isClosed) {
+			return;
+		}
+		isClosed = true;
+
+		Throwable exception = null;
+		try {
+			dataFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = throwable;
+		}
+
+		try {
+			indexFileChannel.close();
+		} catch (Throwable throwable) {
+			exception = ExceptionUtils.firstOrSuppressed(throwable, exception);
+		}
+
+		if (exception != null) {
+			ExceptionUtils.rethrow(exception);

Review comment:
       Nit: You can add `throws Exception` to the method signature (compatible with `AutoClosable`) and then you don't need to wrap it as a RuntimeException.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96fc01e7f199429da15dbcae89762c9314e6780a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510579608



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionedFileWriter.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+import org.apache.flink.util.IOUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * File writer which can write buffers and generate the {@link PartitionedFile}. Data is written region by region.
+ * Before writing any data, {@link #open} must be called and before writing a new region, {@link #startNewRegion}
+ * must be called. After writing all data, {@link #finish} must be called to close all opened files and return the
+ * target {@link PartitionedFile}.
+ */
+@NotThreadSafe
+public class PartitionedFileWriter {
+
+	/** Used when writing data buffers. */
+	private final ByteBuffer[] header = BufferReaderWriterUtil.allocatedWriteBufferArray();
+
+	/** Buffer for writing region index. */
+	private final ByteBuffer indexBuffer;
+
+	/** Number of channels. When writing a buffer, target subpartition must be in this range. */
+	private final int numSubpartitions;
+
+	/** Data file path of the target {@link PartitionedFile}. */
+	private final Path dataFilePath;
+
+	/** Index file path of the target {@link PartitionedFile}. */
+	private final Path indexFilePath;
+
+	/** Number of bytes written for each subpartition in the current region. */
+	private final long[] subpartitionBytes;
+
+	/** Number of buffers written for each subpartition in the current region. */
+	private final int[] subpartitionBuffers;
+
+	/** Opened data file channel of the target {@link PartitionedFile}. */
+	private FileChannel dataFileChannel;
+
+	/** Opened index file channel of the target {@link PartitionedFile}. */
+	private FileChannel indexFileChannel;
+
+	/** Number of bytes written to the target {@link PartitionedFile}. */
+	private long totalBytesWritten;
+
+	/** Number of regions written to the target {@link PartitionedFile}. */
+	private int numRegions;
+
+	/** Current subpartition to write. Buffer writing must be in subpartition order within each region. */
+	private int currentSubpartition;
+
+	/** Whether all index data can be cached in memory or not. */
+	private boolean canCacheAllIndexData = true;
+
+	/** Whether this file writer is finished. */
+	private boolean isFinished;
+
+	public PartitionedFileWriter(String basePath, int numSubpartitions) {
+		checkArgument(basePath != null, "Base path must not be null.");
+		checkArgument(numSubpartitions > 0, "Illegal number of subpartitions.");
+
+		this.numSubpartitions = numSubpartitions;
+		this.subpartitionBytes = new long[numSubpartitions];
+		this.subpartitionBuffers = new int[numSubpartitions];
+		this.dataFilePath = new File(basePath + PartitionedFile.DATA_FILE_SUFFIX).toPath();
+		this.indexFilePath = new File(basePath + PartitionedFile.INDEX_FILE_SUFFIX).toPath();
+
+		this.indexBuffer = ByteBuffer.allocate(100 * 1024 * PartitionedFile.INDEX_ENTRY_SIZE);
+		indexBuffer.order(PartitionedFile.DEFAULT_BYTE_ORDER);
+	}
+
+	/**
+	 * Opens the {@link PartitionedFile} for writing.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void open() throws IOException {
+		checkState(dataFileChannel == null && indexFileChannel == null, "Partitioned file is already opened.");
+
+		dataFileChannel = openFileChannel(dataFilePath);
+		indexFileChannel = openFileChannel(indexFilePath);
+	}
+
+	private FileChannel openFileChannel(Path path) throws IOException {
+		return FileChannel.open(path, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE);
+	}
+
+	/**
+	 * Persists the region index of the current data region and starts a new region to write.
+	 *
+	 * <p>Note: The caller is responsible for releasing the failed {@link PartitionedFile} if any exception
+	 * occurs.
+	 */
+	public void startNewRegion() throws IOException {
+		checkState(!isFinished, "File writer is already finished.");
+		checkState(dataFileChannel != null && indexFileChannel != null, "Must open the partitioned file first.");
+
+		writeRegionIndex();
+	}
+
+	private void writeRegionIndex() throws IOException {
+		if (Arrays.stream(subpartitionBytes).sum() > 0) {
+			for (int subpartition = 0; subpartition < numSubpartitions; ++subpartition) {
+				if (!indexBuffer.hasRemaining()) {
+					flushIndexBuffer();
+					indexBuffer.clear();
+					canCacheAllIndexData = false;
+				}
+
+				indexBuffer.putLong(totalBytesWritten);
+				indexBuffer.putInt(subpartitionBuffers[subpartition]);
+				totalBytesWritten += subpartitionBytes[subpartition];
+			}
+
+			++numRegions;
+			currentSubpartition = 0;
+			Arrays.fill(subpartitionBytes, 0);
+			Arrays.fill(subpartitionBuffers, 0);
+		}
+	}
+
+	private void flushIndexBuffer() throws IOException {
+		if (indexBuffer.position() > 0) {
+			indexBuffer.flip();
+			indexFileChannel.write(indexBuffer);

Review comment:
       I think here we still need to iterates till the whole buffer is write (writeFully). The dataFileChannel has also done this. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041) 
   * 690ec79d32dbd8e2a9c62556c0548a56c6a52521 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522",
       "triggerID" : "761eb39aefc6863bc0cc7c35c78d5a1a9d016997",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529",
       "triggerID" : "4121561bd0e53433d56f19ebe8f8ab35b9755e34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300) 
   * 761eb39aefc6863bc0cc7c35c78d5a1a9d016997 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8522) 
   * 4121561bd0e53433d56f19ebe8f8ab35b9755e34 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8529) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500) 
   * 62395a70a02dfd79375f51134f1c195824eeb12e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f5e7c69522f61d5d992b85371340f232b59f3cdd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950) 
   * 0ea0d823be70a718ca9ac3c7684e8b46552ad634 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035) 
   * d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f5e7c69522f61d5d992b85371340f232b59f3cdd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0ea0d823be70a718ca9ac3c7684e8b46552ad634 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035) 
   * d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511600676



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();

Review comment:
       The sort buffer can not be used, because the internal state has been broken after failing to write the large record. Maybe we can optimize the implementation of sort buffer to achieve that in the future.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510037094



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java
##########
@@ -136,26 +144,41 @@ public ResultPartition create(
 
 			partition = pipelinedPartition;
 		}
-		else if (type == ResultPartitionType.BLOCKING || type == ResultPartitionType.BLOCKING_PERSISTENT) {
-			final BoundedBlockingResultPartition blockingPartition = new BoundedBlockingResultPartition(
-				taskNameWithSubtaskAndId,
-				partitionIndex,
-				id,
-				type,
-				subpartitions,
-				maxParallelism,
-				partitionManager,
-				bufferCompressor,
-				bufferPoolFactory);
-
-			initializeBoundedBlockingPartitions(
-				subpartitions,
-				blockingPartition,
-				blockingSubpartitionType,
-				networkBufferSize,
-				channelManager);
-
-			partition = blockingPartition;
+		else if (isBlockingShuffle(type)) {
+			if (isSortMergeBlockingShuffle(type, numberOfSubpartitions)) {
+				partition = new SortMergeResultPartition(
+					taskNameWithSubtaskAndId,
+					partitionIndex,
+					id,
+					type,
+					subpartitions.length,
+					maxParallelism,
+					networkBufferSize,
+					partitionManager,
+					channelManager,
+					bufferCompressor,
+					bufferPoolFactory);
+			} else {
+				final BoundedBlockingResultPartition blockingPartition = new BoundedBlockingResultPartition(

Review comment:
       directly write as `partition = xxx` ?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716419919


   @StephanEwen 
   `taskmanager.network.sort-shuffle.max-buffers` is only used to control the max required buffers (min size) of the buffer pool now. I have update the code and the calculation logic is as follows:
   ```
   int maxNumberOfMemorySegments = type.isBounded() ? numberOfSubpartitions * networkBuffersPerChannel + floatingNetworkBuffersPerGate : Integer.MAX_VALUE;
   int numRequiredBuffers = !type.isPipelined() && sortMergeBlockingShuffleEnabled ? Math.min(numberOfSubpartitions + 1, maxBuffersPerSortMergePartition) : numberOfSubpartitions + 1;
   
   return bufferPoolFactory.createBufferPool(numRequiredBuffers, maxNumberOfMemorySegments, numberOfSubpartitions, maxBuffersPerChannel);
   ```
   
   I agree that `taskmanager.network.sort-shuffle.min-parallelism` is more flexible than a boolean flag and I will update the PR and use `taskmanager.network.sort-shuffle.min-parallelism`.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7950",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8035",
       "triggerID" : "0ea0d823be70a718ca9ac3c7684e8b46552ad634",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8041",
       "triggerID" : "d574b6bceaf041e56f5ba5e8e4e827b7d97a2c46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8248",
       "triggerID" : "690ec79d32dbd8e2a9c62556c0548a56c6a52521",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8253",
       "triggerID" : "9b97362104410740de393cbbfb1b27ca564dadc0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300",
       "triggerID" : "215743ea7cf68dcb7a61224fdea5a75f29a248a9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 215743ea7cf68dcb7a61224fdea5a75f29a248a9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8300) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716275114


   > Thanks a lot, the code looks generally good to me.
   > 
   > I think the current configuration works also as a feature flag, which is nice. If I read it correctly, sort-shuffle will start from a parallelism of INT_MAX, which means it is never used unless users explicitly configure it to be used. So we do not need any additional feature flag.
   > 
   > What confused my were the minimum and maximum buffer requirements.
   > 
   > * There is a maximum number of buffer (`taskmanager.network.sort-merge-blocking-shuffle.max-buffers-per-partition`). Why do we need that? What would happen if there were more buffers?
   > * We also still have a minimum of `numSubpartitions + 1` buffers. Is this a requirement for the write-phase, or only for the read phase? As I understand it, a part of the motivation of this sort-nased shuffle is to decouple the number of needed memory buffers from the parallelism. Could we drop this minimum?
   
   @StephanEwen 
   1. As suggested by @zhijiangW Zhijiang in the discussion thread, I replace `taskmanager.network.sort-shuffle.min-parallelism` with a feature flag `taskmanager.network.sort-shuffle.enabled` for ease of use.
   2. You are right, the motivation of `taskmanager.network.sort-shuffle.max-buffers` is used to decouple the number of needed memory buffers from the parallelism. So it is the upper bound of required memory, if `numSubpartitions + 1` is greater than this config value, this config value is used. The config option is currently used by write-phase, after we implement partition request restriction or buffer management in FLINK-16641, this option can also be reused by read phase. Then we can totally decouple the number of needed memory buffers from the parallelism for both write-phase and read-phase.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62395a70a02dfd79375f51134f1c195824eeb12e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-718482614


   Thanks for @gaoyunhaii for pointing out the test issue, I misunderstand @zhijiangW 's suggestion before and and add an IT case totally without data. I will supplement some UT case to cover the scenario.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-713667461


   This PR is ready for review.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] gaoyunhaii commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
gaoyunhaii commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-720789792


   There are no issues from my side and +1 for merging! 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96fc01e7f199429da15dbcae89762c9314e6780a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r511591531



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionSortedBuffer.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;
+
+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.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link SortBuffer} implementation which sorts all appended records only by subpartition index. Records of the
+ * same subpartition keep the appended order.
+ *
+ * <p>It maintains a list of {@link MemorySegment}s as a joint buffer. Data will be appended to the joint buffer
+ * sequentially. When writing a record, an index entry will be appended first. Each index entry has 4 fields: 4
+ * bytes record length, 4 bytes {@link DataType} and 8 bytes address pointing to the next index entry of the same
+ * channel which will be used to index the next record to read when coping data from this {@link SortBuffer}. For
+ * simplicity, no index entry can span multiple segments. The corresponding record data sits right after its index
+ * entry and different from the index entry, records have variable length thus may span multiple segments.
+ */
+@NotThreadSafe
+public class PartitionSortedBuffer implements SortBuffer {
+
+	/**
+	 * Size of an index entry: 4 bytes for record length, 4 bytes for data type and 8 bytes
+	 * for pointer to next entry.
+	 */
+	private static final int INDEX_ENTRY_SIZE = 4 + 4 + 8;

Review comment:
       Good point, I can further optimize it in the future PRs.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655",
       "triggerID" : "62395a70a02dfd79375f51134f1c195824eeb12e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5e7c69522f61d5d992b85371340f232b59f3cdd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62395a70a02dfd79375f51134f1c195824eeb12e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7655) 
   * f5e7c69522f61d5d992b85371340f232b59f3cdd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-707168234


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459",
       "triggerID" : "96fc01e7f199429da15dbcae89762c9314e6780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500",
       "triggerID" : "72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96fc01e7f199429da15dbcae89762c9314e6780a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7459) 
   * 72899f22581f9bb0d4cf5c636fd8c8cf6bfe6477 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7500) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-712899427


   I will do some minor update and this PR will be marked as ready for review by tomorrow.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] StephanEwen commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
StephanEwen commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-716404546


   @wsry 
   
   I am confused a bit about the use of max buffers. That value is the upper limit of buffers that will be assigned to the shuffle from the global pool. It is only used for streaming pipelined connections, to limit the amount of in-flight buffers (so checkpoints don't take too long). For batch it should not be used, because there is no harm in using as much memory as possible.
   
   The min buffers is actually hat decouples the memory use from the parallelism. If the min-buffers is related to the number of subpartitions, then we still have the problem that shuffles fail on large parallelism (parallelism is higher than available memory buffers).
   
   So, in conclusion, I think we should not have a max value (because it does not help in decoupling from parallelism) and also need to decouple the min value from the parallelism.
   
   I also liked the idea of having the `taskmanager.network.sort-shuffle.min-parallelism` as a flag. That way low parallelisms (< 50) could use the hash shuffle and larger parallelisms could use the sort shuffle.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wsry commented on pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
wsry commented on pull request #13595:
URL: https://github.com/apache/flink/pull/13595#issuecomment-720899276






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510025704



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,
+			@Nullable BufferCompressor bufferCompressor,
+			SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+
+		super(
+			owningTaskName,
+			partitionIndex,
+			partitionId,
+			partitionType,
+			numSubpartitions,
+			numTargetKeyGroups,
+			partitionManager,
+			bufferCompressor,
+			bufferPoolFactory);
+
+		this.channelManager = checkNotNull(channelManager);
+		this.networkBufferSize = networkBufferSize;
+		this.numDataBuffers = new int[numSubpartitions];
+		this.writeBuffer = MemorySegmentFactory.allocateUnpooledOffHeapMemory(networkBufferSize);
+	}
+
+	@Override
+	protected void releaseInternal() {
+		synchronized (lock) {
+			isFinished = true; // to fail writing faster
+
+			// delete the produced file only when no reader is reading now
+			if (readers.isEmpty()) {
+				if (resultFile != null) {
+					resultFile.deleteQuietly();
+					resultFile = null;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException {
+		emit(record, targetSubpartition, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastRecord(ByteBuffer record) throws IOException {
+		broadcast(record, DataType.DATA_BUFFER);
+	}
+
+	@Override
+	public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException {
+		Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+		try {
+			ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+			broadcast(serializedEvent, buffer.getDataType());
+		} finally {
+			buffer.recycleBuffer();
+		}
+	}
+
+	private void broadcast(ByteBuffer record, DataType dataType) throws IOException {
+		for (int channelIndex = 0; channelIndex < numSubpartitions; ++channelIndex) {
+			record.rewind();
+			emit(record, channelIndex, dataType);
+		}
+	}
+
+	private void emit(ByteBuffer record, int targetSubpartition, DataType dataType) throws IOException {
+		checkInProduceState();
+
+		SortBuffer sortBuffer = getSortBuffer();
+		if (sortBuffer.append(record, targetSubpartition, dataType)) {
+			return;
+		}
+
+		if (!sortBuffer.hasRemaining()) {
+			// the record can not be appended to the free sort buffer because it is too large
+			releaseCurrentSortBuffer();
+			writeLargeRecord(record, targetSubpartition, dataType);
+			return;
+		}
+
+		flushCurrentSortBuffer();
+		emit(record, targetSubpartition, dataType);
+	}
+
+	private void releaseCurrentSortBuffer() {
+		if (currentSortBuffer != null) {
+			currentSortBuffer.release();
+			currentSortBuffer = null;
+		}
+	}
+
+	private SortBuffer getSortBuffer() {
+		if (currentSortBuffer != null) {
+			return currentSortBuffer;
+		}
+
+		currentSortBuffer = new PartitionSortedBuffer(bufferPool, numSubpartitions, networkBufferSize);
+		return currentSortBuffer;
+	}
+
+	private void flushCurrentSortBuffer() throws IOException {
+		if (currentSortBuffer == null || !currentSortBuffer.hasRemaining()) {

Review comment:
       The condition of `currentSortBuffer == null` should not trigger the `releaseCurrentSortBuffer`. Maybe we can split them as below:
   
   ```
   if (currentSortBuffer == null) {
        return;
   }
   
   if (currentSortBuffer.hasRemaining()) {
         .........
   }
   
   releaseCurrentSortBuffer();
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] zhijiangW commented on a change in pull request #13595: [FLINK-19582][network] Introduce sort-merge based blocking shuffle to Flink

Posted by GitBox <gi...@apache.org>.
zhijiangW commented on a change in pull request #13595:
URL: https://github.com/apache/flink/pull/13595#discussion_r510058532



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.FileChannelManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType;
+import static org.apache.flink.runtime.io.network.partition.SortBuffer.BufferWithChannel;
+import static org.apache.flink.util.Preconditions.checkElementIndex;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link SortMergeResultPartition} appends records and events to {@link SortBuffer} and after the {@link SortBuffer}
+ * is full, all data in the {@link SortBuffer} will be copied and spilled to a {@link PartitionedFile} in subpartition
+ * index order sequentially. Large records that can not be appended to an empty {@link SortBuffer} will be spilled to
+ * the {@link PartitionedFile} separately.
+ */
+@NotThreadSafe
+public class SortMergeResultPartition extends ResultPartition {
+
+	private final Object lock = new Object();
+
+	/** All active readers which are consuming data from this result partition now. */
+	@GuardedBy("lock")
+	private final Set<SortMergeSubpartitionReader> readers = new HashSet<>();
+
+	/** {@link PartitionedFile} produced by this result partition. */
+	@GuardedBy("lock")
+	private PartitionedFile resultFile;
+
+	/** Used to generate random file channel ID. */
+	private final FileChannelManager channelManager;
+
+	/** Number of data buffers (excluding events) written for each subpartition. */
+	private final int[] numDataBuffers;
+
+	/** A piece of unmanaged memory for data writing. */
+	private final MemorySegment writeBuffer;
+
+	/** Size of network buffer and write buffer. */
+	private final int networkBufferSize;
+
+	/** Current {@link SortBuffer} to append records to. */
+	private SortBuffer currentSortBuffer;
+
+	/** File writer for this result partition. */
+	private PartitionedFileWriter fileWriter;
+
+	public SortMergeResultPartition(
+			String owningTaskName,
+			int partitionIndex,
+			ResultPartitionID partitionId,
+			ResultPartitionType partitionType,
+			int numSubpartitions,
+			int numTargetKeyGroups,
+			int networkBufferSize,
+			ResultPartitionManager partitionManager,
+			FileChannelManager channelManager,

Review comment:
       nit: might be better to pass `Path` instead of big component `channelManager` as constructor argument.




----------------------------------------------------------------
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:
users@infra.apache.org