xintongsong commented on code in PR #22804:
URL: https://github.com/apache/flink/pull/22804#discussion_r1233232708


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/SegmentNettyPayload.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+
+import java.util.List;
+
+/**
+ * The wrapper class {@link SegmentNettyPayload} for a segment, which holds 
all the {@link
+ * NettyPayload} buffers and the flag that mark whether this segment need to 
be finished.
+ */
+public class SegmentNettyPayload {

Review Comment:
   1. It doesn't make sense that a file-related class is named after netty. I 
see this class internally uses `NettyPayload`. Then it probably means we should 
make `NettyPaylaod` something more general.
   2. These two classes, `SegmentNettyPayload` and `SubpartitionNettyPayload`, 
only serve as the argument type for `PartitionFileWriter`. They barely makes 
any sense independently. I'd suggest to make them inner classes of 
`PartitionFileWriter`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileWriter.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.tiered.storage.file;

Review Comment:
   I'd suggest `*.hybird.tiered.file`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskCacheManager.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.tiered.tier.disk;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SegmentNettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SubpartitionNettyPayload;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link DiskCacheManager} is responsible for managing cached buffers 
before flushing to files.
+ */
+public class DiskCacheManager {
+
+    private final int numSubpartitions;
+
+    private final PartitionFileWriter partitionFileWriter;
+
+    private final SubpartitionDiskCacheManager[] subpartitionCacheManagers;
+
+    private CompletableFuture<Void> hasFlushCompleted;
+
+    public DiskCacheManager(
+            int numSubpartitions,
+            TieredStorageMemoryManager storageMemoryManager,
+            PartitionFileWriter partitionFileWriter) {
+        this.numSubpartitions = numSubpartitions;
+        this.partitionFileWriter = partitionFileWriter;
+        this.subpartitionCacheManagers = new 
SubpartitionDiskCacheManager[numSubpartitions];
+        this.hasFlushCompleted = FutureUtils.completedVoidFuture();
+
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionCacheManagers[subpartitionId] =
+                    new SubpartitionDiskCacheManager(subpartitionId);
+        }
+        
storageMemoryManager.listenBufferReclaimRequest(this::notifyFlushCachedBuffers);
+    }
+
+    /**
+     * Append the end-of-segment event to {@link DiskCacheManager}.
+     *
+     * @param record the end-of-segment event
+     * @param subpartitionId target subpartition of this record.
+     * @param dataType the type of this record. In other words, is it data or 
event.
+     */
+    public void appendEndOfSegmentEvent(
+            ByteBuffer record, int subpartitionId, Buffer.DataType dataType) {
+        
subpartitionCacheManagers[subpartitionId].appendEndOfSegmentEvent(record, 
dataType);
+        flushAndReleaseCacheBuffers();

Review Comment:
   Why do we need flush here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/ProducerMergePartitionFileWriter.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.tiered.storage.file;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+
+import 
org.apache.flink.shaded.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** THe implementation of {@link PartitionFileWriter} with merged logic. */
+public class ProducerMergePartitionFileWriter implements PartitionFileWriter {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(ProducerMergePartitionFileWriter.class);
+
+    /** One thread to perform spill operation. */
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadExecutor(
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("ProducerMergePartitionFileWriter 
Spiller")
+                            
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());
+
+    /** File channel to write data. */
+    private final FileChannel dataFileChannel;
+
+    /** Records the current writing location. */
+    private long totalBytesWritten;
+
+    private final PartitionFileIndex partitionFileIndex;
+
+    public ProducerMergePartitionFileWriter(
+            Path dataFilePath, PartitionFileIndex partitionFileIndex) {
+        LOG.info("Creating partition file " + dataFilePath);
+        try {
+            this.dataFileChannel =
+                    FileChannel.open(
+                            dataFilePath, StandardOpenOption.CREATE_NEW, 
StandardOpenOption.WRITE);
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to create file channel.", e);
+        }
+        this.partitionFileIndex = partitionFileIndex;
+    }
+
+    @Override
+    public CompletableFuture<Void> write(List<SubpartitionNettyPayload> 
toWriteBuffers) {
+        List<NettyPayload> buffersToSpill =
+                toWriteBuffers.stream()
+                        .map(SubpartitionNettyPayload::getSegmentNettyPayloads)
+                        .flatMap(
+                                (Function<List<SegmentNettyPayload>, 
Stream<SegmentNettyPayload>>)
+                                        Collection::stream)
+                        .map(SegmentNettyPayload::getNettyPayloads)
+                        .flatMap(
+                                (Function<List<NettyPayload>, 
Stream<NettyPayload>>)
+                                        Collection::stream)
+                        .collect(Collectors.toList());

Review Comment:
   Then why do we need the complicated `SubpartitionNettyPayload` and 
`SegmentNettyPayload` at the first place?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskCacheManager.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.tiered.tier.disk;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SegmentNettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SubpartitionNettyPayload;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link DiskCacheManager} is responsible for managing cached buffers 
before flushing to files.
+ */
+public class DiskCacheManager {
+
+    private final int numSubpartitions;
+
+    private final PartitionFileWriter partitionFileWriter;
+
+    private final SubpartitionDiskCacheManager[] subpartitionCacheManagers;
+
+    private CompletableFuture<Void> hasFlushCompleted;
+
+    public DiskCacheManager(
+            int numSubpartitions,
+            TieredStorageMemoryManager storageMemoryManager,
+            PartitionFileWriter partitionFileWriter) {
+        this.numSubpartitions = numSubpartitions;
+        this.partitionFileWriter = partitionFileWriter;
+        this.subpartitionCacheManagers = new 
SubpartitionDiskCacheManager[numSubpartitions];
+        this.hasFlushCompleted = FutureUtils.completedVoidFuture();
+
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionCacheManagers[subpartitionId] =
+                    new SubpartitionDiskCacheManager(subpartitionId);
+        }
+        
storageMemoryManager.listenBufferReclaimRequest(this::notifyFlushCachedBuffers);
+    }
+
+    /**
+     * Append the end-of-segment event to {@link DiskCacheManager}.
+     *
+     * @param record the end-of-segment event
+     * @param subpartitionId target subpartition of this record.
+     * @param dataType the type of this record. In other words, is it data or 
event.
+     */
+    public void appendEndOfSegmentEvent(
+            ByteBuffer record, int subpartitionId, Buffer.DataType dataType) {
+        
subpartitionCacheManagers[subpartitionId].appendEndOfSegmentEvent(record, 
dataType);
+        flushAndReleaseCacheBuffers();
+    }
+
+    /**
+     * Append buffer to {@link DiskCacheManager}.
+     *
+     * @param buffer to be managed by this class.
+     * @param subpartitionId the subpartition of this record.
+     */
+    public void append(Buffer buffer, int subpartitionId) {
+        subpartitionCacheManagers[subpartitionId].append(buffer);
+    }
+
+    /**
+     * Return the finished buffer index.
+     *
+     * @param subpartitionId the target subpartition id
+     * @return the finished buffer index
+     */
+    public int getFinishedBufferIndex(int subpartitionId) {
+        return 
subpartitionCacheManagers[subpartitionId].getFinishedBufferIndex();
+    }
+
+    /** Close this {@link DiskCacheManager}, it means no data can append to 
memory. */
+    public void close() {
+        flushAndReleaseCacheBuffers();
+    }
+
+    /**
+     * Release this {@link DiskCacheManager}, it means all memory taken by 
this class will recycle.
+     */
+    public void release() {
+        
Arrays.stream(subpartitionCacheManagers).forEach(SubpartitionDiskCacheManager::release);
+        partitionFileWriter.release();
+    }
+
+    // ------------------------------------
+    //           Internal Method
+    // ------------------------------------
+
+    private void notifyFlushCachedBuffers() {
+        flushBuffers(false);
+    }
+
+    private void flushAndReleaseCacheBuffers() {
+        flushBuffers(true);
+    }
+
+    /**
+     * Note that the request of flushing buffers may come from the disk check 
thread or the task
+     * thread, so the method itself should ensure the thread safety.
+     */
+    private synchronized void flushBuffers(boolean needForceFlush) {
+        if (!needForceFlush && !hasFlushCompleted.isDone()) {
+            return;
+        }
+        List<SubpartitionNettyPayload> toWriteBuffers = new ArrayList<>();
+        int numToWriteBuffers = getSubpartitionBuffersToFlush(toWriteBuffers);
+
+        if (numToWriteBuffers > 0) {
+            CompletableFuture<Void> flushCompletableFuture =
+                    partitionFileWriter.write(toWriteBuffers);
+            if (!needForceFlush) {
+                hasFlushCompleted = flushCompletableFuture;
+            }
+        }
+    }
+
+    private int getSubpartitionBuffersToFlush(List<SubpartitionNettyPayload> 
toWriteBuffers) {
+        int numToWriteBuffers = 0;
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
subpartitionId++) {
+            List<NettyPayload> nettyPayloads =
+                    subpartitionCacheManagers[subpartitionId].getAllBuffers();
+            toWriteBuffers.add(
+                    new SubpartitionNettyPayload(
+                            subpartitionId,
+                            Collections.singletonList(
+                                    new SegmentNettyPayload(-1, nettyPayloads, 
false))));

Review Comment:
   Why is the hardcoded `-1` segment id and `false` for finishing segment?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileIndex.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The {@link PartitionFileIndex} represents the indexes and the regions of 
the spilled buffers. For
+ * each spilled data buffer, this maintains the subpartition it belongs to, 
the buffer index within
+ * the subpartition, the offset in file it begin with. The {@link Region} 
represents a series of
+ * physically continuous buffers in the file, which are from the same 
subpartition.
+ */
+public interface PartitionFileIndex {
+
+    /**
+     * When writing the buffers to the partition file, the {@link 
PartitionFileIndex} should record
+     * the {@link Region} info of these spilled buffers.
+     *
+     * @param spilledBuffers the buffers to be spilled
+     */
+    void addRegionForBuffers(List<SpilledBuffer> spilledBuffers);
+
+    /**
+     * When reading partition file, use the method to get the next {@link 
Region} for the reader.
+     *
+     * @param subpartitionId the subpartition id
+     * @param nettyServiceWriterId the netty connection which reads the 
buffers from the partition
+     *     file, then writes the buffers to the netty.
+     */
+    Optional<Region> getNextRegion(int subpartitionId, NettyConnectionId 
nettyServiceWriterId);

Review Comment:
   Why is there a netty connection id?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskCacheManager.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.tiered.tier.disk;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SegmentNettyPayload;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.file.SubpartitionNettyPayload;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link DiskCacheManager} is responsible for managing cached buffers 
before flushing to files.
+ */
+public class DiskCacheManager {
+
+    private final int numSubpartitions;
+
+    private final PartitionFileWriter partitionFileWriter;
+
+    private final SubpartitionDiskCacheManager[] subpartitionCacheManagers;
+
+    private CompletableFuture<Void> hasFlushCompleted;
+
+    public DiskCacheManager(
+            int numSubpartitions,
+            TieredStorageMemoryManager storageMemoryManager,
+            PartitionFileWriter partitionFileWriter) {
+        this.numSubpartitions = numSubpartitions;
+        this.partitionFileWriter = partitionFileWriter;
+        this.subpartitionCacheManagers = new 
SubpartitionDiskCacheManager[numSubpartitions];
+        this.hasFlushCompleted = FutureUtils.completedVoidFuture();
+
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionCacheManagers[subpartitionId] =
+                    new SubpartitionDiskCacheManager(subpartitionId);
+        }
+        
storageMemoryManager.listenBufferReclaimRequest(this::notifyFlushCachedBuffers);
+    }
+
+    /**
+     * Append the end-of-segment event to {@link DiskCacheManager}.
+     *
+     * @param record the end-of-segment event
+     * @param subpartitionId target subpartition of this record.
+     * @param dataType the type of this record. In other words, is it data or 
event.
+     */
+    public void appendEndOfSegmentEvent(
+            ByteBuffer record, int subpartitionId, Buffer.DataType dataType) {
+        
subpartitionCacheManagers[subpartitionId].appendEndOfSegmentEvent(record, 
dataType);
+        flushAndReleaseCacheBuffers();
+    }
+
+    /**
+     * Append buffer to {@link DiskCacheManager}.
+     *
+     * @param buffer to be managed by this class.
+     * @param subpartitionId the subpartition of this record.
+     */
+    public void append(Buffer buffer, int subpartitionId) {
+        subpartitionCacheManagers[subpartitionId].append(buffer);
+    }
+
+    /**
+     * Return the finished buffer index.
+     *
+     * @param subpartitionId the target subpartition id
+     * @return the finished buffer index
+     */
+    public int getFinishedBufferIndex(int subpartitionId) {
+        return 
subpartitionCacheManagers[subpartitionId].getFinishedBufferIndex();

Review Comment:
   Which buffer is the finished buffer?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileIndex.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The {@link PartitionFileIndex} represents the indexes and the regions of 
the spilled buffers. For
+ * each spilled data buffer, this maintains the subpartition it belongs to, 
the buffer index within
+ * the subpartition, the offset in file it begin with. The {@link Region} 
represents a series of
+ * physically continuous buffers in the file, which are from the same 
subpartition.
+ */
+public interface PartitionFileIndex {
+
+    /**
+     * When writing the buffers to the partition file, the {@link 
PartitionFileIndex} should record
+     * the {@link Region} info of these spilled buffers.
+     *
+     * @param spilledBuffers the buffers to be spilled
+     */
+    void addRegionForBuffers(List<SpilledBuffer> spilledBuffers);

Review Comment:
   The contract here is ambiguous. What does this method do? What are the 
relation ship between regions and the provided buffers?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileIndexImpl.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The default implementation of {@link PartitionFileIndex}. */
+public class PartitionFileIndexImpl implements PartitionFileIndex {
+
+    /**
+     * The regions belonging to each subpartitions.
+     *
+     * <p>Note that the field can be accessed by the writing and reading IO 
thread, so the lock is
+     * to ensure the thread safety.
+     */
+    @GuardedBy("lock")
+    private final List<List<Region>> subpartitionRegions;
+
+    /**
+     * The region index of a reader is reading for each subpartition. The list 
index is
+     * corresponding to the subpartition id. The key in the map represents the 
reader, the value in
+     * the map represents the reading region index.
+     */
+    @GuardedBy("lock")
+    private final List<Map<NettyConnectionId, Integer>> 
subpartitionReaderRegionIndexes;
+
+    @GuardedBy("lock")
+    private boolean isReleased;
+
+    private final Object lock = new Object();
+
+    public PartitionFileIndexImpl(int numSubpartitions) {
+        this.subpartitionRegions = new ArrayList<>();
+        this.subpartitionReaderRegionIndexes = new ArrayList<>();
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
++subpartitionId) {
+            subpartitionRegions.add(new ArrayList<>());
+            subpartitionReaderRegionIndexes.add(new HashMap<>());
+        }
+    }
+
+    @Override
+    public Optional<Region> getNextRegion(
+            int subpartitionId, NettyConnectionId nettyServiceWriterId) {
+        synchronized (lock) {
+            if (isReleased) {
+                return Optional.empty();
+            }
+
+            int currentRegionIndex =
+                    subpartitionReaderRegionIndexes
+                            .get(subpartitionId)
+                            .getOrDefault(nettyServiceWriterId, 0);
+            List<Region> currentRegions = 
subpartitionRegions.get(subpartitionId);
+            if (currentRegionIndex < currentRegions.size()) {
+                Region region = currentRegions.get(currentRegionIndex);
+                ++currentRegionIndex;
+                subpartitionReaderRegionIndexes
+                        .get(subpartitionId)
+                        .put(nettyServiceWriterId, currentRegionIndex);
+                return Optional.of(region);
+            }
+            return Optional.empty();
+        }
+    }
+
+    @Override
+    public void addRegionForBuffers(List<SpilledBuffer> spilledBuffers) {
+        if (spilledBuffers.isEmpty()) {
+            return;
+        }
+
+        Map<Integer, List<Region>> convertedRegions = 
convertToRegions(spilledBuffers);
+        synchronized (lock) {
+            convertedRegions.forEach(
+                    (subpartition, regions) ->
+                            
subpartitionRegions.get(subpartition).addAll(regions));
+        }
+    }
+
+    @Override
+    public void release() {
+        synchronized (lock) {
+            subpartitionRegions.clear();
+            subpartitionReaderRegionIndexes.clear();
+            isReleased = true;
+        }
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private static Map<Integer, List<Region>> 
convertToRegions(List<SpilledBuffer> spilledBuffers) {
+        Map<Integer, List<Region>> subpartitionRegionMap = new HashMap<>();
+        Iterator<SpilledBuffer> iterator = spilledBuffers.iterator();
+        SpilledBuffer firstBufferInRegion = iterator.next();
+        SpilledBuffer lastBufferInRegion = firstBufferInRegion;
+
+        while (iterator.hasNext()) {
+            SpilledBuffer currentBuffer = iterator.next();
+            if (currentBuffer.getSubpartitionId() != 
firstBufferInRegion.getSubpartitionId()
+                    || currentBuffer.getBufferIndex() != 
lastBufferInRegion.getBufferIndex() + 1) {
+                // the current buffer belongs to a new region, close the 
previous region
+                addInternalRegionToMap(
+                        firstBufferInRegion, lastBufferInRegion, 
subpartitionRegionMap);

Review Comment:
   It seems we require `spilledBuffers` to be continuous, which is never 
mentioned in the contract.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/SubpartitionDiskCacheManager.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.tiered.tier.disk;
+
+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.Buffer.DataType;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyPayload;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The {@link SubpartitionDiskCacheManager} is responsible to manage the 
cached buffers in a single
+ * subpartition.
+ */
+public class SubpartitionDiskCacheManager {
+
+    private final int subpartitionId;
+
+    private int finishedBufferIndex;
+
+    // Note that this field can be accessed by the task thread or the write IO 
thread, so the thread
+    // safety should be ensured.
+    private final Deque<NettyPayload> allBuffers = new LinkedList<>();
+
+    public SubpartitionDiskCacheManager(int subpartitionId) {
+        this.subpartitionId = subpartitionId;
+    }
+
+    // ------------------------------------------------------------------------
+    //  Called by DiskCacheManager
+    // ------------------------------------------------------------------------
+    void appendEndOfSegmentEvent(ByteBuffer record, DataType dataType) {
+        writeEvent(record, dataType);
+    }
+
+    int getFinishedBufferIndex() {
+        return finishedBufferIndex;
+    }
+
+    void append(Buffer buffer) {
+        NettyPayload toAddBuffer =
+                NettyPayload.newBuffer(buffer, finishedBufferIndex, 
subpartitionId);
+        addFinishedBuffer(toAddBuffer);
+    }
+
+    // Note that allBuffers can be touched by multiple threads.
+    List<NettyPayload> getAllBuffers() {
+        synchronized (allBuffers) {
+            List<NettyPayload> targetBuffers = new ArrayList<>(allBuffers);
+            allBuffers.clear();

Review Comment:
   Neither the name nor any javadoc has suggested that the buffers will be 
cleared.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileIndex.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The {@link PartitionFileIndex} represents the indexes and the regions of 
the spilled buffers. For
+ * each spilled data buffer, this maintains the subpartition it belongs to, 
the buffer index within
+ * the subpartition, the offset in file it begin with. The {@link Region} 
represents a series of
+ * physically continuous buffers in the file, which are from the same 
subpartition.
+ */
+public interface PartitionFileIndex {

Review Comment:
   Which component is this interface designed to be used by? What different 
implementations can it possibly have?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/file/PartitionFileIndex.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.tiered.storage.file;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The {@link PartitionFileIndex} represents the indexes and the regions of 
the spilled buffers. For
+ * each spilled data buffer, this maintains the subpartition it belongs to, 
the buffer index within
+ * the subpartition, the offset in file it begin with. The {@link Region} 
represents a series of
+ * physically continuous buffers in the file, which are from the same 
subpartition.
+ */
+public interface PartitionFileIndex {
+
+    /**
+     * When writing the buffers to the partition file, the {@link 
PartitionFileIndex} should record
+     * the {@link Region} info of these spilled buffers.
+     *
+     * @param spilledBuffers the buffers to be spilled
+     */
+    void addRegionForBuffers(List<SpilledBuffer> spilledBuffers);
+
+    /**
+     * When reading partition file, use the method to get the next {@link 
Region} for the reader.
+     *
+     * @param subpartitionId the subpartition id
+     * @param nettyServiceWriterId the netty connection which reads the 
buffers from the partition
+     *     file, then writes the buffers to the netty.
+     */
+    Optional<Region> getNextRegion(int subpartitionId, NettyConnectionId 
nettyServiceWriterId);

Review Comment:
   Who's calling this? Why would the index remembers which region it has read 
previously?



-- 
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