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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileWriter.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link PartitionFileWriter} interface defines the write logic for 
different types of shuffle
+ * files.
+ */
+public interface PartitionFileWriter {
+
+    /**
+     * Write the {@link SpilledBufferContext}s to the partition file. The 
written buffers may belong
+     * to multiple subpartitions.
+     *
+     * @return the completable future indicating whether the writing file 
process has finished. If
+     *     the {@link CompletableFuture} is completed, the written process is 
completed.
+     */
+    CompletableFuture<Void> write(List<SubpartitionSpilledBufferContext> 
spilledBuffers);

Review Comment:
   Should explicitly mention that `spilledBuffers` should be consecutive in the 
file.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SpilledBufferContext.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The buffer context to be flushed, including the {@link Buffer}, the buffer 
index, the
+ * subpartition id, the segment id, etc.
+ */
+public class SpilledBufferContext {
+
+    /** The data buffer. Note that the buffer should not be null. */
+    private final Buffer buffer;
+
+    /** The index of buffer. */
+    private final int bufferIndex;
+
+    /** The id of subpartition. */
+    private final int subpartitionId;
+
+    /** The id of segment. */
+    private final int segmentId;

Review Comment:
   Why do we need these when we already have `SubpartitionSpilledBufferContext` 
and `SegmentSpilledBufferContext`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileReader.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+
+import java.io.IOException;
+
+/** {@link PartitionFileReader} defines the read logic for different types of 
shuffle files. */
+public interface PartitionFileReader {
+
+    /**
+     * Read a buffer from partition file.
+     *
+     * @param partitionId partition id indicates the id of partition.
+     * @param subpartitionId subpartition id indicates the id of subpartition.
+     * @param fileOffset file offset indicates the current reading offset.
+     * @param memorySegment memory segment indicates a segment of available 
buffer.
+     * @param recycler recycler indicates the owner of the buffer.
+     * @return the buffer.
+     */
+    Buffer readBuffer(
+            TieredStoragePartitionId partitionId,
+            TieredStorageSubpartitionId subpartitionId,
+            int segmentId,
+            long fileOffset,
+            MemorySegment memorySegment,
+            BufferRecycler recycler)
+            throws IOException;

Review Comment:
   There's 6 parameters, only 5 appear in the JavaDoc



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileIndex.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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 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 {@link PartitionFileIndex} represents the indexes and the regions of 
the spilled buffers. The
+ * region indexes are generated when writing the spilled buffers, and these 
region indexes are used
+ * when reading data from disk.

Review Comment:
   This is hard to understand. What is a region? Why maintaining the indexes? 
What does this class do?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileWriter.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link PartitionFileWriter} interface defines the write logic for 
different types of shuffle
+ * files.
+ */
+public interface PartitionFileWriter {
+
+    /**
+     * Write the {@link SpilledBufferContext}s to the partition file. The 
written buffers may belong
+     * to multiple subpartitions.
+     *
+     * @return the completable future indicating whether the writing file 
process has finished. If
+     *     the {@link CompletableFuture} is completed, the written process is 
completed.
+     */
+    CompletableFuture<Void> write(List<SubpartitionSpilledBufferContext> 
spilledBuffers);

Review Comment:
   The name `spilledBuffers` is also confusing.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileReader.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+
+import java.io.IOException;
+
+/** {@link PartitionFileReader} defines the read logic for different types of 
shuffle files. */
+public interface PartitionFileReader {
+
+    /**
+     * Read a buffer from partition file.
+     *
+     * @param partitionId partition id indicates the id of partition.
+     * @param subpartitionId subpartition id indicates the id of subpartition.
+     * @param fileOffset file offset indicates the current reading offset.
+     * @param memorySegment memory segment indicates a segment of available 
buffer.
+     * @param recycler recycler indicates the owner of the buffer.

Review Comment:
   These descriptions are hard to understand.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileWriter.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The {@link PartitionFileWriter} interface defines the write logic for 
different types of shuffle
+ * files.
+ */
+public interface PartitionFileWriter {
+
+    /**
+     * Write the {@link SpilledBufferContext}s to the partition file. The 
written buffers may belong
+     * to multiple subpartitions.
+     *
+     * @return the completable future indicating whether the writing file 
process has finished. If
+     *     the {@link CompletableFuture} is completed, the written process is 
completed.
+     */
+    CompletableFuture<Void> write(List<SubpartitionSpilledBufferContext> 
spilledBuffers);
+
+    /** Release all the resources of the {@link PartitionFileWriter}. */
+    void release();
+
+    /**
+     * The {@link SubpartitionSpilledBufferContext} contains all the buffers 
that will be spilled in
+     * this subpartition.
+     */
+    class SubpartitionSpilledBufferContext {
+
+        /** The subpartition id. */
+        private final int subpartitionId;
+
+        /** he {@link SegmentSpilledBufferContext}s belonging to this 
subpartition. */
+        private final List<SegmentSpilledBufferContext> 
segmentSpilledBufferContexts;
+
+        public SubpartitionSpilledBufferContext(
+                int subpartitionId,
+                List<SegmentSpilledBufferContext> 
segmentSpilledBufferContexts) {
+            this.subpartitionId = subpartitionId;
+            this.segmentSpilledBufferContexts = segmentSpilledBufferContexts;
+        }
+
+        public int getSubpartitionId() {
+            return subpartitionId;
+        }
+
+        public List<SegmentSpilledBufferContext> 
getSegmentSpillBufferContexts() {
+            return segmentSpilledBufferContexts;
+        }
+    }
+
+    /**
+     * The wrapper class {@link SegmentSpilledBufferContext} contains all the 
{@link
+     * SpilledBufferContext}s of the segment. Note that when this indicates 
the segment need to be
+     * finished, the field {@code spilledBufferContexts} should be empty.
+     */
+    class SegmentSpilledBufferContext {
+
+        /** The segment id. */
+        private final int segmentId;
+
+        /** The {@link SpilledBufferContext}s indicate the buffers to be 
spilled. */
+        private final List<SpilledBufferContext> spilledBufferContexts;
+
+        /** Whether it is necessary to finish the segment. */
+        private final boolean needFinishSegment;

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



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SpilledBufferContext.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.runtime.io.network.buffer.Buffer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The buffer context to be flushed, including the {@link Buffer}, the buffer 
index, the
+ * subpartition id, the segment id, etc.
+ */
+public class SpilledBufferContext {

Review Comment:
   Why this is in a separate file, while `SubpartitionSpilledBufferContext` and 
`SegmentSpilledBufferContext` are inner to `PartitionFileWriter`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/PartitionFileReader.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+
+import java.io.IOException;
+
+/** {@link PartitionFileReader} defines the read logic for different types of 
shuffle files. */
+public interface PartitionFileReader {
+
+    /**
+     * Read a buffer from partition file.
+     *
+     * @param partitionId partition id indicates the id of partition.
+     * @param subpartitionId subpartition id indicates the id of subpartition.
+     * @param fileOffset file offset indicates the current reading offset.

Review Comment:
   Why do we need a file offset? Shouldn't it be a buffer index?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/SubpartitionDiskCacheManager.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+class SubpartitionDiskCacheManager {
+
+    /** The segment id of the {@link SubpartitionDiskCacheManager}. */
+    private final int subpartitionId;
+
+    // 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<>();

Review Comment:
   Why `NettyPayload`?



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