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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/DefaultVertexParallelismDecider.java:
##########
@@ -150,6 +193,202 @@ private int calculateParallelism(List<BlockingResultInfo> 
consumedResults) {
         return parallelism;
     }
 
+    private ParallelismAndInputInfos loadBalanceForAllToAllInputs(
+            List<BlockingResultInfo> inputs, int parallelism) {
+        checkArgument(parallelism == ExecutionConfig.PARALLELISM_DEFAULT);
+        checkArgument(!inputs.isEmpty());
+        inputs.forEach(resultInfo -> checkState(!resultInfo.isPointwise()));
+
+        final List<BlockingResultInfo> nonBroadcastInputs = 
getNonBroadcastResultInfos(inputs);
+        long broadcastBytes = getReasonableBroadcastBytes(inputs);
+        long nonBroadcastBytes = getNonBroadcastBytes(inputs);
+
+        int subpartitionNum = checkAndGetSubpartitionNum(nonBroadcastInputs);
+
+        long nonBroadcastBytesPerTaskLimit = dataVolumePerTask - 
broadcastBytes;
+        long[] nonBroadcastBytesBySubpartition = new long[subpartitionNum];
+        Arrays.fill(nonBroadcastBytesBySubpartition, 0L);
+        for (BlockingResultInfo resultInfo : nonBroadcastInputs) {
+            List<Long> subpartitionBytes =
+                    ((AllToAllBlockingResultInfo) 
resultInfo).getAggregatedSubpartitionBytes();
+            for (int i = 0; i < subpartitionNum; ++i) {
+                nonBroadcastBytesBySubpartition[i] += subpartitionBytes.get(i);
+            }
+        }
+
+        // compute subpartition ranges
+        List<IndexRange> subpartitionRanges =
+                computeSubpartitionRanges(
+                        nonBroadcastBytesBySubpartition, 
nonBroadcastBytesPerTaskLimit);
+
+        if (subpartitionRanges.size() < minParallelism) {
+            long minSubpartitionBytes =
+                    
Arrays.stream(nonBroadcastBytesBySubpartition).min().getAsLong();
+            // find a legal limit so that the computed parallelism >= 
minParallelism
+            long adjustLimit =
+                    BisectionSearchUtils.findMaxLegalValue(
+                            value ->
+                                    
computeParallelism(nonBroadcastBytesBySubpartition, value)
+                                            >= minParallelism,
+                            minSubpartitionBytes,
+                            nonBroadcastBytesPerTaskLimit);
+
+            // the smaller the limit, the more even the distribution
+            final long expectedParallelism =
+                    computeParallelism(nonBroadcastBytesBySubpartition, 
adjustLimit);
+            adjustLimit =
+                    BisectionSearchUtils.findMinLegalValue(
+                            value ->
+                                    
computeParallelism(nonBroadcastBytesBySubpartition, value)
+                                            <= expectedParallelism,
+                            minSubpartitionBytes,
+                            adjustLimit);
+
+            subpartitionRanges =
+                    computeSubpartitionRanges(nonBroadcastBytesBySubpartition, 
adjustLimit);
+        } else if (subpartitionRanges.size() > maxParallelism) {
+            // find a legal limit so that the computed parallelism <= 
minParallelism
+            long adjustLimit =
+                    BisectionSearchUtils.findMinLegalValue(
+                            value ->
+                                    
computeParallelism(nonBroadcastBytesBySubpartition, value)
+                                            <= maxParallelism,
+                            nonBroadcastBytesPerTaskLimit,
+                            nonBroadcastBytes);
+
+            subpartitionRanges =
+                    computeSubpartitionRanges(nonBroadcastBytesBySubpartition, 
adjustLimit);
+        }
+
+        checkState(isLegalParallelism(subpartitionRanges.size()));
+        return createParallelismAndInputInfos(inputs, subpartitionRanges);
+    }
+
+    private boolean isLegalParallelism(int parallelism) {
+        return parallelism >= minParallelism && parallelism <= maxParallelism;
+    }
+
+    private static int checkAndGetSubpartitionNum(List<BlockingResultInfo> 
inputs) {
+        final Set<Integer> subpartitionNumSet =
+                inputs.stream()
+                        .flatMap(
+                                resultInfo ->
+                                        IntStream.range(0, 
resultInfo.getNumPartitions())
+                                                .boxed()
+                                                
.map(resultInfo::getNumSubpartitions))
+                        .collect(Collectors.toSet());
+        // all partitions have the same subpartition num
+        checkState(subpartitionNumSet.size() == 1);
+        return subpartitionNumSet.iterator().next();
+    }
+
+    private static ParallelismAndInputInfos createParallelismAndInputInfos(
+            List<BlockingResultInfo> inputs, List<IndexRange> 
subpartitionRanges) {
+
+        final Map<IntermediateDataSetID, JobVertexInputInfo> vertexInputInfos 
= new HashMap<>();
+        inputs.forEach(
+                resultInfo -> {
+                    int sourceParallelism = resultInfo.getNumPartitions();
+                    IndexRange partitionRange = new IndexRange(0, 
sourceParallelism - 1);
+
+                    List<ExecutionVertexInputInfo> executionVertexInputInfos = 
new ArrayList<>();
+                    for (int i = 0; i < subpartitionRanges.size(); ++i) {
+                        IndexRange subpartitionRange;
+                        if (resultInfo.isBroadcast()) {
+                            subpartitionRange = new IndexRange(0, 0);
+                        } else {
+                            subpartitionRange = subpartitionRanges.get(i);
+                        }
+                        ExecutionVertexInputInfo executionVertexInputInfo =
+                                new ExecutionVertexInputInfo(i, 
partitionRange, subpartitionRange);
+                        
executionVertexInputInfos.add(executionVertexInputInfo);
+                    }
+
+                    vertexInputInfos.put(
+                            resultInfo.getResultId(),
+                            new JobVertexInputInfo(executionVertexInputInfos));
+                });
+        return new ParallelismAndInputInfos(subpartitionRanges.size(), 
vertexInputInfos);
+    }
+
+    private static List<IndexRange> computeSubpartitionRanges(long[] nums, 
long limit) {
+        List<IndexRange> subpartitionRanges = new ArrayList<>();
+        long tmpSum = 0;
+        int startIndex = 0;
+        for (int i = 0; i < nums.length; ++i) {
+            long num = nums[i];
+            if (tmpSum == 0 || tmpSum + num <= limit) {
+                tmpSum += num;
+            } else {
+                subpartitionRanges.add(new IndexRange(startIndex, i - 1));
+                startIndex = i;
+                tmpSum = num;
+            }
+        }
+        subpartitionRanges.add(new IndexRange(startIndex, nums.length - 1));
+        return subpartitionRanges;
+    }
+
+    private static int computeParallelism(long[] nums, long limit) {
+

Review Comment:
   useless empty line



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