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


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/VertexInputInfoComputerTestUtil.java:
##########
@@ -86,22 +90,55 @@ public static List<BlockingInputInfo> 
createBlockingInputInfos(
         return blockingInputInfos;
     }
 
-    public static void checkJobVertexInputInfo(
+    public static void checkParallelism(
             int targetParallelism,
-            List<BlockingInputInfo> inputInfos,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap) 
{
+        vertexInputInfoMap
+                .values()
+                .forEach(
+                        info ->
+                                
assertThat(info.getExecutionVertexInputInfos().size())
+                                        .isEqualTo(targetParallelism));
+    }
+
+    public static void checkConsumedSubpartitionGroups(
             List<Map<IndexRange, IndexRange>> targetConsumedSubpartitionGroups,
+            List<BlockingInputInfo> inputInfos,
             Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap) 
{
         JobVertexInputInfo vertexInputInfo =
                 checkAndGetJobVertexInputInfo(inputInfos, vertexInputInfoMap);
         List<ExecutionVertexInputInfo> executionVertexInputInfos =
                 vertexInputInfo.getExecutionVertexInputInfos();
-        
assertThat(executionVertexInputInfos.size()).isEqualTo(targetParallelism);
-        for (int i = 0; i < targetParallelism; i++) {
+        for (int i = 0; i < executionVertexInputInfos.size(); i++) {
             
assertThat(executionVertexInputInfos.get(i).getConsumedSubpartitionGroups())
                     .isEqualTo(targetConsumedSubpartitionGroups.get(i));
         }
     }
 
+    public static void checkConsumedDataVolumePerSubtask(
+            long[] targetConsumedDataVolume,
+            List<BlockingInputInfo> inputInfos,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs) {
+        long[] consumedDataVolume = new long[targetConsumedDataVolume.length];
+        for (BlockingInputInfo inputInfo : inputInfos) {
+            JobVertexInputInfo vertexInputInfo = 
vertexInputs.get(inputInfo.getResultId());
+            List<ExecutionVertexInputInfo> executionVertexInputInfos =
+                    vertexInputInfo.getExecutionVertexInputInfos();
+            for (int i = 0; i < executionVertexInputInfos.size(); ++i) {
+                ExecutionVertexInputInfo executionVertexInputInfo =
+                        executionVertexInputInfos.get(i);
+                consumedDataVolume[i] +=
+                        
executionVertexInputInfo.getConsumedSubpartitionGroups().entrySet().stream()
+                                .mapToLong(
+                                        entry ->
+                                                inputInfo.getNumBytesProduced(
+                                                        entry.getKey(), 
entry.getValue()))
+                                .sum();
+            }
+        }
+        assertThat(consumedDataVolume).isEqualTo(targetConsumedDataVolume);
+    }
+
     public static JobVertexInputInfo checkAndGetJobVertexInputInfo(

Review Comment:
   can be private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/VertexInputInfoComputerTestUtil.java:
##########
@@ -86,22 +90,55 @@ public static List<BlockingInputInfo> 
createBlockingInputInfos(
         return blockingInputInfos;
     }
 
-    public static void checkJobVertexInputInfo(
+    public static void checkParallelism(

Review Comment:
   can be private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/VertexInputInfoComputerTestUtil.java:
##########
@@ -0,0 +1,357 @@
+/*
+ * 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;
+
+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.jobgraph.IntermediateDataSetID;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.runtime.executiongraph.IndexRangeUtil.mergeIndexRanges;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Utils for vertex input info computer test. */
+public class VertexInputInfoComputerTestUtil {
+    public static List<BlockingInputInfo> createBlockingInputInfos(
+            int typeNumber,
+            int numInputInfos,
+            int numPartitions,
+            int numSubpartitions,
+            boolean existIntraInputKeyCorrelation,
+            boolean existInterInputsKeyCorrelation,
+            int defaultSize,
+            double skewedFactor,
+            List<Integer> skewedPartitionIndex,
+            List<Integer> skewedSubpartitionIndex,
+            boolean isPointwise) {
+        List<BlockingInputInfo> blockingInputInfos = new ArrayList<>();
+        for (int i = 0; i < numInputInfos; i++) {
+            Map<Integer, long[]> subpartitionBytesByPartitionIndex = new 
HashMap<>();
+            for (int j = 0; j < numPartitions; j++) {
+                long[] subpartitionBytes = new long[numSubpartitions];
+                for (int k = 0; k < numSubpartitions; k++) {
+                    if (skewedSubpartitionIndex.contains(k) || 
skewedPartitionIndex.contains(j)) {
+                        subpartitionBytes[k] = (long) (defaultSize * 
skewedFactor);
+                    } else {
+                        subpartitionBytes[k] = defaultSize;
+                    }
+                }
+                subpartitionBytesByPartitionIndex.put(j, subpartitionBytes);
+            }
+            BlockingResultInfo resultInfo;
+            if (isPointwise) {
+                resultInfo =
+                        new PointwiseBlockingResultInfo(
+                                new IntermediateDataSetID(),
+                                numPartitions,
+                                numSubpartitions,
+                                subpartitionBytesByPartitionIndex);
+            } else {
+                resultInfo =
+                        new AllToAllBlockingResultInfo(
+                                new IntermediateDataSetID(),
+                                numPartitions,
+                                numSubpartitions,
+                                false,
+                                subpartitionBytesByPartitionIndex);
+            }
+            blockingInputInfos.add(
+                    new BlockingInputInfo(
+                            resultInfo,
+                            typeNumber,
+                            existInterInputsKeyCorrelation,
+                            existIntraInputKeyCorrelation));
+        }
+        return blockingInputInfos;
+    }
+
+    public static void checkParallelism(
+            int targetParallelism,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap) 
{
+        vertexInputInfoMap
+                .values()
+                .forEach(
+                        info ->
+                                
assertThat(info.getExecutionVertexInputInfos().size())
+                                        .isEqualTo(targetParallelism));
+    }
+
+    public static void checkConsumedSubpartitionGroups(
+            List<Map<IndexRange, IndexRange>> targetConsumedSubpartitionGroups,
+            List<BlockingInputInfo> inputInfos,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap) 
{
+        JobVertexInputInfo vertexInputInfo =
+                checkAndGetJobVertexInputInfo(inputInfos, vertexInputInfoMap);
+        List<ExecutionVertexInputInfo> executionVertexInputInfos =
+                vertexInputInfo.getExecutionVertexInputInfos();
+        for (int i = 0; i < executionVertexInputInfos.size(); i++) {
+            
assertThat(executionVertexInputInfos.get(i).getConsumedSubpartitionGroups())
+                    .isEqualTo(targetConsumedSubpartitionGroups.get(i));
+        }
+    }
+
+    public static void checkConsumedDataVolumePerSubtask(
+            long[] targetConsumedDataVolume,
+            List<BlockingInputInfo> inputInfos,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs) {
+        long[] consumedDataVolume = new long[targetConsumedDataVolume.length];
+        for (BlockingInputInfo inputInfo : inputInfos) {
+            JobVertexInputInfo vertexInputInfo = 
vertexInputs.get(inputInfo.getResultId());
+            List<ExecutionVertexInputInfo> executionVertexInputInfos =
+                    vertexInputInfo.getExecutionVertexInputInfos();
+            for (int i = 0; i < executionVertexInputInfos.size(); ++i) {
+                ExecutionVertexInputInfo executionVertexInputInfo =
+                        executionVertexInputInfos.get(i);
+                consumedDataVolume[i] +=
+                        
executionVertexInputInfo.getConsumedSubpartitionGroups().entrySet().stream()
+                                .mapToLong(
+                                        entry ->
+                                                inputInfo.getNumBytesProduced(
+                                                        entry.getKey(), 
entry.getValue()))
+                                .sum();
+            }
+        }
+        assertThat(consumedDataVolume).isEqualTo(targetConsumedDataVolume);
+    }
+
+    public static JobVertexInputInfo checkAndGetJobVertexInputInfo(
+            List<BlockingInputInfo> inputInfos,
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap) 
{
+        List<JobVertexInputInfo> vertexInputInfos =
+                inputInfos.stream()
+                        .map(inputInfo -> 
vertexInputInfoMap.get(inputInfo.getResultId()))
+                        .collect(Collectors.toList());
+        assertThat(vertexInputInfos.size()).isEqualTo(inputInfos.size());
+        JobVertexInputInfo baseVertexInputInfo = vertexInputInfos.get(0);
+        for (int i = 1; i < vertexInputInfos.size(); i++) {
+            assertThat(vertexInputInfos.get(i)).isEqualTo(baseVertexInputInfo);
+        }
+        return baseVertexInputInfo;
+    }
+
+    public static void checkCorrectnessForNonCorrelatedInput(
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap,
+            BlockingInputInfo inputInfo,
+            int targetParallelism) {
+        checkParallelism(targetParallelism, vertexInputInfoMap);
+        Map<Integer, List<IndexRange>> consumedPartitionToSubpartitionRanges = 
new HashMap<>();
+        vertexInputInfoMap
+                .get(inputInfo.getResultId())
+                .getExecutionVertexInputInfos()
+                .forEach(
+                        info ->
+                                info.getConsumedSubpartitionGroups()
+                                        .forEach(
+                                                (partitionRange, 
subpartitionRange) -> {
+                                                    for (int i = 
partitionRange.getStartIndex();
+                                                            i <= 
partitionRange.getEndIndex();
+                                                            ++i) {
+                                                        
consumedPartitionToSubpartitionRanges
+                                                                
.computeIfAbsent(
+                                                                        i, k 
-> new ArrayList<>())
+                                                                
.add(subpartitionRange);
+                                                    }
+                                                }));
+        Set<Integer> partitionIndex =
+                IntStream.rangeClosed(0, inputInfo.getNumPartitions() - 1)
+                        .boxed()
+                        .collect(Collectors.toSet());
+        IndexRange subpartitionRange = new IndexRange(0, 
inputInfo.getNumSubpartitions(0) - 1);
+        
assertThat(consumedPartitionToSubpartitionRanges.keySet()).isEqualTo(partitionIndex);
+        consumedPartitionToSubpartitionRanges
+                .values()
+                .forEach(
+                        subpartitionRanges -> {
+                            List<IndexRange> mergedRange = 
mergeIndexRanges(subpartitionRanges);
+                            assertThat(mergedRange.size()).isEqualTo(1);
+                            
assertThat(mergedRange.get(0)).isEqualTo(subpartitionRange);
+                        });
+    }
+
+    public static void checkCorrectnessForCorrelatedInputs(
+            Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfoMap,
+            List<BlockingInputInfo> inputInfos,
+            int targetParallelism,
+            int numSubpartitions) {
+        checkParallelism(targetParallelism, vertexInputInfoMap);
+        Map<Integer, List<BlockingInputInfo>> inputInfosGroupByTypeNumber =
+                inputInfos.stream()
+                        
.collect(Collectors.groupingBy(BlockingInputInfo::getInputTypeNumber));
+
+        Map<Integer, List<JobVertexInputInfo>> 
vertexInputInfosGroupByTypeNumber =
+                inputInfosGroupByTypeNumber.entrySet().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        Map.Entry::getKey,
+                                        e ->
+                                                e.getValue().stream()
+                                                        .map(
+                                                                v ->
+                                                                        
vertexInputInfoMap.get(
+                                                                               
 v.getResultId()))
+                                                        
.collect(Collectors.toList())));
+
+        Map<JobVertexInputInfo, Integer> vertexInputInfoToNumPartitionsMap =
+                inputInfosGroupByTypeNumber.values().stream()
+                        .flatMap(List::stream)
+                        .collect(
+                                Collectors.toMap(
+                                        v -> 
vertexInputInfoMap.get(v.getResultId()),
+                                        BlockingInputInfo::getNumPartitions));
+        processPairs(
+                new ArrayList<>(vertexInputInfosGroupByTypeNumber.values()),
+                0,
+                new ArrayList<>(),
+                vertexInputInfoToNumPartitionsMap,
+                numSubpartitions);
+    }
+
+    private static void processPairs(

Review Comment:
   IIUC, it only does verification if there are two type numbers, i.e. when 
lists.size() == 2.
   In other cases, the verification in 
`checkCorrectnessForConsumedSubpartitionRanges` will never be invoked.
   
   If so, maybe simplify its logical to 
`checkCorrectnessForCorrelatedInputs(inputsWithTypeNumber1, 
inputsWithTypeNumber2, vertexInputInfoToNumPartitionsMap, numSubpartitions)`?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/util/PointwiseVertexInputInfoComputerTest.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.IndexRange;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.scheduler.adaptivebatch.BlockingInputInfo;
+import 
org.apache.flink.runtime.scheduler.adaptivebatch.VertexInputInfoComputerTestUtil;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.VertexInputInfoComputerTestUtil.checkConsumedDataVolumePerSubtask;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.VertexInputInfoComputerTestUtil.checkConsumedSubpartitionGroups;
+import static 
org.apache.flink.runtime.scheduler.adaptivebatch.VertexInputInfoComputerTestUtil.checkCorrectnessForNonCorrelatedInput;
+
+/** Tests for {@link PointwiseVertexInputInfoComputer}. */
+class PointwiseVertexInputInfoComputerTest {
+
+    @Test
+    void testComputeNormalInput() {
+        PointwiseVertexInputInfoComputer computer = 
createPointwiseVertexInputInfoComputer();
+        List<BlockingInputInfo> inputInfos = createBlockingInputInfos(2, 
List.of(), false);
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs =
+                computer.compute(inputInfos, 2, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfos.get(0), 
2);
+        checkConsumedDataVolumePerSubtask(new long[] {3L, 3L}, inputInfos, 
vertexInputs);
+
+        // with different parallelism
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs2 =
+                computer.compute(inputInfos, 3, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs2, 
inputInfos.get(0), 3);
+        checkConsumedDataVolumePerSubtask(new long[] {2L, 2L, 2L}, inputInfos, 
vertexInputs2);
+    }
+
+    @Test
+    void testComputeSkewedInputsWithDifferentSkewedPartitions() {
+        PointwiseVertexInputInfoComputer computer = 
createPointwiseVertexInputInfoComputer();
+        List<BlockingInputInfo> inputInfosWithDifferentSkewedPartitions = new 
ArrayList<>();
+        BlockingInputInfo inputInfo1 = createBlockingInputInfo(3, 3, 
List.of(0), false);
+        BlockingInputInfo inputInfo2 = createBlockingInputInfo(3, 3, 
List.of(1), false);
+        inputInfosWithDifferentSkewedPartitions.add(inputInfo1);
+        inputInfosWithDifferentSkewedPartitions.add(inputInfo2);
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs =
+                computer.compute(inputInfosWithDifferentSkewedPartitions, 3, 
10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo1, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {10L, 10L, 16L}, List.of(inputInfo1), vertexInputs);
+
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo2, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {13L, 10L, 13L}, List.of(inputInfo2), vertexInputs);
+    }
+
+    @Test
+    void testComputeSkewedInputsWithDifferentNumPartitions() {
+        PointwiseVertexInputInfoComputer computer = 
createPointwiseVertexInputInfoComputer();
+        List<BlockingInputInfo> inputInfosWithDifferentNumPartitions = new 
ArrayList<>();
+        BlockingInputInfo inputInfo1 = createBlockingInputInfo(3, 3, 
List.of(1), false);
+        BlockingInputInfo inputInfo2 = createBlockingInputInfo(2, 3, 
List.of(1), false);
+        inputInfosWithDifferentNumPartitions.add(inputInfo1);
+        inputInfosWithDifferentNumPartitions.add(inputInfo2);
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs =
+                computer.compute(inputInfosWithDifferentNumPartitions, 3, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo1, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {13L, 10L, 13L}, List.of(inputInfo1), vertexInputs);
+
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo2, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {13L, 10L, 10L}, List.of(inputInfo2), vertexInputs);
+    }
+
+    @Test
+    void testComputeSkewedInputsWithDifferentNumSubpartitions() {
+        PointwiseVertexInputInfoComputer computer = 
createPointwiseVertexInputInfoComputer();
+        List<BlockingInputInfo> inputInfosWithDifferentNumSubpartitions = new 
ArrayList<>();
+        BlockingInputInfo inputInfo1 = createBlockingInputInfo(3, 3, 
List.of(1), false);
+        BlockingInputInfo inputInfo2 = createBlockingInputInfo(3, 5, 
List.of(1), false);
+        inputInfosWithDifferentNumSubpartitions.add(inputInfo1);
+        inputInfosWithDifferentNumSubpartitions.add(inputInfo2);
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs =
+                computer.compute(inputInfosWithDifferentNumSubpartitions, 3, 
10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo1, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {13L, 10L, 13L}, List.of(inputInfo1), vertexInputs);
+
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfo2, 3);
+        checkConsumedDataVolumePerSubtask(
+                new long[] {25L, 20L, 15L}, List.of(inputInfo2), vertexInputs);
+    }
+
+    @Test
+    void testComputeInputWithIntraCorrelation() {
+        PointwiseVertexInputInfoComputer computer = 
createPointwiseVertexInputInfoComputer();
+        List<BlockingInputInfo> inputInfos = createBlockingInputInfos(3, 
List.of(), true);
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs =
+                computer.compute(inputInfos, 3, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs, inputInfos.get(0), 
3);
+        checkConsumedSubpartitionGroups(
+                List.of(
+                        Map.of(new IndexRange(0, 0), new IndexRange(0, 2)),
+                        Map.of(new IndexRange(1, 1), new IndexRange(0, 2)),
+                        Map.of(new IndexRange(2, 2), new IndexRange(0, 2))),
+                inputInfos,
+                vertexInputs);
+
+        // with different parallelism
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs2 =
+                computer.compute(inputInfos, 2, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs2, 
inputInfos.get(0), 2);
+        checkConsumedSubpartitionGroups(
+                List.of(
+                        Map.of(new IndexRange(0, 1), new IndexRange(0, 2)),
+                        Map.of(new IndexRange(2, 2), new IndexRange(0, 2))),
+                inputInfos,
+                vertexInputs2);
+
+        // with bigger parallelism and fallback to
+        // VertexInputInfoComputationUtils.computeVertexInputInfoForPointwise
+        Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputs3 =
+                computer.compute(inputInfos, 4, 10);
+        checkCorrectnessForNonCorrelatedInput(vertexInputs3, 
inputInfos.get(0), 4);
+        checkConsumedSubpartitionGroups(
+                List.of(
+                        Map.of(new IndexRange(0, 0), new IndexRange(0, 0)),
+                        Map.of(new IndexRange(0, 0), new IndexRange(1, 2)),

Review Comment:
   The data of partition 0 should not be split if the input has intra key 
correlation.
   
   However, the case with a bigger parallelism and intra key correlation seems 
to be an invalid case?



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