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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/SubpartitionSegmentIdTracker.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+/**
+ * {@link SubpartitionSegmentIdTracker} is to track segment index for each 
subpartition. Each {@link
+ * TierProducerAgent}'s data manager has a separate {@link 
SubpartitionSegmentIdTracker}.
+ */
+public interface SubpartitionSegmentIdTracker {

Review Comment:
   I wonder whether it's necessary to separate the interface and implementation 
in this case.
   
   Usually, we need interface to:
   - Provide multiple implementations 
   - Restrict access to part of the public methods



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/SubpartitionSegmentIdTracker.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+/**
+ * {@link SubpartitionSegmentIdTracker} is to track segment index for each 
subpartition. Each {@link
+ * TierProducerAgent}'s data manager has a separate {@link 
SubpartitionSegmentIdTracker}.
+ */
+public interface SubpartitionSegmentIdTracker {
+
+    /**
+     * Add a new segment index belong to the subpartition.
+     *
+     * @param subpartitionId the subpartition index
+     * @param segmentId the segment index to be added
+     */
+    void addSegmentIndex(TieredStorageSubpartitionId subpartitionId, int 
segmentId);
+
+    /**
+     * Return whether the {@link SubpartitionSegmentIdTracker} contains the 
segment index of the
+     * specific subpartition.
+     *
+     * @param subpartitionId the subpartition index
+     * @param segmentId the segment index to be added
+     * @return whether the tracker contains the segment index of the 
subpartition
+     */
+    boolean hasCurrentSegment(TieredStorageSubpartitionId subpartitionId, int 
segmentId);

Review Comment:
   `hasSegment`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClient.java:
##########
@@ -119,7 +145,66 @@ private void writeAccumulatedBuffers(
     private void writeAccumulatedBuffer(
             TieredStorageSubpartitionId subpartitionId, Buffer 
accumulatedBuffer)
             throws IOException {
-        // TODO, Try to write the accumulated buffer to the appropriate tier. 
After the tier is
-        // decided, then write the accumulated buffer to the tier.
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        int tierIndex = chooseStorageTierIfNeeded(subpartitionId);
+
+        Buffer compressedBuffer = compressBufferIfPossible(accumulatedBuffer);
+        updateStatistics(compressedBuffer);
+        boolean isLastBufferInSegment =
+                tierProducerAgents.get(tierIndex).write(subpartitionId, 
compressedBuffer);
+        if (isLastBufferInSegment) {
+            isSubpartitionSegmentFinished[subpartitionIndex] = true;
+        }
+    }
+
+    private int chooseStorageTierIfNeeded(TieredStorageSubpartitionId 
subpartitionId)
+            throws IOException {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (isSubpartitionSegmentFinished[subpartitionIndex]) {
+            // If the current segment is finished, this subpartition need 
choose a new storage tier.
+            internalChooseStorageTier(subpartitionId);
+            isSubpartitionSegmentFinished[subpartitionIndex] = false;
+        }
+        return currentSubpartitionTierIndex[subpartitionIndex];
+    }
+
+    private void internalChooseStorageTier(TieredStorageSubpartitionId 
subpartitionId)
+            throws IOException {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        int segmentIndex = currentSubpartitionSegmentId[subpartitionIndex];
+        int nextSegmentIndex = segmentIndex + 1;
+
+        for (int tierIndex = 0; tierIndex < tierProducerAgents.size(); 
++tierIndex) {
+            if (tierProducerAgents
+                    .get(tierIndex)

Review Comment:
   This can be simplified if we change `int[] currentSubpartitionTierIndex` to 
`TierProducerAgent currentSubpartitionTier`. We won't need to loop over the 
integers then get it from the list.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/SubpartitionSegmentIdTracker.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+/**
+ * {@link SubpartitionSegmentIdTracker} is to track segment index for each 
subpartition. Each {@link
+ * TierProducerAgent}'s data manager has a separate {@link 
SubpartitionSegmentIdTracker}.
+ */
+public interface SubpartitionSegmentIdTracker {
+
+    /**
+     * Add a new segment index belong to the subpartition.
+     *
+     * @param subpartitionId the subpartition index
+     * @param segmentId the segment index to be added
+     */
+    void addSegmentIndex(TieredStorageSubpartitionId subpartitionId, int 
segmentId);

Review Comment:
   `addSegment`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/SubpartitionSegmentIdTracker.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+
+/**
+ * {@link SubpartitionSegmentIdTracker} is to track segment index for each 
subpartition. Each {@link
+ * TierProducerAgent}'s data manager has a separate {@link 
SubpartitionSegmentIdTracker}.

Review Comment:
   What is "data manager"?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClient.java:
##########
@@ -42,6 +45,17 @@ public class TieredStorageProducerClient {
 
     private final List<TierProducerAgent> tierProducerAgents;
 
+    /** The current writing segment index for each subpartition. */
+    private final int[] currentSubpartitionSegmentId;
+
+    /** The current writing tier index for each subpartition. */
+    private final int[] currentSubpartitionTierIndex;
+
+    /** Indicate whether the segment for each subpartition is finished. */
+    private final boolean[] isSubpartitionSegmentFinished;

Review Comment:
   `isCurrentSubpartitionSegmentFinished`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClient.java:
##########
@@ -119,7 +145,66 @@ private void writeAccumulatedBuffers(
     private void writeAccumulatedBuffer(
             TieredStorageSubpartitionId subpartitionId, Buffer 
accumulatedBuffer)
             throws IOException {
-        // TODO, Try to write the accumulated buffer to the appropriate tier. 
After the tier is
-        // decided, then write the accumulated buffer to the tier.
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        int tierIndex = chooseStorageTierIfNeeded(subpartitionId);
+
+        Buffer compressedBuffer = compressBufferIfPossible(accumulatedBuffer);
+        updateStatistics(compressedBuffer);
+        boolean isLastBufferInSegment =
+                tierProducerAgents.get(tierIndex).write(subpartitionId, 
compressedBuffer);

Review Comment:
   `TierProducerAgent#write` lacks docs for the return value semantics. I 
thought the return-value mean whether the buffer is successfully written. How 
does the agent know whether it's the last buffer without knowing the size of 
next buffer?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClient.java:
##########
@@ -80,13 +101,18 @@ public void write(
 
         if (isBroadcast && !isBroadcastOnly) {
             for (int i = 0; i < numSubpartitions; ++i) {
-                bufferAccumulator.receive(record.duplicate(), subpartitionId, 
dataType);
+                bufferAccumulator.receive(
+                        record.duplicate(), new 
TieredStorageSubpartitionId(i), dataType);

Review Comment:
   Let's fix this in a separate hotfix commit.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/SubpartitionSegmentIdTrackerImpl.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId;
+import 
org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * The implementation of {@link SubpartitionSegmentIdTracker}. Each {@link 
TierProducerAgent} has a
+ * {@link SubpartitionSegmentIdTrackerImpl} to record the segments belong to 
the producer agent.
+ */
+public class SubpartitionSegmentIdTrackerImpl implements 
SubpartitionSegmentIdTracker {
+
+    /**
+     * Each subpartition calculates the amount of data written to a tier 
separately. If the amount
+     * of data exceeds the threshold, the current writing segment will be 
finished. Because
+     * different subpartitions may have duplicate segment indexes, this field 
needs to distinguish
+     * between different subpartitions.
+     */
+    private final Map<TieredStorageSubpartitionId, HashSet<Integer>> 
subpartitionSegmentIds;
+
+    /** Indicate whether this is a broadcast only partition. */
+    private final Boolean isBroadCastOnly;
+
+    /** Record the latest segment index for each subpartition. */
+    private final int[] latestSegmentIds;
+
+    /** The locks for all subpartitions. */
+    private final Lock[] locks;
+
+    public SubpartitionSegmentIdTrackerImpl(int numSubpartitions, Boolean 
isBroadcastOnly) {
+        this.isBroadCastOnly = isBroadcastOnly;
+        int effectiveNumSubpartitions = isBroadcastOnly ? 1 : numSubpartitions;
+        this.latestSegmentIds = new int[effectiveNumSubpartitions];
+        this.subpartitionSegmentIds = new HashMap<>();
+        this.locks = new Lock[effectiveNumSubpartitions];
+
+        Arrays.fill(latestSegmentIds, -1);
+        for (int i = 0; i < effectiveNumSubpartitions; i++) {
+            locks[i] = new ReentrantLock();
+            subpartitionSegmentIds.put(new TieredStorageSubpartitionId(i), new 
HashSet<>());

Review Comment:
   This is heavy. Try replace `new HashSet<>()` with 
`ConcurrentHashMap.newKeySet()`. This basically creates a "ConcurrentHashSet" 
backed by `ConcurrentHashMap`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClient.java:
##########
@@ -119,7 +145,66 @@ private void writeAccumulatedBuffers(
     private void writeAccumulatedBuffer(
             TieredStorageSubpartitionId subpartitionId, Buffer 
accumulatedBuffer)
             throws IOException {
-        // TODO, Try to write the accumulated buffer to the appropriate tier. 
After the tier is
-        // decided, then write the accumulated buffer to the tier.
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        int tierIndex = chooseStorageTierIfNeeded(subpartitionId);
+
+        Buffer compressedBuffer = compressBufferIfPossible(accumulatedBuffer);
+        updateStatistics(compressedBuffer);
+        boolean isLastBufferInSegment =
+                tierProducerAgents.get(tierIndex).write(subpartitionId, 
compressedBuffer);
+        if (isLastBufferInSegment) {
+            isSubpartitionSegmentFinished[subpartitionIndex] = true;
+        }
+    }
+
+    private int chooseStorageTierIfNeeded(TieredStorageSubpartitionId 
subpartitionId)
+            throws IOException {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        if (isSubpartitionSegmentFinished[subpartitionIndex]) {
+            // If the current segment is finished, this subpartition need 
choose a new storage tier.
+            internalChooseStorageTier(subpartitionId);
+            isSubpartitionSegmentFinished[subpartitionIndex] = false;
+        }
+        return currentSubpartitionTierIndex[subpartitionIndex];
+    }
+
+    private void internalChooseStorageTier(TieredStorageSubpartitionId 
subpartitionId)
+            throws IOException {
+        int subpartitionIndex = subpartitionId.getSubpartitionId();
+        int segmentIndex = currentSubpartitionSegmentId[subpartitionIndex];
+        int nextSegmentIndex = segmentIndex + 1;
+
+        for (int tierIndex = 0; tierIndex < tierProducerAgents.size(); 
++tierIndex) {

Review Comment:
   We should make it explicit that `tierProducerAgents` stores agents ordered 
by priority.



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