1996fanrui commented on a change in pull request #18852: URL: https://github.com/apache/flink/pull/18852#discussion_r818443053
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ########## @@ -565,20 +563,46 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { request.props, checkpointInfo.f0, request.isPeriodic, - checkpointInfo.f1.checkpointId, - checkpointInfo.f1.checkpointStorageLocation, + checkpointInfo.f1, request.getOnCompletionFuture()), timer); final CompletableFuture<?> coordinatorCheckpointsComplete = - pendingCheckpointCompletableFuture.thenComposeAsync( - (pendingCheckpoint) -> - OperatorCoordinatorCheckpoints - .triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( - coordinatorsToCheckpoint, - pendingCheckpoint, - timer), - timer); + pendingCheckpointCompletableFuture + .thenApplyAsync( + pendingCheckpoint -> { + try { + CheckpointStorageLocation checkpointStorageLocation = + initializeCheckpointLocation( + pendingCheckpoint.getCheckpointID(), + request.props, + request.externalSavepointLocation, + initializeBaseLocations); + return Tuple2.of( + pendingCheckpoint, checkpointStorageLocation); + } catch (Throwable e) { + throw new CompletionException(e); + } + }, + executor) + .thenApplyAsync( + (checkpointInfo) -> { + PendingCheckpoint pendingCheckpoint = checkpointInfo.f0; + synchronized (lock) { + pendingCheckpoint.setCheckpointTargetLocation( + checkpointInfo.f1); + } + return pendingCheckpoint; + }, + timer) + .thenComposeAsync( + (pendingCheckpoint) -> + OperatorCoordinatorCheckpoints + .triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( + coordinatorsToCheckpoint, + pendingCheckpoint, + timer), + timer); Review comment: Separation is clearer. I have squashed them. -- 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