-
Notifications
You must be signed in to change notification settings - Fork 13.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-36067][runtime] Support optimize stream graph based on input info. #25790
base: master
Are you sure you want to change the base?
Changes from all commits
8a40335
5ac62c5
ecc3183
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,6 +35,14 @@ public interface IntermediateResultInfo { | |
*/ | ||
boolean isBroadcast(); | ||
|
||
/** | ||
* Indicates whether every downstream consumer needs to consume all produced sub-partitions. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see this method seems to be relate to broadcasts - it would be good to explain how. |
||
* | ||
* @return true if every downstream consumer needs to consume all produced sub-partitions, false | ||
* otherwise. | ||
*/ | ||
boolean isEveryConsumerConsumeAllSubPartitions(); | ||
|
||
/** | ||
* Whether it is a pointwise result. | ||
* | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -273,6 +273,16 @@ public void onNewJobVerticesAdded(List<JobVertex> newVertices, int pendingOperat | |
|
||
// 4. update json plan | ||
getExecutionGraph().setJsonPlan(JsonPlanGenerator.generatePlan(getJobGraph())); | ||
|
||
// 5. update the DistributionPattern of the upstream results consumed by the newly created | ||
// JobVertex and aggregate subpartition bytes. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. aggregate subpartition bytes. -> aggregate subpartition bytes when possible. I would find is useful to describe in more detail was aggregate subpartition bytes means and why we can perform this optimization in some circumstances. |
||
for (JobVertex newVertex : newVertices) { | ||
for (JobEdge input : newVertex.getInputs()) { | ||
tryUpdateResultInfo(input.getSourceId(), input.getDistributionPattern()); | ||
Optional.ofNullable(blockingResultInfos.get(input.getSourceId())) | ||
.ifPresent(this::maybeAggregateSubpartitionBytes); | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
|
@@ -482,15 +492,29 @@ private void updateResultPartitionBytesMetrics( | |
result.getId(), | ||
(ignored, resultInfo) -> { | ||
if (resultInfo == null) { | ||
resultInfo = createFromIntermediateResult(result); | ||
resultInfo = | ||
createFromIntermediateResult(result, new HashMap<>()); | ||
} | ||
resultInfo.recordPartitionInfo( | ||
partitionId.getPartitionNumber(), partitionBytes); | ||
maybeAggregateSubpartitionBytes(resultInfo); | ||
return resultInfo; | ||
}); | ||
}); | ||
} | ||
|
||
private void maybeAggregateSubpartitionBytes(BlockingResultInfo resultInfo) { | ||
IntermediateResult intermediateResult = | ||
getExecutionGraph().getAllIntermediateResults().get(resultInfo.getResultId()); | ||
|
||
if (intermediateResult.areAllConsumerVerticesCreated() | ||
&& intermediateResult.getConsumerVertices().stream() | ||
.map(this::getExecutionJobVertex) | ||
.allMatch(ExecutionJobVertex::isInitialized)) { | ||
resultInfo.aggregateSubpartitionBytes(); | ||
} | ||
} | ||
|
||
@Override | ||
public void allocateSlotsAndDeploy(final List<ExecutionVertexID> verticesToDeploy) { | ||
List<ExecutionVertex> executionVertices = | ||
|
@@ -657,6 +681,7 @@ public void initializeVerticesIfPossible() { | |
parallelismAndInputInfos.getJobVertexInputInfos(), | ||
createTimestamp); | ||
newlyInitializedJobVertices.add(jobVertex); | ||
consumedResultsInfo.get().forEach(this::maybeAggregateSubpartitionBytes); | ||
} | ||
} | ||
} | ||
|
@@ -909,21 +934,24 @@ private static void resetDynamicParallelism(Iterable<JobVertex> vertices) { | |
} | ||
} | ||
|
||
private static BlockingResultInfo createFromIntermediateResult(IntermediateResult result) { | ||
private static BlockingResultInfo createFromIntermediateResult( | ||
IntermediateResult result, Map<Integer, long[]> subpartitionBytesByPartitionIndex) { | ||
checkArgument(result != null); | ||
// Note that for dynamic graph, different partitions in the same result have the same number | ||
// of subpartitions. | ||
if (result.getConsumingDistributionPattern() == DistributionPattern.POINTWISE) { | ||
return new PointwiseBlockingResultInfo( | ||
result.getId(), | ||
result.getNumberOfAssignedPartitions(), | ||
result.getPartitions()[0].getNumberOfSubpartitions()); | ||
result.getPartitions()[0].getNumberOfSubpartitions(), | ||
subpartitionBytesByPartitionIndex); | ||
} else { | ||
return new AllToAllBlockingResultInfo( | ||
result.getId(), | ||
result.getNumberOfAssignedPartitions(), | ||
result.getPartitions()[0].getNumberOfSubpartitions(), | ||
result.isBroadcast()); | ||
result.isBroadcast(), | ||
subpartitionBytesByPartitionIndex); | ||
} | ||
} | ||
|
||
|
@@ -937,6 +965,26 @@ SpeculativeExecutionHandler getSpeculativeExecutionHandler() { | |
return speculativeExecutionHandler; | ||
} | ||
|
||
private void tryUpdateResultInfo(IntermediateDataSetID id, DistributionPattern targetPattern) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. could you add some comments detailing the algorithm and its benefits. |
||
if (blockingResultInfos.containsKey(id)) { | ||
BlockingResultInfo resultInfo = blockingResultInfos.get(id); | ||
IntermediateResult result = getExecutionGraph().getAllIntermediateResults().get(id); | ||
|
||
if ((targetPattern == DistributionPattern.ALL_TO_ALL && resultInfo.isPointwise()) | ||
|| (targetPattern == DistributionPattern.POINTWISE | ||
&& !resultInfo.isPointwise())) { | ||
|
||
BlockingResultInfo newInfo = | ||
createFromIntermediateResult( | ||
result, resultInfo.getSubpartitionBytesByPartitionIndex()); | ||
|
||
blockingResultInfos.put(id, newInfo); | ||
} else if (targetPattern == DistributionPattern.ALL_TO_ALL) { | ||
((AllToAllBlockingResultInfo) resultInfo).setBroadcast(result.isBroadcast()); | ||
} | ||
} | ||
} | ||
|
||
private class DefaultBatchJobRecoveryContext implements BatchJobRecoveryContext { | ||
|
||
private final FailoverStrategy restartStrategyOnResultConsumable = | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am confused by why this method is checking broadcasts - how does this relate to downstream consumers being able to consume the subpartitions