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:
[email protected]


Reply via email to