@@ -274,9 +274,14 @@ public void onNewJobVerticesAdded(List<JobVertex> newVertices, int pendingOperat
274
274
// 4. update json plan
275
275
getExecutionGraph ().setJsonPlan (JsonPlanGenerator .generatePlan (getJobGraph ()));
276
276
277
- // 5. try aggregate subpartition bytes
277
+ // 5. In broadcast join optimization, results might be written first with a hash
278
+ // method and then read with a broadcast method. Therefore, we need to update the
279
+ // result info:
280
+ // 1. Update the DistributionPattern to reflect the optimized data distribution.
281
+ // 2. Aggregate subpartition bytes when possible for efficiency.
278
282
for (JobVertex newVertex : newVertices ) {
279
283
for (JobEdge input : newVertex .getInputs ()) {
284
+ tryUpdateResultInfo (input .getSourceId (), input .getDistributionPattern ());
280
285
Optional .ofNullable (blockingResultInfos .get (input .getSourceId ()))
281
286
.ifPresent (this ::maybeAggregateSubpartitionBytes );
282
287
}
@@ -490,7 +495,8 @@ private void updateResultPartitionBytesMetrics(
490
495
result .getId (),
491
496
(ignored , resultInfo ) -> {
492
497
if (resultInfo == null ) {
493
- resultInfo = createFromIntermediateResult (result );
498
+ resultInfo =
499
+ createFromIntermediateResult (result , new HashMap <>());
494
500
}
495
501
resultInfo .recordPartitionInfo (
496
502
partitionId .getPartitionNumber (), partitionBytes );
@@ -500,6 +506,16 @@ private void updateResultPartitionBytesMetrics(
500
506
});
501
507
}
502
508
509
+ /**
510
+ * Aggregates subpartition bytes if all conditions are met. This method checks whether the
511
+ * result info instance is of type {@link AllToAllBlockingResultInfo}, whether all consumer
512
+ * vertices are created, and whether all consumer vertices are initialized. If these conditions
513
+ * are satisfied, the fine-grained statistic info will not be required by consumer vertices, and
514
+ * then we could aggregate the subpartition bytes.
515
+ *
516
+ * @param resultInfo the BlockingResultInfo instance to potentially aggregate subpartition bytes
517
+ * for.
518
+ */
503
519
private void maybeAggregateSubpartitionBytes (BlockingResultInfo resultInfo ) {
504
520
IntermediateResult intermediateResult =
505
521
getExecutionGraph ().getAllIntermediateResults ().get (resultInfo .getResultId ());
@@ -937,21 +953,24 @@ private static void resetDynamicParallelism(Iterable<JobVertex> vertices) {
937
953
}
938
954
}
939
955
940
- private static BlockingResultInfo createFromIntermediateResult (IntermediateResult result ) {
956
+ private static BlockingResultInfo createFromIntermediateResult (
957
+ IntermediateResult result , Map <Integer , long []> subpartitionBytesByPartitionIndex ) {
941
958
checkArgument (result != null );
942
959
// Note that for dynamic graph, different partitions in the same result have the same number
943
960
// of subpartitions.
944
961
if (result .getConsumingDistributionPattern () == DistributionPattern .POINTWISE ) {
945
962
return new PointwiseBlockingResultInfo (
946
963
result .getId (),
947
964
result .getNumberOfAssignedPartitions (),
948
- result .getPartitions ()[0 ].getNumberOfSubpartitions ());
965
+ result .getPartitions ()[0 ].getNumberOfSubpartitions (),
966
+ subpartitionBytesByPartitionIndex );
949
967
} else {
950
968
return new AllToAllBlockingResultInfo (
951
969
result .getId (),
952
970
result .getNumberOfAssignedPartitions (),
953
971
result .getPartitions ()[0 ].getNumberOfSubpartitions (),
954
- result .isBroadcast ());
972
+ result .isSingleSubpartitionContainsAllData (),
973
+ subpartitionBytesByPartitionIndex );
955
974
}
956
975
}
957
976
@@ -965,6 +984,45 @@ SpeculativeExecutionHandler getSpeculativeExecutionHandler() {
965
984
return speculativeExecutionHandler ;
966
985
}
967
986
987
+ /**
988
+ * Tries to update the result information for a given IntermediateDataSetID according to the
989
+ * specified DistributionPattern. This ensures consistency between the distribution pattern and
990
+ * the stored result information.
991
+ *
992
+ * <p>The result information is updated under the following conditions:
993
+ *
994
+ * <ul>
995
+ * <li>If the target pattern is ALL_TO_ALL and the current result info is POINTWISE, a new
996
+ * BlockingResultInfo is created and stored.
997
+ * <li>If the target pattern is POINTWISE and the current result info is ALL_TO_ALL, a
998
+ * conversion is similarly triggered.
999
+ * <li>Additionally, for ALL_TO_ALL patterns, the status of broadcast of the result info
1000
+ * should be updated.
1001
+ * </ul>
1002
+ *
1003
+ * @param id The ID of the intermediate dataset to update.
1004
+ * @param targetPattern The target distribution pattern to apply.
1005
+ */
1006
+ private void tryUpdateResultInfo (IntermediateDataSetID id , DistributionPattern targetPattern ) {
1007
+ if (blockingResultInfos .containsKey (id )) {
1008
+ BlockingResultInfo resultInfo = blockingResultInfos .get (id );
1009
+ IntermediateResult result = getExecutionGraph ().getAllIntermediateResults ().get (id );
1010
+
1011
+ if ((targetPattern == DistributionPattern .ALL_TO_ALL && resultInfo .isPointwise ())
1012
+ || (targetPattern == DistributionPattern .POINTWISE
1013
+ && !resultInfo .isPointwise ())) {
1014
+
1015
+ BlockingResultInfo newInfo =
1016
+ createFromIntermediateResult (
1017
+ result , resultInfo .getSubpartitionBytesByPartitionIndex ());
1018
+
1019
+ blockingResultInfos .put (id , newInfo );
1020
+ } else if (resultInfo instanceof AllToAllBlockingResultInfo ) {
1021
+ ((AllToAllBlockingResultInfo ) resultInfo ).setBroadcast (result .isBroadcast ());
1022
+ }
1023
+ }
1024
+ }
1025
+
968
1026
private class DefaultBatchJobRecoveryContext implements BatchJobRecoveryContext {
969
1027
970
1028
private final FailoverStrategy restartStrategyOnResultConsumable =
0 commit comments