zhuzhurk commented on code in PR #25552:
URL: https://github.com/apache/flink/pull/25552#discussion_r1900508861


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultVertexParallelismAndInputInfosDecider.java:
##########
@@ -126,52 +136,101 @@ public ParallelismAndInputInfos 
decideParallelismAndInputInfosForVertex(
                             ? vertexInitialParallelism
                             : computeSourceParallelismUpperBound(jobVertexId, 
vertexMaxParallelism);
             return new ParallelismAndInputInfos(parallelism, 
Collections.emptyMap());
-        } else {

Review Comment:
   How about to introduce a method `decideParallelismAndInputInfosForNonSource` 
to make the contents more organized?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputer.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.scheduler.adaptivebatch.util;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.VertexInputInfoComputationUtils;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.DefaultVertexParallelismAndInputInfosDecider.MAX_NUM_SUBPARTITIONS_PER_TASK_CONSUME;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.adjustToClosestLegalParallelism;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.checkAndGetSubpartitionNum;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class that computes VertexInputInfo for pointwise input. */
+public class PointwiseVertexInputInfoComputer {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(PointwiseVertexInputInfoComputer.class);
+
+    private final long dataVolumePerTask;
+
+    public PointwiseVertexInputInfoComputer(long dataVolumePerTask) {
+        this.dataVolumePerTask = dataVolumePerTask;
+    }
+
+    /**
+     * Decide parallelism and input infos, which will make the data be evenly 
distributed to
+     * downstream subtasks for POINTWISE, such that different downstream 
subtasks consume roughly
+     * the same amount of data.
+     *
+     * @param inputInfo The information of consumed blocking results
+     * @param parallelism The parallelism of the job vertex
+     * @return the vertex input info
+     */
+    public JobVertexInputInfo compute(BlockingInputInfo inputInfo, Integer 
parallelism) {
+
+        Map<Integer, long[]> subpartitionBytesByPartitionIndex =
+                inputInfo.getSubpartitionBytesByPartitionIndex();
+        int numPartitions = inputInfo.getNumPartitions();
+        int numSubPartitions = 
checkAndGetSubpartitionNum(Collections.singletonList(inputInfo));
+        // compute the size of each subpartition

Review Comment:
   Looks to me the comment is not accurate. It just converts a 2D array into a 
1D array.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputer.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.scheduler.adaptivebatch.util;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.VertexInputInfoComputationUtils;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.DefaultVertexParallelismAndInputInfosDecider.MAX_NUM_SUBPARTITIONS_PER_TASK_CONSUME;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.adjustToClosestLegalParallelism;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.checkAndGetSubpartitionNum;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class that computes VertexInputInfo for pointwise input. */
+public class PointwiseVertexInputInfoComputer {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(PointwiseVertexInputInfoComputer.class);
+
+    private final long dataVolumePerTask;
+
+    public PointwiseVertexInputInfoComputer(long dataVolumePerTask) {
+        this.dataVolumePerTask = dataVolumePerTask;
+    }
+
+    /**
+     * Decide parallelism and input infos, which will make the data be evenly 
distributed to
+     * downstream subtasks for POINTWISE, such that different downstream 
subtasks consume roughly
+     * the same amount of data.
+     *
+     * @param inputInfo The information of consumed blocking results
+     * @param parallelism The parallelism of the job vertex
+     * @return the vertex input info
+     */
+    public JobVertexInputInfo compute(BlockingInputInfo inputInfo, Integer 
parallelism) {

Review Comment:
   Integer -> int



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputer.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.scheduler.adaptivebatch.util;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.VertexInputInfoComputationUtils;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.DefaultVertexParallelismAndInputInfosDecider.MAX_NUM_SUBPARTITIONS_PER_TASK_CONSUME;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.adjustToClosestLegalParallelism;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.checkAndGetSubpartitionNum;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class that computes VertexInputInfo for pointwise input. */
+public class PointwiseVertexInputInfoComputer {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(PointwiseVertexInputInfoComputer.class);
+
+    private final long dataVolumePerTask;
+
+    public PointwiseVertexInputInfoComputer(long dataVolumePerTask) {
+        this.dataVolumePerTask = dataVolumePerTask;
+    }
+
+    /**
+     * Decide parallelism and input infos, which will make the data be evenly 
distributed to
+     * downstream subtasks for POINTWISE, such that different downstream 
subtasks consume roughly
+     * the same amount of data.
+     *
+     * @param inputInfo The information of consumed blocking results
+     * @param parallelism The parallelism of the job vertex
+     * @return the vertex input info
+     */
+    public JobVertexInputInfo compute(BlockingInputInfo inputInfo, Integer 
parallelism) {
+

Review Comment:
   Unnecessary empty line.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputer.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.scheduler.adaptivebatch.util;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.VertexInputInfoComputationUtils;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.DefaultVertexParallelismAndInputInfosDecider.MAX_NUM_SUBPARTITIONS_PER_TASK_CONSUME;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.adjustToClosestLegalParallelism;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.checkAndGetSubpartitionNum;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class that computes VertexInputInfo for pointwise input. */
+public class PointwiseVertexInputInfoComputer {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(PointwiseVertexInputInfoComputer.class);
+
+    private final long dataVolumePerTask;
+
+    public PointwiseVertexInputInfoComputer(long dataVolumePerTask) {
+        this.dataVolumePerTask = dataVolumePerTask;
+    }
+
+    /**
+     * Decide parallelism and input infos, which will make the data be evenly 
distributed to
+     * downstream subtasks for POINTWISE, such that different downstream 
subtasks consume roughly
+     * the same amount of data.
+     *
+     * @param inputInfo The information of consumed blocking results
+     * @param parallelism The parallelism of the job vertex
+     * @return the vertex input info
+     */
+    public JobVertexInputInfo compute(BlockingInputInfo inputInfo, Integer 
parallelism) {
+
+        Map<Integer, long[]> subpartitionBytesByPartitionIndex =
+                inputInfo.getSubpartitionBytesByPartitionIndex();
+        int numPartitions = inputInfo.getNumPartitions();
+        int numSubPartitions = 
checkAndGetSubpartitionNum(Collections.singletonList(inputInfo));
+        // compute the size of each subpartition
+        long[] nums = new long[numPartitions * numSubPartitions];

Review Comment:
   The name `nums` is not accurate.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultVertexParallelismAndInputInfosDecider.java:
##########
@@ -126,52 +136,101 @@ public ParallelismAndInputInfos 
decideParallelismAndInputInfosForVertex(
                             ? vertexInitialParallelism
                             : computeSourceParallelismUpperBound(jobVertexId, 
vertexMaxParallelism);
             return new ParallelismAndInputInfos(parallelism, 
Collections.emptyMap());
-        } else {
-            int minParallelism = Math.max(globalMinParallelism, 
vertexMinParallelism);
-            int maxParallelism = globalMaxParallelism;
-
-            if (vertexInitialParallelism == ExecutionConfig.PARALLELISM_DEFAULT
-                    && vertexMaxParallelism < minParallelism) {
-                LOG.info(
-                        "The vertex maximum parallelism {} is smaller than the 
minimum parallelism {}. "
-                                + "Use {} as the lower bound to decide 
parallelism of job vertex {}.",
-                        vertexMaxParallelism,
-                        minParallelism,
-                        vertexMaxParallelism,
-                        jobVertexId);
-                minParallelism = vertexMaxParallelism;
-            }
-            if (vertexInitialParallelism == ExecutionConfig.PARALLELISM_DEFAULT
-                    && vertexMaxParallelism < maxParallelism) {
-                LOG.info(
-                        "The vertex maximum parallelism {} is smaller than the 
global maximum parallelism {}. "
-                                + "Use {} as the upper bound to decide 
parallelism of job vertex {}.",
-                        vertexMaxParallelism,
-                        maxParallelism,
-                        vertexMaxParallelism,
-                        jobVertexId);
-                maxParallelism = vertexMaxParallelism;
-            }
-            checkState(maxParallelism >= minParallelism);
-
-            if (vertexInitialParallelism == ExecutionConfig.PARALLELISM_DEFAULT
-                    && areAllInputsAllToAll(consumedResults)
-                    && !areAllInputsBroadcast(consumedResults)) {
-                return decideParallelismAndEvenlyDistributeData(
-                        jobVertexId,
-                        consumedResults,
-                        vertexInitialParallelism,
-                        minParallelism,
-                        maxParallelism);
-            } else {
-                return decideParallelismAndEvenlyDistributeSubpartitions(
-                        jobVertexId,
-                        consumedResults,
-                        vertexInitialParallelism,
-                        minParallelism,
-                        maxParallelism);
+        }
+
+        int minParallelism = Math.max(globalMinParallelism, 
vertexMinParallelism);
+        int maxParallelism = globalMaxParallelism;
+
+        if (vertexInitialParallelism == ExecutionConfig.PARALLELISM_DEFAULT
+                && vertexMaxParallelism < minParallelism) {
+            LOG.info(
+                    "The vertex maximum parallelism {} is smaller than the 
minimum parallelism {}. "
+                            + "Use {} as the lower bound to decide parallelism 
of job vertex {}.",
+                    vertexMaxParallelism,
+                    minParallelism,
+                    vertexMaxParallelism,
+                    jobVertexId);
+            minParallelism = vertexMaxParallelism;
+        }
+        if (vertexInitialParallelism == ExecutionConfig.PARALLELISM_DEFAULT
+                && vertexMaxParallelism < maxParallelism) {
+            LOG.info(
+                    "The vertex maximum parallelism {} is smaller than the 
global maximum parallelism {}. "
+                            + "Use {} as the upper bound to decide parallelism 
of job vertex {}.",
+                    vertexMaxParallelism,
+                    maxParallelism,
+                    vertexMaxParallelism,
+                    jobVertexId);
+            maxParallelism = vertexMaxParallelism;
+        }
+        checkState(maxParallelism >= minParallelism);
+
+        int parallelism =
+                vertexInitialParallelism > 0
+                        ? vertexInitialParallelism
+                        : decideParallelism(
+                                jobVertexId, consumedResults, minParallelism, 
maxParallelism);
+
+        Map<Boolean, List<BlockingInputInfo>> inputsGroupByInterCorrelation =
+                consumedResults.stream()
+                        .collect(
+                                Collectors.groupingBy(
+                                        
BlockingInputInfo::existInterInputsKeyCorrelation));
+
+        // For AllToAll like inputs, we derive parallelism as a whole, while 
for Pointwise inputs,
+        // we need to derive parallelism separately for each input.
+        //
+        // In the following cases, we need to reset min parallelism and max 
parallelism to ensure
+        // that the decide parallelism for all inputs is consistent :
+        // 1.  Vertex has a specified parallelism
+        // 2.  There are edges that don't need to follow intergroup constraint
+        if (vertexInitialParallelism > 0 || 
inputsGroupByInterCorrelation.containsKey(false)) {
+            minParallelism = parallelism;
+            maxParallelism = parallelism;
+        }
+
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap = 
new HashMap<>();
+
+        if (inputsGroupByInterCorrelation.containsKey(true)) {
+            vertexInputInfoMap.putAll(
+                    allToAllVertexInputInfoComputer.compute(
+                            jobVertexId,
+                            inputsGroupByInterCorrelation.get(true),
+                            parallelism,
+                            minParallelism,
+                            maxParallelism));
+        }
+
+        if (inputsGroupByInterCorrelation.containsKey(false)) {

Review Comment:
   Comments are needed to explain why we handle inputs with inter/intra 
correlation separately.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputer.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.scheduler.adaptivebatch.util;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.VertexInputInfoComputationUtils;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.DefaultVertexParallelismAndInputInfosDecider.MAX_NUM_SUBPARTITIONS_PER_TASK_CONSUME;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.adjustToClosestLegalParallelism;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.util.VertexParallelismAndInputInfosDeciderUtils.checkAndGetSubpartitionNum;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Helper class that computes VertexInputInfo for pointwise input. */
+public class PointwiseVertexInputInfoComputer {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(PointwiseVertexInputInfoComputer.class);
+
+    private final long dataVolumePerTask;
+
+    public PointwiseVertexInputInfoComputer(long dataVolumePerTask) {
+        this.dataVolumePerTask = dataVolumePerTask;
+    }
+
+    /**
+     * Decide parallelism and input infos, which will make the data be evenly 
distributed to
+     * downstream subtasks for POINTWISE, such that different downstream 
subtasks consume roughly
+     * the same amount of data.
+     *
+     * @param inputInfo The information of consumed blocking results
+     * @param parallelism The parallelism of the job vertex
+     * @return the vertex input info
+     */
+    public JobVertexInputInfo compute(BlockingInputInfo inputInfo, Integer 
parallelism) {
+
+        Map<Integer, long[]> subpartitionBytesByPartitionIndex =
+                inputInfo.getSubpartitionBytesByPartitionIndex();
+        int numPartitions = inputInfo.getNumPartitions();
+        int numSubPartitions = 
checkAndGetSubpartitionNum(Collections.singletonList(inputInfo));
+        // compute the size of each subpartition
+        long[] nums = new long[numPartitions * numSubPartitions];

Review Comment:
   Why it assumes that each partition contains the same number of 
sub-partitions?



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to