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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+
+/**
+ * Calculate the buffers specs(the exclusive buffers per channel, min/max 
optional buffers per gate)
+ * based on the configurations, result partition type, and the number of 
channels.
+ *
+ * <p>The threshold is configured by {@link
+ * NettyShuffleEnvironmentOptions#NETWORK_READ_MAX_REQUIRED_BUFFERS_PER_GATE}. 
If the option is not
+ * configured, the threshold for Batch jobs is {@link
+ * InputGateSpecUitls#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {@link 
InputGateSpecUitls#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_STREAM}.
+ */

Review Comment:
   JavaDoc needs to be updated.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateSpecUitls.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.consumer;
+
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Utils to manage the specs of the {@link InputGate}, for example, {@link 
GateBuffersSpec}. */
+public class InputGateSpecUitls {
+
+    public static final int DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH = 
1000;
+
+    public static final int DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_STREAM = 
Integer.MAX_VALUE;
+
+    public static GateBuffersSpec createGateBuffersSpec(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int targetRequiredBuffersPerGate =
+                getRequiredBuffersTargetPerGate(
+                        numInputChannels, configuredNetworkBuffersPerChannel);
+        int targetTotalBuffersPerGate =
+                getTotalBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int requiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
targetRequiredBuffersPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                getExclusiveBuffersPerChannel(
+                        configuredNetworkBuffersPerChannel,
+                        numInputChannels,
+                        requiredBuffersPerGate);
+        int effectiveExclusiveBuffersPerGate =
+                getEffectiveExclusiveBuffersPerGate(
+                        numInputChannels, effectiveExclusiveBuffersPerChannel);
+
+        int requiredFloatingBuffers = requiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+        int totalFloatingBuffers = targetTotalBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+
+        checkState(requiredFloatingBuffers > 0, "Must be positive.");
+        checkState(
+                requiredFloatingBuffers <= totalFloatingBuffers,
+                "Wrong number of floating buffers.");
+
+        return new GateBuffersSpec(
+                effectiveExclusiveBuffersPerChannel,
+                requiredFloatingBuffers,
+                totalFloatingBuffers,
+                targetTotalBuffersPerGate);
+    }
+
+    public static int getEffectiveMaxRequiredBuffersPerGate(
+            ResultPartitionType partitionType,
+            Optional<Integer> configuredMaxRequiredBuffersPerGate) {
+        return configuredMaxRequiredBuffersPerGate.orElseGet(
+                () ->
+                        isPipelineResultPartition(partitionType)
+                                ? 
DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_STREAM
+                                : 
DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH);
+    }
+
+    /**
+     * Since at least one floating buffer is required, the number of required 
buffers is reduced by
+     * 1, and then the average number of buffers per channel is calculated. 
Returning the minimum
+     * value to ensure that the number of required buffers per gate is not 
more than the given
+     * requiredBuffersPerGate.}.
+     */
+    public static int getExclusiveBuffersPerChannel(
+            int configuredNetworkBuffersPerChannel,
+            int numInputChannels,
+            int requiredBuffersPerGate) {
+        checkArgument(numInputChannels > 0, "Must be positive.");
+        checkArgument(requiredBuffersPerGate >= 1, "Require at least 1 buffer 
per gate.");
+        return Math.min(
+                configuredNetworkBuffersPerChannel,
+                (requiredBuffersPerGate - 1) / numInputChannels);
+    }
+
+    public static int getRequiredBuffersTargetPerGate(
+            int numInputChannels, int configuredNetworkBuffersPerChannel) {
+        return numInputChannels * configuredNetworkBuffersPerChannel + 1;
+    }
+
+    public static int getTotalBuffersTargetPerGate(
+            int numInputChannels,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingBuffersPerGate) {
+        return numInputChannels * configuredNetworkBuffersPerChannel
+                + configuredFloatingBuffersPerGate;
+    }
+
+    public static int getEffectiveExclusiveBuffersPerGate(
+            int numInputChannels, int effectiveExclusiveBuffersPerChannel) {
+        return effectiveExclusiveBuffersPerChannel * numInputChannels;
+    }

Review Comment:
   Should be private.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateSpecUitls.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.consumer;
+
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Utils to manage the specs of the {@link InputGate}, for example, {@link 
GateBuffersSpec}. */
+public class InputGateSpecUitls {
+
+    public static final int DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH = 
1000;
+
+    public static final int DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_STREAM = 
Integer.MAX_VALUE;
+
+    public static GateBuffersSpec createGateBuffersSpec(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int targetRequiredBuffersPerGate =
+                getRequiredBuffersTargetPerGate(
+                        numInputChannels, configuredNetworkBuffersPerChannel);
+        int targetTotalBuffersPerGate =
+                getTotalBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int requiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
targetRequiredBuffersPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                getExclusiveBuffersPerChannel(
+                        configuredNetworkBuffersPerChannel,
+                        numInputChannels,
+                        requiredBuffersPerGate);
+        int effectiveExclusiveBuffersPerGate =
+                getEffectiveExclusiveBuffersPerGate(
+                        numInputChannels, effectiveExclusiveBuffersPerChannel);
+
+        int requiredFloatingBuffers = requiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+        int totalFloatingBuffers = targetTotalBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+
+        checkState(requiredFloatingBuffers > 0, "Must be positive.");
+        checkState(
+                requiredFloatingBuffers <= totalFloatingBuffers,
+                "Wrong number of floating buffers.");
+
+        return new GateBuffersSpec(
+                effectiveExclusiveBuffersPerChannel,
+                requiredFloatingBuffers,
+                totalFloatingBuffers,
+                targetTotalBuffersPerGate);
+    }
+
+    public static int getEffectiveMaxRequiredBuffersPerGate(
+            ResultPartitionType partitionType,
+            Optional<Integer> configuredMaxRequiredBuffersPerGate) {
+        return configuredMaxRequiredBuffersPerGate.orElseGet(
+                () ->
+                        isPipelineResultPartition(partitionType)
+                                ? 
DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_STREAM
+                                : 
DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH);
+    }
+
+    /**
+     * Since at least one floating buffer is required, the number of required 
buffers is reduced by
+     * 1, and then the average number of buffers per channel is calculated. 
Returning the minimum
+     * value to ensure that the number of required buffers per gate is not 
more than the given
+     * requiredBuffersPerGate.}.
+     */
+    public static int getExclusiveBuffersPerChannel(
+            int configuredNetworkBuffersPerChannel,
+            int numInputChannels,
+            int requiredBuffersPerGate) {
+        checkArgument(numInputChannels > 0, "Must be positive.");
+        checkArgument(requiredBuffersPerGate >= 1, "Require at least 1 buffer 
per gate.");
+        return Math.min(
+                configuredNetworkBuffersPerChannel,
+                (requiredBuffersPerGate - 1) / numInputChannels);
+    }
+
+    public static int getRequiredBuffersTargetPerGate(
+            int numInputChannels, int configuredNetworkBuffersPerChannel) {
+        return numInputChannels * configuredNetworkBuffersPerChannel + 1;
+    }
+
+    public static int getTotalBuffersTargetPerGate(
+            int numInputChannels,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingBuffersPerGate) {
+        return numInputChannels * configuredNetworkBuffersPerChannel
+                + configuredFloatingBuffersPerGate;
+    }
+
+    public static int getEffectiveExclusiveBuffersPerGate(
+            int numInputChannels, int effectiveExclusiveBuffersPerChannel) {
+        return effectiveExclusiveBuffersPerChannel * numInputChannels;
+    }
+
+    public static boolean isPipelineResultPartition(ResultPartitionType 
partitionType) {
+        return partitionType.isPipelinedOrPipelinedBoundedResultPartition()
+                || partitionType == ResultPartitionType.PIPELINED_APPROXIMATE;
+    }

Review Comment:
   Use `ResultPartitionType#isPipelinedOrPipelinedBoundedResultPartition()`.



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