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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsConsumerId.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.hybrid;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+/** This class represents the identifier of hybrid shuffle's consumer. */
+public class HsConsumerId {
+    /**
+     * This consumer id is used in the scenarios that information related to 
specific consumer needs
+     * to be ignored.
+     */
+    public static final HsConsumerId ANY = new HsConsumerId(-1);
+
+    /**
+     * This consumer id is used in the scenarios that only one consumer is 
allowed for a single
+     * subpartition.
+     */
+    public static final HsConsumerId DEFAULT = new HsConsumerId(0);

Review Comment:
   JavaDoc needs update.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartition.java:
##########
@@ -295,4 +297,15 @@ private HsSpillingStrategy getSpillingStrategy(
                 throw new IllegalConfigurationException("Illegal spilling 
strategy.");
         }
     }
+
+    private void checkMultipleConsumerIsAllowed(
+            HsConsumerId newConsumerId, HybridShuffleConfiguration 
hybridShuffleConfiguration) {
+        if (hybridShuffleConfiguration.getSpillingStrategyType()
+                == SpillingStrategyType.SELECTIVE) {
+            checkState(
+                    newConsumerId == HsConsumerId.DEFAULT,

Review Comment:
   I'd suggest to check that the last id is `null`, rather than the current id 
is `DEFAULT`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java:
##########
@@ -216,6 +218,11 @@ public ResultPartition create(
             }
         } else if (type == ResultPartitionType.HYBRID_FULL
                 || type == ResultPartitionType.HYBRID_SELECTIVE) {
+            if (isBroadcast) {
+                // for broadcast result partition, it can be optimized to 
always use full spilling
+                // strategy to significantly reduce shuffle data writing cost.
+                type = ResultPartitionType.HYBRID_FULL;

Review Comment:
   It would be nice to print some log when overwriting things.



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