reswqa commented on code in PR #20371:
URL: https://github.com/apache/flink/pull/20371#discussion_r933261437


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartition.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.hybrid;
+
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.StopMode;
+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.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link HsResultPartition} appends records and events to {@link 
HsMemoryDataManager}, the shuffle
+ * data maybe spilled to disk according to the {@link HsSpillingStrategy}, and 
the downstream can
+ * consume data from memory or disk.
+ */
+public class HsResultPartition extends ResultPartition {
+    public static final String DATA_FILE_SUFFIX = ".hybrid.data";
+
+    private final HsFileDataIndex dataIndex;
+
+    private final HsFileDataManager fileDataManager;
+
+    private final Path dataFilePath;
+
+    private final int networkBufferSize;
+
+    private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+    private boolean hasNotifiedEndOfUserRecords;
+
+    private HsMemoryDataManager memoryDataManager;
+
+    public HsResultPartition(
+            String owningTaskName,
+            int partitionIndex,
+            ResultPartitionID partitionId,
+            ResultPartitionType partitionType,
+            int numSubpartitions,
+            int numTargetKeyGroups,
+            BatchShuffleReadBufferPool readBufferPool,
+            Executor readIOExecutor,
+            ResultPartitionManager partitionManager,
+            String dataFileBashPath,
+            int networkBufferSize,
+            HybridShuffleConfiguration hybridShuffleConfiguration,
+            @Nullable BufferCompressor bufferCompressor,
+            SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+        super(
+                owningTaskName,
+                partitionIndex,
+                partitionId,
+                partitionType,
+                numSubpartitions,
+                numTargetKeyGroups,
+                partitionManager,
+                bufferCompressor,
+                bufferPoolFactory);
+        this.networkBufferSize = networkBufferSize;
+        this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+        this.dataFilePath = new File(dataFileBashPath + 
DATA_FILE_SUFFIX).toPath();
+        this.hybridShuffleConfiguration = hybridShuffleConfiguration;
+        this.fileDataManager =
+                new HsFileDataManager(
+                        readBufferPool,
+                        readIOExecutor,
+                        dataIndex,
+                        dataFilePath,
+                        HsSubpartitionFileReaderImpl.Factory.INSTANCE,
+                        hybridShuffleConfiguration);
+    }
+
+    // Called by task thread.
+    @Override
+    protected void setupInternal() throws IOException {
+        if (isReleased()) {
+            throw new IOException("Result partition has been released.");
+        }
+        this.fileDataManager.setup();
+        this.memoryDataManager =
+                new HsMemoryDataManager(
+                        numSubpartitions,
+                        networkBufferSize,
+                        bufferPool,
+                        getSpillingStrategy(hybridShuffleConfiguration),
+                        dataIndex,
+                        dataFilePath);
+    }
+
+    @Override
+    public void emitRecord(ByteBuffer record, int targetSubpartition) throws 
IOException {
+        emit(record, targetSubpartition, Buffer.DataType.DATA_BUFFER);
+    }
+
+    @Override
+    public void broadcastRecord(ByteBuffer record) throws IOException {
+        broadcast(record, Buffer.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, Buffer.DataType dataType) throws 
IOException {
+        for (int i = 0; i < numSubpartitions; i++) {
+            emit(record.duplicate(), i, dataType);
+        }
+    }
+
+    private void emit(ByteBuffer record, int targetSubpartition, 
Buffer.DataType dataType)
+            throws IOException {
+        checkInProduceState();
+        memoryDataManager.append(record, targetSubpartition, dataType);
+    }
+
+    @Override
+    public ResultSubpartitionView createSubpartitionView(
+            int subpartitionId, BufferAvailabilityListener 
availabilityListener)
+            throws IOException {
+        checkState(!isReleased(), "ResultPartition already released.");
+        HsSubpartitionView subpartitionView = new 
HsSubpartitionView(availabilityListener);
+        HsDataView diskDataView =
+                fileDataManager.registerNewSubpartition(subpartitionId, 
subpartitionView);
+
+        memoryDataManager.registerSubpartitionView(subpartitionId, 
subpartitionView);
+        subpartitionView.setDiskDataView(diskDataView);
+        subpartitionView.setMemoryDataView(
+                memoryDataManager.getSubpartitionDataView(subpartitionId));
+        return subpartitionView;
+    }
+
+    @Override
+    public void alignedBarrierTimeout(long checkpointId) throws IOException {
+        // Nothing to do.
+    }
+
+    @Override
+    public void abortCheckpoint(long checkpointId, CheckpointException cause) {
+        // Nothing to do.
+    }
+
+    @Override
+    public void flushAll() {
+        // Nothing to do.
+    }
+
+    @Override
+    public void flush(int subpartitionIndex) {
+        // Nothing to do.
+    }
+
+    @Override
+    public void finish() throws IOException {
+        broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+
+        checkState(!isReleased(), "Result partition is already released.");
+
+        super.finish();
+    }
+
+    @Override
+    public void close() {
+        // close is called when task is finished or failed.
+        memoryDataManager.close();
+        super.close();
+    }
+
+    @Override
+    protected void releaseInternal() {
+        // release is called when release by scheduler, later than close.
+        // mainly work :
+        // 1. release read scheduler.
+        // 2. delete shuffle file.
+        // 3. release all data in memory.
+
+        // delete the shuffle file only when no reader is reading now.
+        fileDataManager.release().thenRun(fileDataManager::deleteShuffleFile);

Review Comment:
   Combined.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to