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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFileWriter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.file;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import 
org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+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 static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.createSubpartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getSegmentPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.writeSegmentFinishFile;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation of {@link PartitionFileWriter} with hash mode. In this 
mode, each segment of
+ * one subpartition is written to an independent file.
+ *
+ * <p>Note that after finishing writing a segment, a segment-finish file is 
written to ensure the
+ * downstream reads only when the entire segment file is written, avoiding 
partial data reads. The
+ * downstream can determine if the current segment is complete by checking for 
the existence of the
+ * segment-finish file.

Review Comment:
   This doubles the number of files on the remote file system. I think there 
are better ways to achieve this.
   * For object storages, an object becomes visible only when it's fully 
written. All the objects the reader can see via a list operation should be 
considered finished.
   * For appendable storages, it's easy to append a flag at the end of the 
segment-file to indicate the end of the segment. That means the reader can only 
know whether the segment is finished after reading it to the end. If it's not 
finished, the next read can continue from the position where the reader has 
already read to.
   * Alternatively, for each subpartition we can only keep a segment-finish 
file for the latest segment, and all segments whose id is smaller than it 
should be considered finished. That means removing the old segment-finish file 
after adding the new one.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFile.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.file;
+
+/**
+ * The partition file in the hash mode. In this mode, each segment of one 
subpartition is written to
+ * an independent file.
+ */
+public class HashPartitionFile {

Review Comment:
   I'd suggest the name `SegmentPartitionFile`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFileWriter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.file;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import 
org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+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 static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.createSubpartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getSegmentPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.writeSegmentFinishFile;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation of {@link PartitionFileWriter} with hash mode. In this 
mode, each segment of
+ * one subpartition is written to an independent file.
+ *
+ * <p>Note that after finishing writing a segment, a segment-finish file is 
written to ensure the
+ * downstream reads only when the entire segment file is written, avoiding 
partial data reads. The
+ * downstream can determine if the current segment is complete by checking for 
the existence of the
+ * segment-finish file.
+ */
+public class HashPartitionFileWriter implements PartitionFileWriter {
+
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadExecutor(
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("Hash partition file flush thread")
+                            
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());
+
+    private final String basePath;
+
+    private final WritableByteChannel[] subpartitionChannels;
+
+    private volatile boolean isReleased;
+
+    public HashPartitionFileWriter(String basePath, int numSubpartitions) {
+        this.basePath = basePath;
+        this.subpartitionChannels = new WritableByteChannel[numSubpartitions];
+        Arrays.fill(subpartitionChannels, null);
+    }
+
+    @Override
+    public CompletableFuture<Void> write(
+            TieredStoragePartitionId partitionId, 
List<SubpartitionBufferContext> buffersToWrite) {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        buffersToWrite.forEach(
+                subpartitionBuffers -> {
+                    int subpartitionId = 
subpartitionBuffers.getSubpartitionId();
+                    List<SegmentBufferContext> multiSegmentBuffers =
+                            subpartitionBuffers.getSegmentBufferContexts();
+                    multiSegmentBuffers.forEach(
+                            segmentBuffers -> {
+                                CompletableFuture<Void> flushSuccessNotifier =
+                                        new CompletableFuture<>();
+                                Runnable writeRunnable =
+                                        getFlushOrFinisheSegmentRunnable(

Review Comment:
   ```suggestion
                                           getFlushOrFinishSegmentRunnable(
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/TierFactory.java:
##########
@@ -32,7 +32,9 @@
 public interface TierFactory {
 
     /** Creates the master-side agent of a Tier. */
-    TierMasterAgent createMasterAgent(TieredStorageResourceRegistry 
tieredStorageResourceRegistry);
+    TierMasterAgent createMasterAgent(
+            String remoteStorageBaseHomePath,

Review Comment:
   This might be an argument of `RemoteTierFactory` constructor.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/SubpartitionRemoteCacheManager.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import net.jcip.annotations.GuardedBy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This {@link SubpartitionRemoteCacheManager} is responsible for managing the 
buffers in a single
+ * subpartition.
+ */
+class SubpartitionRemoteCacheManager {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SubpartitionRemoteCacheManager.class);
+
+    private final TieredStoragePartitionId partitionId;
+
+    private final int subpartitionId;
+
+    private final PartitionFileWriter partitionFileWriter;
+
+    /**
+     * All the buffers. The first field of the tuple is the buffer, while the 
second field of the
+     * buffer is the buffer index.
+     *
+     * <p>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<Tuple2<Buffer, Integer>> allBuffers = new 
LinkedList<>();
+
+    private CompletableFuture<Void> flushCompletableFuture = 
FutureUtils.completedVoidFuture();
+
+    /**
+     * Record the segment id that is writing to.
+     *
+     * <p>Note that when flushing buffers, this can be touched by task thread 
or the flushing
+     * thread, so the thread safety should be ensured.
+     */
+    @GuardedBy("allBuffers")
+    private int segmentId = -1;
+
+    /**
+     * Record the buffer index in the {@link SubpartitionRemoteCacheManager}. 
Each time a new buffer
+     * is added to the {@code allBuffers}, this field is increased by one.
+     *
+     * <p>Note that the field can only be touched by the task thread, so this 
field need not be
+     * guarded by any lock or synchronizations.
+     */
+    private int bufferIndex;
+
+    public SubpartitionRemoteCacheManager(
+            TieredStoragePartitionId partitionId,
+            int subpartitionId,
+            TieredStorageMemoryManager storageMemoryManager,
+            PartitionFileWriter partitionFileWriter) {
+        this.partitionId = partitionId;
+        this.subpartitionId = subpartitionId;
+        this.partitionFileWriter = partitionFileWriter;
+        storageMemoryManager.listenBufferReclaimRequest(this::flushBuffers);
+    }
+
+    // ------------------------------------------------------------------------
+    //  Called by RemoteCacheManager
+    // ------------------------------------------------------------------------
+
+    void startSegment(int segmentId) {
+        synchronized (allBuffers) {
+            this.segmentId = segmentId;
+        }
+    }
+
+    void addBuffer(Buffer buffer) {
+        Tuple2<Buffer, Integer> toAddBuffer = new Tuple2<>(buffer, 
bufferIndex++);
+        synchronized (allBuffers) {
+            allBuffers.add(toAddBuffer);
+        }
+    }
+
+    void finishSegment(int segmentId) {
+        synchronized (allBuffers) {
+            checkState(this.segmentId == segmentId, "Wrong segment id.");
+        }
+        int numFlushedBuffers = flushBuffers();
+        if (numFlushedBuffers > 0) {
+            PartitionFileWriter.SubpartitionBufferContext bufferContext =
+                    new PartitionFileWriter.SubpartitionBufferContext(
+                            subpartitionId,
+                            Collections.singletonList(
+                                    new 
PartitionFileWriter.SegmentBufferContext(
+                                            segmentId, 
Collections.emptyList(), true)));
+            // Notify the partition file writer that the segment is finished 
through writing the
+            // buffer context
+            flushCompletableFuture =
+                    partitionFileWriter.write(
+                            partitionId, 
Collections.singletonList(bufferContext));
+        }

Review Comment:
   This doesn't seem right. `numFlushedBuffers == 0` doesn't mean there's no 
data in the segment. It is possible that the buffers are flushed before calling 
`finishSegment`. E.g., on buffer reclaim request.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFileWriter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.file;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import 
org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+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 static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.createSubpartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getSegmentPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.writeSegmentFinishFile;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation of {@link PartitionFileWriter} with hash mode. In this 
mode, each segment of
+ * one subpartition is written to an independent file.
+ *
+ * <p>Note that after finishing writing a segment, a segment-finish file is 
written to ensure the
+ * downstream reads only when the entire segment file is written, avoiding 
partial data reads. The
+ * downstream can determine if the current segment is complete by checking for 
the existence of the
+ * segment-finish file.
+ */
+public class HashPartitionFileWriter implements PartitionFileWriter {
+
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadExecutor(
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("Hash partition file flush thread")
+                            
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());
+
+    private final String basePath;
+
+    private final WritableByteChannel[] subpartitionChannels;
+
+    private volatile boolean isReleased;
+
+    public HashPartitionFileWriter(String basePath, int numSubpartitions) {
+        this.basePath = basePath;
+        this.subpartitionChannels = new WritableByteChannel[numSubpartitions];
+        Arrays.fill(subpartitionChannels, null);
+    }
+
+    @Override
+    public CompletableFuture<Void> write(
+            TieredStoragePartitionId partitionId, 
List<SubpartitionBufferContext> buffersToWrite) {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        buffersToWrite.forEach(
+                subpartitionBuffers -> {
+                    int subpartitionId = 
subpartitionBuffers.getSubpartitionId();
+                    List<SegmentBufferContext> multiSegmentBuffers =
+                            subpartitionBuffers.getSegmentBufferContexts();
+                    multiSegmentBuffers.forEach(
+                            segmentBuffers -> {
+                                CompletableFuture<Void> flushSuccessNotifier =
+                                        new CompletableFuture<>();
+                                Runnable writeRunnable =
+                                        getFlushOrFinisheSegmentRunnable(
+                                                partitionId,
+                                                subpartitionId,
+                                                segmentBuffers,
+                                                flushSuccessNotifier);
+                                ioExecutor.execute(writeRunnable);

Review Comment:
   Why getting a `Runnable` here? Can we simply have a 
`flushOrFinishSegmentRunnable` method and call it with `ioExecutor.execute(() 
-> flushOrFinishSegmentRunnable(arguments))`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageUtils.java:
##########
@@ -49,4 +60,61 @@ public static void setBufferWithHeader(
         bufferWithHeaders[index] = header;
         bufferWithHeaders[index + 1] = buffer.getNioBufferReadable();
     }
+
+    public static void writeBuffers(
+            WritableByteChannel writeChannel, long expectedBytes, ByteBuffer[] 
bufferWithHeaders)
+            throws IOException {
+        int writeSize = 0;
+        for (ByteBuffer bufferWithHeader : bufferWithHeaders) {
+            writeSize += writeChannel.write(bufferWithHeader);
+        }
+        checkState(writeSize == expectedBytes);
+    }
+
+    public static String getSubpartitionPath(
+            String basePath, TieredStoragePartitionId partitionId, int 
subpartitionId) {
+        while (basePath.endsWith("/") && basePath.length() > 1) {
+            basePath = basePath.substring(0, basePath.length() - 1);
+        }
+        return String.format(
+                "%s/%s/%s",
+                basePath, TieredStorageIdMappingUtils.convertId(partitionId), 
subpartitionId);
+    }
+
+    public static Path getSegmentPath(

Review Comment:
   This, and many other fields and methods in this class, is quite specific to 
the hash partition file. IIUC, the purpose is to reuse them between the reader 
and writer. In that case, we can put them in `HashPartitionFile`.
   
   Ideally, we should have:
   * Constants and methods that needed by both hash and producer-merged 
partition files should be placed in `TieredStorageUtils`. Or maybe rename it as 
`TieredStorageFileUtils` if all the contents are related to partition files.
   * Constants and methods that are needed only by hash / producer-merged 
partition and are needed by both reader and writer should be placed in a hash / 
producer-merged specific class. This can be a dedicated 
`HashPartitionFileUtils` / `ProducerMergedPartitionFileUtils`, or simply use 
`HashPartitionFile` / `ProducerMergedPartitionFile`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierMasterAgent.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.core.fs.Path;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierMasterAgent;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.deletePathQuietly;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getPartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.removePartitionFiles;
+
+/** The implementation of {@link TierMasterAgent} for the remote tier. */
+public class RemoteTierMasterAgent implements TierMasterAgent {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(RemoteTierMasterAgent.class);
+
+    private final TieredStorageResourceRegistry resourceRegistry;
+
+    private final String remoteStorageBaseHomePath;
+
+    RemoteTierMasterAgent(
+            String remoteStorageBaseHomePath, TieredStorageResourceRegistry 
resourceRegistry) {
+        this.remoteStorageBaseHomePath = remoteStorageBaseHomePath;
+        this.resourceRegistry = resourceRegistry;
+    }
+
+    @Override
+    public void addPartition(TieredStoragePartitionId partitionID) {
+        resourceRegistry.registerResource(
+                partitionID,
+                () -> deletePathQuietly(getPartitionPath(partitionID, 
remoteStorageBaseHomePath)));
+    }
+
+    @Override
+    public void releasePartition(TieredStoragePartitionId partitionID) {
+        try {
+            removePartitionFiles(new Path(remoteStorageBaseHomePath), 
partitionID);
+        } catch (IOException e) {
+            LOG.error("Failed to release the partition file for {}", 
partitionID);
+        }
+
+        resourceRegistry.clearResourceFor(partitionID);
+    }
+
+    @Override
+    public void release(String toRelease) {
+        deletePathQuietly(toRelease);
+    }

Review Comment:
   Why adding a new interface? Shouldn't we use resource registry for releasing 
resources?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFileWriter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.file;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import 
org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+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 static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.createSubpartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getSegmentPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.writeSegmentFinishFile;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation of {@link PartitionFileWriter} with hash mode. In this 
mode, each segment of
+ * one subpartition is written to an independent file.
+ *
+ * <p>Note that after finishing writing a segment, a segment-finish file is 
written to ensure the
+ * downstream reads only when the entire segment file is written, avoiding 
partial data reads. The
+ * downstream can determine if the current segment is complete by checking for 
the existence of the
+ * segment-finish file.
+ */
+public class HashPartitionFileWriter implements PartitionFileWriter {
+
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadExecutor(
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("Hash partition file flush thread")
+                            
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());
+
+    private final String basePath;
+
+    private final WritableByteChannel[] subpartitionChannels;
+
+    private volatile boolean isReleased;
+
+    public HashPartitionFileWriter(String basePath, int numSubpartitions) {
+        this.basePath = basePath;
+        this.subpartitionChannels = new WritableByteChannel[numSubpartitions];
+        Arrays.fill(subpartitionChannels, null);
+    }
+
+    @Override
+    public CompletableFuture<Void> write(
+            TieredStoragePartitionId partitionId, 
List<SubpartitionBufferContext> buffersToWrite) {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        buffersToWrite.forEach(
+                subpartitionBuffers -> {
+                    int subpartitionId = 
subpartitionBuffers.getSubpartitionId();
+                    List<SegmentBufferContext> multiSegmentBuffers =
+                            subpartitionBuffers.getSegmentBufferContexts();
+                    multiSegmentBuffers.forEach(
+                            segmentBuffers -> {
+                                CompletableFuture<Void> flushSuccessNotifier =
+                                        new CompletableFuture<>();
+                                Runnable writeRunnable =
+                                        getFlushOrFinisheSegmentRunnable(
+                                                partitionId,
+                                                subpartitionId,
+                                                segmentBuffers,
+                                                flushSuccessNotifier);
+                                ioExecutor.execute(writeRunnable);
+                                completableFutures.add(flushSuccessNotifier);
+                            });
+                });
+        return FutureUtils.waitForAll(completableFutures);
+    }
+
+    @Override
+    public void release() {
+        if (isReleased) {
+            return;
+        }
+        isReleased = true;
+        try {
+            ioExecutor.shutdown();
+            if (!ioExecutor.awaitTermination(5L, TimeUnit.MINUTES)) {
+                throw new TimeoutException("Timeout to shutdown the flush 
thread.");
+            }
+            for (WritableByteChannel writeChannel : subpartitionChannels) {
+                if (writeChannel != null) {
+                    writeChannel.close();
+                }
+            }
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private Runnable getFlushOrFinisheSegmentRunnable(
+            TieredStoragePartitionId partitionId,
+            int subpartitionId,
+            SegmentBufferContext segmentBuffers,
+            CompletableFuture<Void> flushSuccessNotifier) {
+        int segmentId = segmentBuffers.getSegmentId();
+        List<Tuple2<Buffer, Integer>> buffersToFlush = 
segmentBuffers.getBufferAndIndexes();
+        boolean isFinishSegment = segmentBuffers.isSegmentFinished();
+        checkState(!buffersToFlush.isEmpty() || isFinishSegment);
+
+        return buffersToFlush.size() > 0
+                ? () ->
+                        flush(
+                                partitionId,
+                                subpartitionId,
+                                segmentId,
+                                buffersToFlush,
+                                flushSuccessNotifier)
+                : () ->
+                        writeFinishSegmentFile(
+                                partitionId, subpartitionId, segmentId, 
flushSuccessNotifier);

Review Comment:
   This doesn't feels right. A `SegmentBufferContext` may contains buffers and 
also finished.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierProducerAgent.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The implementation of {@link TierProducerAgent} for the remote tier. */
+public class RemoteTierProducerAgent implements TierProducerAgent {
+
+    private final int numSubpartitions;
+
+    private final int numBuffersPerSegment;
+
+    private final RemoteCacheManager cacheDataManager;
+
+    private final int[] currentSubpartitionWriteBuffers;

Review Comment:
   ```suggestion
       private final int[] currentSubpartitionSegmentWriteBuffers;
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierProducerAgent.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The implementation of {@link TierProducerAgent} for the remote tier. */
+public class RemoteTierProducerAgent implements TierProducerAgent {
+
+    private final int numSubpartitions;
+
+    private final int numBuffersPerSegment;
+
+    private final RemoteCacheManager cacheDataManager;
+
+    private final int[] currentSubpartitionWriteBuffers;
+
+    RemoteTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int numBytesPerSegment,
+            int bufferSizeBytes,
+            boolean isBroadcastOnly,
+            PartitionFileWriter partitionFileWriter,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(
+                numBytesPerSegment >= bufferSizeBytes,
+                "One segment should contain at least one buffer.");
+
+        this.numSubpartitions = numSubpartitions;
+        this.numBuffersPerSegment = numBytesPerSegment / bufferSizeBytes;
+        this.cacheDataManager =
+                new RemoteCacheManager(
+                        partitionId,
+                        isBroadcastOnly ? 1 : numSubpartitions,
+                        memoryManager,
+                        partitionFileWriter);
+        this.currentSubpartitionWriteBuffers = new int[numSubpartitions];
+        Arrays.fill(currentSubpartitionWriteBuffers, 0);
+        resourceRegistry.registerResource(partitionId, 
this::releaseAllResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        cacheDataManager.startSegment(subpartitionId.getSubpartitionId(), 
segmentId);
+        // The remote storage tier should always be able to start a new 
segment.
+        return true;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
buffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (currentSubpartitionWriteBuffers[subpartitionIndex] != 0
+                && currentSubpartitionWriteBuffers[subpartitionIndex] + 1 > 
numBuffersPerSegment) {
+            cacheDataManager.finishSegment(subpartitionIndex);
+            currentSubpartitionWriteBuffers[subpartitionIndex] = 0;
+            return false;
+        }
+        currentSubpartitionWriteBuffers[subpartitionIndex]++;
+        emitBuffer(buffer, subpartitionIndex);
+        return true;
+    }
+
+    @Override
+    public void close() {
+        for (int subpartitionId = 0; subpartitionId < numSubpartitions; 
subpartitionId++) {
+            cacheDataManager.finishSegment(subpartitionId);
+        }
+        cacheDataManager.close();
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private void releaseAllResources() {
+        cacheDataManager.release();
+    }
+
+    private void emitBuffer(Buffer finishedBuffer, int subpartitionId) {
+        cacheDataManager.appendBuffer(finishedBuffer, subpartitionId);
+    }

Review Comment:
   1. The name `emit` is confusing.
   2. Why do we need this method when its body is only one line of code?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/SubpartitionRemoteCacheManager.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import net.jcip.annotations.GuardedBy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This {@link SubpartitionRemoteCacheManager} is responsible for managing the 
buffers in a single
+ * subpartition.
+ */
+class SubpartitionRemoteCacheManager {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SubpartitionRemoteCacheManager.class);
+
+    private final TieredStoragePartitionId partitionId;
+
+    private final int subpartitionId;
+
+    private final PartitionFileWriter partitionFileWriter;
+
+    /**
+     * All the buffers. The first field of the tuple is the buffer, while the 
second field of the
+     * buffer is the buffer index.
+     *
+     * <p>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<Tuple2<Buffer, Integer>> allBuffers = new 
LinkedList<>();
+
+    private CompletableFuture<Void> flushCompletableFuture = 
FutureUtils.completedVoidFuture();
+
+    /**
+     * Record the segment id that is writing to.
+     *
+     * <p>Note that when flushing buffers, this can be touched by task thread 
or the flushing
+     * thread, so the thread safety should be ensured.
+     */
+    @GuardedBy("allBuffers")
+    private int segmentId = -1;
+
+    /**
+     * Record the buffer index in the {@link SubpartitionRemoteCacheManager}. 
Each time a new buffer
+     * is added to the {@code allBuffers}, this field is increased by one.
+     *
+     * <p>Note that the field can only be touched by the task thread, so this 
field need not be
+     * guarded by any lock or synchronizations.
+     */
+    private int bufferIndex;
+
+    public SubpartitionRemoteCacheManager(
+            TieredStoragePartitionId partitionId,
+            int subpartitionId,
+            TieredStorageMemoryManager storageMemoryManager,
+            PartitionFileWriter partitionFileWriter) {
+        this.partitionId = partitionId;
+        this.subpartitionId = subpartitionId;
+        this.partitionFileWriter = partitionFileWriter;
+        storageMemoryManager.listenBufferReclaimRequest(this::flushBuffers);
+    }
+
+    // ------------------------------------------------------------------------
+    //  Called by RemoteCacheManager
+    // ------------------------------------------------------------------------
+
+    void startSegment(int segmentId) {
+        synchronized (allBuffers) {
+            this.segmentId = segmentId;

Review Comment:
   Should check `allBuffers` is empty.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierProducerAgent.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.remote;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.file.PartitionFileWriter;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The implementation of {@link TierProducerAgent} for the remote tier. */
+public class RemoteTierProducerAgent implements TierProducerAgent {
+
+    private final int numSubpartitions;
+
+    private final int numBuffersPerSegment;
+
+    private final RemoteCacheManager cacheDataManager;
+
+    private final int[] currentSubpartitionWriteBuffers;
+
+    RemoteTierProducerAgent(
+            TieredStoragePartitionId partitionId,
+            int numSubpartitions,
+            int numBytesPerSegment,
+            int bufferSizeBytes,
+            boolean isBroadcastOnly,
+            PartitionFileWriter partitionFileWriter,
+            TieredStorageMemoryManager memoryManager,
+            TieredStorageResourceRegistry resourceRegistry) {
+        checkArgument(
+                numBytesPerSegment >= bufferSizeBytes,
+                "One segment should contain at least one buffer.");
+
+        this.numSubpartitions = numSubpartitions;
+        this.numBuffersPerSegment = numBytesPerSegment / bufferSizeBytes;
+        this.cacheDataManager =
+                new RemoteCacheManager(
+                        partitionId,
+                        isBroadcastOnly ? 1 : numSubpartitions,
+                        memoryManager,
+                        partitionFileWriter);
+        this.currentSubpartitionWriteBuffers = new int[numSubpartitions];
+        Arrays.fill(currentSubpartitionWriteBuffers, 0);
+        resourceRegistry.registerResource(partitionId, 
this::releaseAllResources);
+    }
+
+    @Override
+    public boolean tryStartNewSegment(TieredStorageSubpartitionId 
subpartitionId, int segmentId) {
+        cacheDataManager.startSegment(subpartitionId.getSubpartitionId(), 
segmentId);
+        // The remote storage tier should always be able to start a new 
segment.
+        return true;
+    }
+
+    @Override
+    public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer 
buffer) {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (currentSubpartitionWriteBuffers[subpartitionIndex] != 0
+                && currentSubpartitionWriteBuffers[subpartitionIndex] + 1 > 
numBuffersPerSegment) {

Review Comment:
   We already checked in the constructor that one segment at least contain one 
buffer. `currentSubpartitionWriteBuffers[subpartitionIndex] != 0` is probably 
not needed here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/HashPartitionFileWriter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.file;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+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.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import 
org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+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 static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.createSubpartitionPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.generateBufferWithHeaders;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.getSegmentPath;
+import static 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageUtils.writeSegmentFinishFile;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The implementation of {@link PartitionFileWriter} with hash mode. In this 
mode, each segment of
+ * one subpartition is written to an independent file.
+ *
+ * <p>Note that after finishing writing a segment, a segment-finish file is 
written to ensure the
+ * downstream reads only when the entire segment file is written, avoiding 
partial data reads. The
+ * downstream can determine if the current segment is complete by checking for 
the existence of the
+ * segment-finish file.
+ */
+public class HashPartitionFileWriter implements PartitionFileWriter {
+
+    private final ExecutorService ioExecutor =
+            Executors.newSingleThreadExecutor(
+                    new ThreadFactoryBuilder()
+                            .setNameFormat("Hash partition file flush thread")
+                            
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
+                            .build());
+
+    private final String basePath;
+
+    private final WritableByteChannel[] subpartitionChannels;
+
+    private volatile boolean isReleased;
+
+    public HashPartitionFileWriter(String basePath, int numSubpartitions) {
+        this.basePath = basePath;
+        this.subpartitionChannels = new WritableByteChannel[numSubpartitions];
+        Arrays.fill(subpartitionChannels, null);
+    }
+
+    @Override
+    public CompletableFuture<Void> write(
+            TieredStoragePartitionId partitionId, 
List<SubpartitionBufferContext> buffersToWrite) {
+        List<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+        buffersToWrite.forEach(
+                subpartitionBuffers -> {
+                    int subpartitionId = 
subpartitionBuffers.getSubpartitionId();
+                    List<SegmentBufferContext> multiSegmentBuffers =
+                            subpartitionBuffers.getSegmentBufferContexts();
+                    multiSegmentBuffers.forEach(
+                            segmentBuffers -> {
+                                CompletableFuture<Void> flushSuccessNotifier =
+                                        new CompletableFuture<>();
+                                Runnable writeRunnable =
+                                        getFlushOrFinisheSegmentRunnable(
+                                                partitionId,
+                                                subpartitionId,
+                                                segmentBuffers,
+                                                flushSuccessNotifier);
+                                ioExecutor.execute(writeRunnable);
+                                completableFutures.add(flushSuccessNotifier);
+                            });
+                });
+        return FutureUtils.waitForAll(completableFutures);
+    }
+
+    @Override
+    public void release() {
+        if (isReleased) {
+            return;
+        }
+        isReleased = true;
+        try {
+            ioExecutor.shutdown();
+            if (!ioExecutor.awaitTermination(5L, TimeUnit.MINUTES)) {
+                throw new TimeoutException("Timeout to shutdown the flush 
thread.");
+            }
+            for (WritableByteChannel writeChannel : subpartitionChannels) {
+                if (writeChannel != null) {
+                    writeChannel.close();
+                }
+            }
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+    }
+
+    // ------------------------------------------------------------------------
+    //  Internal Methods
+    // ------------------------------------------------------------------------
+
+    private Runnable getFlushOrFinisheSegmentRunnable(
+            TieredStoragePartitionId partitionId,
+            int subpartitionId,
+            SegmentBufferContext segmentBuffers,
+            CompletableFuture<Void> flushSuccessNotifier) {
+        int segmentId = segmentBuffers.getSegmentId();
+        List<Tuple2<Buffer, Integer>> buffersToFlush = 
segmentBuffers.getBufferAndIndexes();
+        boolean isFinishSegment = segmentBuffers.isSegmentFinished();
+        checkState(!buffersToFlush.isEmpty() || isFinishSegment);
+
+        return buffersToFlush.size() > 0
+                ? () ->
+                        flush(
+                                partitionId,
+                                subpartitionId,
+                                segmentId,
+                                buffersToFlush,
+                                flushSuccessNotifier)
+                : () ->
+                        writeFinishSegmentFile(
+                                partitionId, subpartitionId, segmentId, 
flushSuccessNotifier);

Review Comment:
   I just noticed that `DiskCacheManager#getSubpartitionToFlushBuffers` is not 
passing the correct value for `segmentFinished` when creating 
`SegmentBufferContext`.



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