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


##########
flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java:
##########
@@ -225,6 +225,35 @@ public class NettyShuffleEnvironmentOptions {
                                     + " help relieve back-pressure caused by 
unbalanced data distribution among the subpartitions. This value should be"
                                     + " increased in case of higher round trip 
times between nodes and/or larger number of machines in the cluster.");
 
+    /**
+     * Maximum number of network buffers to use for each outgoing/incoming 
gate (result
+     * partition/input gate), which contains all exclusive network buffers for 
all subpartitions and
+     * all floating buffers for the gate. The exclusive network buffers for 
one channel is
+     * configured by {@link #NETWORK_BUFFERS_PER_CHANNEL} and the floating 
buffers for one gate is
+     * configured by {@link #NETWORK_EXTRA_BUFFERS_PER_GATE}.
+     */
+    @Experimental
+    @Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+    public static final ConfigOption<Integer> 
NETWORK_READ_MAX_REQUIRED_BUFFERS_PER_GATE_MAX =

Review Comment:
   ```suggestion
       public static final ConfigOption<Integer> 
NETWORK_READ_MAX_REQUIRED_BUFFERS_PER_GATE_MAX =
   ```
   ```suggestion
       public static final ConfigOption<Integer> 
NETWORK_READ_MAX_REQUIRED_BUFFERS_PER_GATE =
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =

Review Comment:
   ```suggestion
           int targetRequiredBuffersPerGate =
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java:
##########
@@ -119,6 +139,82 @@ public static int computeNetworkBuffersForAnnouncing(
         return requirementForInputs + requirementForOutputs;
     }
 
+    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);
+    }
+
+    private static int getNumBuffersToAnnounceForInputGate(
+            ResultPartitionType type,
+            int configuredNetworkBuffersPerChannel,
+            int floatingNetworkBuffersPerGate,
+            Optional<Integer> maxRequiredBuffersPerGate,
+            int numInputChannels) {
+        GateBuffersSpec gateBuffersSpec =
+                GateBuffersSpec.from(
+                        maxRequiredBuffersPerGate,
+                        configuredNetworkBuffersPerChannel,
+                        floatingNetworkBuffersPerGate,
+                        type,
+                        numInputChannels);
+        return gateBuffersSpec.maxEffectiveTotalGateBuffers();
+    }
+
+    public static boolean isPipelineResultPartition(ResultPartitionType 
partitionType) {
+        return partitionType.isPipelinedOrPipelinedBoundedResultPartition()
+                || partitionType == ResultPartitionType.PIPELINED_APPROXIMATE;
+    }
+
+    /**
+     * Since at least one floating buffer is required, the number of min 
required buffers is reduced
+     * by 1, and then the average number of buffers per channel is calculated. 
Take the minimum
+     * value to ensure that the number of required buffers per gate is not 
more than the given
+     * minRequiredBuffersPerGate.}.
+     */
+    public static int adjustExclusiveBuffersPerChannel(
+            int configuredNetworkBuffersPerChannel,
+            int numInputChannels,
+            int minRequiredBuffersPerGate) {
+        checkArgument(numInputChannels > 0, "Must be positive.");
+        return Math.min(
+                configuredNetworkBuffersPerChannel,
+                (minRequiredBuffersPerGate - 1) / numInputChannels);
+    }
+
+    public static int getMinBuffersTargetPerGate(
+            int numInputChannels, int configuredNetworkBuffersPerChannel) {
+        return numInputChannels * configuredNetworkBuffersPerChannel + 1;
+    }
+
+    public static int getMaxBuffersTargetPerGate(
+            int numInputChannels,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingBuffersPerGate) {
+        return numInputChannels * configuredNetworkBuffersPerChannel
+                + Math.max(1, configuredFloatingBuffersPerGate);

Review Comment:
   `configuredFloatingBuffersPerGate` is required to be at least `1`. We should 
add check for this.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =

Review Comment:
   ```suggestion
           int targetTotalBuffersPerGate =
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java:
##########
@@ -119,6 +139,82 @@ public static int computeNetworkBuffersForAnnouncing(
         return requirementForInputs + requirementForOutputs;
     }
 
+    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);
+    }
+
+    private static int getNumBuffersToAnnounceForInputGate(
+            ResultPartitionType type,
+            int configuredNetworkBuffersPerChannel,
+            int floatingNetworkBuffersPerGate,
+            Optional<Integer> maxRequiredBuffersPerGate,
+            int numInputChannels) {
+        GateBuffersSpec gateBuffersSpec =
+                GateBuffersSpec.from(
+                        maxRequiredBuffersPerGate,
+                        configuredNetworkBuffersPerChannel,
+                        floatingNetworkBuffersPerGate,
+                        type,
+                        numInputChannels);
+        return gateBuffersSpec.maxEffectiveTotalGateBuffers();
+    }
+
+    public static boolean isPipelineResultPartition(ResultPartitionType 
partitionType) {
+        return partitionType.isPipelinedOrPipelinedBoundedResultPartition()
+                || partitionType == ResultPartitionType.PIPELINED_APPROXIMATE;
+    }
+
+    /**
+     * Since at least one floating buffer is required, the number of min 
required buffers is reduced
+     * by 1, and then the average number of buffers per channel is calculated. 
Take the minimum
+     * value to ensure that the number of required buffers per gate is not 
more than the given
+     * minRequiredBuffersPerGate.}.
+     */
+    public static int adjustExclusiveBuffersPerChannel(
+            int configuredNetworkBuffersPerChannel,
+            int numInputChannels,
+            int minRequiredBuffersPerGate) {
+        checkArgument(numInputChannels > 0, "Must be positive.");
+        return Math.min(
+                configuredNetworkBuffersPerChannel,
+                (minRequiredBuffersPerGate - 1) / numInputChannels);

Review Comment:
   Check that `minRequiredBuffersPerGate >= 1`. 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =
+                getMaxBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int minRequiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
minBuffersTargetPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                adjustExclusiveBuffersPerChannel(
+                        configuredNetworkBuffersPerChannel,
+                        numInputChannels,
+                        minRequiredBuffersPerGate);
+        int effectiveExclusiveBuffersPerGate =
+                getEffectiveExclusiveBuffersPerGate(
+                        numInputChannels, effectiveExclusiveBuffersPerChannel);
+
+        int minOptionalBuffers = minRequiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+        int maxOptionalBuffers = maxBuffersTargetPerGate - 
effectiveExclusiveBuffersPerGate;

Review Comment:
   ```suggestion
           int totalFloatingBuffers = maxBuffersTargetPerGate - 
effectiveExclusiveBuffersPerGate;
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =
+                getMaxBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int minRequiredBuffersPerGate =

Review Comment:
   ```suggestion
           int requiredBUffersPerGate =
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java:
##########
@@ -119,6 +139,82 @@ public static int computeNetworkBuffersForAnnouncing(
         return requirementForInputs + requirementForOutputs;
     }
 
+    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);
+    }
+
+    private static int getNumBuffersToAnnounceForInputGate(
+            ResultPartitionType type,
+            int configuredNetworkBuffersPerChannel,
+            int floatingNetworkBuffersPerGate,
+            Optional<Integer> maxRequiredBuffersPerGate,
+            int numInputChannels) {
+        GateBuffersSpec gateBuffersSpec =
+                GateBuffersSpec.from(
+                        maxRequiredBuffersPerGate,
+                        configuredNetworkBuffersPerChannel,
+                        floatingNetworkBuffersPerGate,
+                        type,
+                        numInputChannels);
+        return gateBuffersSpec.maxEffectiveTotalGateBuffers();
+    }
+
+    public static boolean isPipelineResultPartition(ResultPartitionType 
partitionType) {
+        return partitionType.isPipelinedOrPipelinedBoundedResultPartition()
+                || partitionType == ResultPartitionType.PIPELINED_APPROXIMATE;
+    }
+
+    /**
+     * Since at least one floating buffer is required, the number of min 
required buffers is reduced
+     * by 1, and then the average number of buffers per channel is calculated. 
Take the minimum
+     * value to ensure that the number of required buffers per gate is not 
more than the given
+     * minRequiredBuffersPerGate.}.
+     */
+    public static int adjustExclusiveBuffersPerChannel(
+            int configuredNetworkBuffersPerChannel,
+            int numInputChannels,
+            int minRequiredBuffersPerGate) {
+        checkArgument(numInputChannels > 0, "Must be positive.");
+        return Math.min(
+                configuredNetworkBuffersPerChannel,
+                (minRequiredBuffersPerGate - 1) / numInputChannels);
+    }
+
+    public static int getMinBuffersTargetPerGate(
+            int numInputChannels, int configuredNetworkBuffersPerChannel) {
+        return numInputChannels * configuredNetworkBuffersPerChannel + 1;
+    }
+
+    public static int getMaxBuffersTargetPerGate(
+            int numInputChannels,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingBuffersPerGate) {
+        return numInputChannels * configuredNetworkBuffersPerChannel
+                + Math.max(1, configuredFloatingBuffersPerGate);
+    }
+
+    public static int getEffectiveExclusiveBuffersPerGate(
+            int numInputChannels, int effectiveExclusiveBuffersPerChannel) {
+        return effectiveExclusiveBuffersPerChannel * numInputChannels;
+    }
+
+    private static int getReusePartitionInputBuffers(
+            int numTotalInputGates,
+            Map<IntermediateDataSetID, Integer> inputChannelNums,
+            int maxSingleGateBuffers) {
+        checkState(numTotalInputGates >= inputChannelNums.size());
+        if (numTotalInputGates == inputChannelNums.size()) {
+            return 0;
+        }
+        return (numTotalInputGates - inputChannelNums.size()) * 
maxSingleGateBuffers;
+    }
+

Review Comment:
   We might introduce a new util class for the input gate related utils.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =
+                getMaxBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int minRequiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
minBuffersTargetPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                adjustExclusiveBuffersPerChannel(

Review Comment:
   ```suggestion
                   getExclusiveBuffersPerChannel(
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =
+                getMaxBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int minRequiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
minBuffersTargetPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                adjustExclusiveBuffersPerChannel(
+                        configuredNetworkBuffersPerChannel,
+                        numInputChannels,
+                        minRequiredBuffersPerGate);
+        int effectiveExclusiveBuffersPerGate =
+                getEffectiveExclusiveBuffersPerGate(
+                        numInputChannels, effectiveExclusiveBuffersPerChannel);
+
+        int minOptionalBuffers = minRequiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
+        int maxOptionalBuffers = maxBuffersTargetPerGate - 
effectiveExclusiveBuffersPerGate;
+        int maxEffectiveTotalBuffersPerGate = effectiveExclusiveBuffersPerGate 
+ maxOptionalBuffers;

Review Comment:
   Shouldn't this always be `maxBuffersTargetPerGate`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpec.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.consumer;
+
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.shuffle.NettyShuffleUtils;
+
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.adjustExclusiveBuffersPerChannel;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveExclusiveBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getEffectiveMaxRequiredBuffersPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMaxBuffersTargetPerGate;
+import static 
org.apache.flink.runtime.shuffle.NettyShuffleUtils.getMinBuffersTargetPerGate;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * 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_MAX}. 
If the option is
+ * not configured, the threshold for Batch jobs is {@link
+ * NettyShuffleUtils#DEFAULT_MAX_REQUIRED_BUFFERS_PER_GATE_FOR_BATCH} and the 
threshold for
+ * Streaming jobs is {#link 
NettyShuffleUtils#DEFAULT_MAX_BUFFERS_PER_GATE_FOR_STREAMING}.
+ */
+public class GateBuffersSpec {
+
+    private final int effectiveExclusiveBuffersPerChannel;
+
+    private final int minOptionalBuffers;
+
+    private final int maxOptionalBuffers;
+
+    private final int maxEffectiveTotalBuffersPerGate;
+
+    private GateBuffersSpec(
+            int effectiveExclusiveBuffersPerChannel,
+            int minOptionalBuffers,
+            int maxOptionalBuffers,
+            int maxEffectiveTotalBuffersPerGate) {
+        this.effectiveExclusiveBuffersPerChannel = 
effectiveExclusiveBuffersPerChannel;
+        this.minOptionalBuffers = minOptionalBuffers;
+        this.maxOptionalBuffers = maxOptionalBuffers;
+        this.maxEffectiveTotalBuffersPerGate = maxEffectiveTotalBuffersPerGate;
+    }
+
+    int minOptionalBuffers() {
+        return minOptionalBuffers;
+    }
+
+    int maxOptionalBuffers() {
+        return maxOptionalBuffers;
+    }
+
+    int getEffectiveExclusiveBuffersPerChannel() {
+        return effectiveExclusiveBuffersPerChannel;
+    }
+
+    public int maxEffectiveTotalGateBuffers() {
+        return maxEffectiveTotalBuffersPerGate;
+    }
+
+    public static GateBuffersSpec from(
+            Optional<Integer> configuredMaxRequiredBuffersPerGate,
+            int configuredNetworkBuffersPerChannel,
+            int configuredFloatingNetworkBuffersPerGate,
+            ResultPartitionType partitionType,
+            int numInputChannels) {
+        int maxRequiredBuffersThresholdPerGate =
+                getEffectiveMaxRequiredBuffersPerGate(
+                        partitionType, configuredMaxRequiredBuffersPerGate);
+        int minBuffersTargetPerGate =
+                getMinBuffersTargetPerGate(numInputChannels, 
configuredNetworkBuffersPerChannel);
+        int maxBuffersTargetPerGate =
+                getMaxBuffersTargetPerGate(
+                        numInputChannels,
+                        configuredNetworkBuffersPerChannel,
+                        configuredFloatingNetworkBuffersPerGate);
+        int minRequiredBuffersPerGate =
+                Math.min(maxRequiredBuffersThresholdPerGate, 
minBuffersTargetPerGate);
+
+        int effectiveExclusiveBuffersPerChannel =
+                adjustExclusiveBuffersPerChannel(
+                        configuredNetworkBuffersPerChannel,
+                        numInputChannels,
+                        minRequiredBuffersPerGate);
+        int effectiveExclusiveBuffersPerGate =
+                getEffectiveExclusiveBuffersPerGate(
+                        numInputChannels, effectiveExclusiveBuffersPerChannel);
+
+        int minOptionalBuffers = minRequiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;

Review Comment:
   ```suggestion
           int requiredFloatingBuffers = minRequiredBuffersPerGate - 
effectiveExclusiveBuffersPerGate;
   ```



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