ifndef-SleePy commented on a change in pull request #10332: 
[FLINK-13905][checkpointing] Separate checkpoint triggering into several 
asynchronous stages
URL: https://github.com/apache/flink/pull/10332#discussion_r366676607
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 ##########
 @@ -484,77 +484,158 @@ public boolean isShutdown() {
                        CheckpointProperties props,
                        @Nullable String externalSavepointLocation,
                        boolean isPeriodic,
-                       boolean advanceToEndOfTime) throws CheckpointException {
+                       boolean advanceToEndOfTime) {
 
                if (advanceToEndOfTime && !(props.isSynchronous() && 
props.isSavepoint())) {
-                       throw new IllegalArgumentException("Only synchronous 
savepoints are allowed to advance the watermark to MAX.");
+                       return FutureUtils.completedExceptionally(new 
IllegalArgumentException(
+                               "Only synchronous savepoints are allowed to 
advance the watermark to MAX."));
                }
 
-               // make some eager pre-checks
+               final CompletableFuture<CompletedCheckpoint> 
onCompletionPromise =
+                       new CompletableFuture<>();
                synchronized (lock) {
-                       preCheckBeforeTriggeringCheckpoint(isPeriodic, 
props.forceCheckpoint());
-               }
-
-               // check if all tasks that we need to trigger are running.
-               // if not, abort the checkpoint
-               Execution[] executions = new Execution[tasksToTrigger.length];
-               for (int i = 0; i < tasksToTrigger.length; i++) {
-                       Execution ee = 
tasksToTrigger[i].getCurrentExecutionAttempt();
-                       if (ee == null) {
-                               LOG.info("Checkpoint triggering task {} of job 
{} is not being executed at the moment. Aborting checkpoint.",
-                                               
tasksToTrigger[i].getTaskNameWithSubtaskIndex(),
-                                               job);
-                               throw new 
CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
-                       } else if (ee.getState() == ExecutionState.RUNNING) {
-                               executions[i] = ee;
-                       } else {
-                               LOG.info("Checkpoint triggering task {} of job 
{} is not in state {} but {} instead. Aborting checkpoint.",
-                                               
tasksToTrigger[i].getTaskNameWithSubtaskIndex(),
-                                               job,
-                                               ExecutionState.RUNNING,
-                                               ee.getState());
-                               throw new 
CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+                       if (isTriggering || !triggerRequestQueue.isEmpty()) {
+                               // we can't trigger checkpoint directly if 
there is a trigger request being processed
+                               // or queued
+                               triggerRequestQueue.add(new 
CheckpointTriggerRequest(
+                                       timestamp,
+                                       props,
+                                       externalSavepointLocation,
+                                       isPeriodic,
+                                       advanceToEndOfTime,
+                                       onCompletionPromise));
+                               return onCompletionPromise;
                        }
                }
+               startTriggeringCheckpoint(
+                       timestamp,
+                       props,
+                       externalSavepointLocation,
+                       isPeriodic,
+                       advanceToEndOfTime,
+                       onCompletionPromise);
+               return onCompletionPromise;
+       }
 
-               // next, check if all tasks that need to acknowledge the 
checkpoint are running.
-               // if not, abort the checkpoint
-               Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new 
HashMap<>(tasksToWaitFor.length);
+       private void startTriggeringCheckpoint(
+               long timestamp,
+               CheckpointProperties props,
+               @Nullable String externalSavepointLocation,
+               boolean isPeriodic,
+               boolean advanceToEndOfTime,
+               CompletableFuture<CompletedCheckpoint> onCompletionPromise) {
 
-               for (ExecutionVertex ev : tasksToWaitFor) {
-                       Execution ee = ev.getCurrentExecutionAttempt();
-                       if (ee != null) {
-                               ackTasks.put(ee.getAttemptId(), ev);
-                       } else {
-                               LOG.info("Checkpoint acknowledging task {} of 
job {} is not being executed at the moment. Aborting checkpoint.",
-                                               
ev.getTaskNameWithSubtaskIndex(),
-                                               job);
-                               throw new 
CheckpointException(CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
+               try {
+                       // make some eager pre-checks
+                       synchronized (lock) {
+                               preCheckBeforeTriggeringCheckpoint(isPeriodic, 
props.forceCheckpoint());
                        }
-               }
 
-               // we will actually trigger this checkpoint!
+                       final Execution[] executions = getTriggerExecutions();
+                       final Map<ExecutionAttemptID, ExecutionVertex> ackTasks 
= getAckTasks();
+
+                       // we will actually trigger this checkpoint!
+                       Preconditions.checkState(!isTriggering);
+                       isTriggering = true;
+
+                       // keep a reference outside the async calls
+                       final AtomicReference<PendingCheckpoint> 
currentCheckpointRef = new AtomicReference<>();
+                       initializeCheckpoint(props, externalSavepointLocation)
+                               
.thenComposeAsync((checkpointIdAndStorageLocation) -> {
+                                       synchronized (lock) {
+                                               // since we haven't created the 
PendingCheckpoint yet, we need to check the
+                                               // global state here.
+                                               // otherwise the 
PendingCheckpoint would be discarded if the global state
+                                               // changed (shutdown or 
periodic scheduling stopped)
+                                               if (shutdown) {
+                                                       throw new 
CompletionException(new 
CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN));
+                                               }
+                                               if (isPeriodic && 
!periodicScheduling) {
+                                                       throw new 
CompletionException(new 
CheckpointException(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN));
+                                               }
+                                       }
+                                       final PendingCheckpoint 
pendingCheckpoint = createPendingCheckpoint(
+                                               timestamp,
+                                               props,
+                                               ackTasks,
+                                               isPeriodic,
+                                               
checkpointIdAndStorageLocation.checkpointId,
+                                               
checkpointIdAndStorageLocation.checkpointStorageLocation,
+                                               onCompletionPromise);
+                                       
currentCheckpointRef.set(pendingCheckpoint);
+
+                                       LOG.info("Triggering checkpoint {} @ {} 
for job {}.",
+                                               
checkpointIdAndStorageLocation.checkpointId, timestamp, job);
+                                       return 
snapshotMasterState(pendingCheckpoint);
+                               }, timer)
+                               .thenRun(() -> {
+                                       // it must be executed in timer thread.
+                                       // we could add a checking after 
replace timer with main thread
+                                       final PendingCheckpoint checkpoint = 
currentCheckpointRef.get();
+                                       checkState(!checkpoint.isDiscarded());
+                                       snapshotTaskState(
+                                               timestamp,
+                                               checkpoint.getCheckpointId(),
+                                               
checkpoint.getCheckpointStorageLocation(),
+                                               props,
+                                               executions,
+                                               advanceToEndOfTime);
+                                       onTriggerSuccess();
+                               })
+                               .whenCompleteAsync((ignored, throwable) -> {
+                                       // the error handling has to be 
executed in timer thread
+                                       if (throwable != null) {
+                                               // the initialization might not 
be finished yet
+                                               if (currentCheckpointRef.get() 
== null) {
+                                                       
onTriggerFailure(onCompletionPromise, throwable);
+                                               } else {
+                                                       
onTriggerFailure(currentCheckpointRef.get(), throwable);
+                                               }
+                                       }
+                               }, timer);
 
 Review comment:
   BTW, I replaced the `checkState(!checkpoint.isDiscarded());` by 
`!checkpoint.isDiscarded()`. Because I found there is a possibility that 
`PendingCheckpoint` might be cancelled during the asynchronous call if there is 
no master hook (otherwise `snapshotMasterState` would check state of 
`PendingCheckpoint`).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to