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_r367778123
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 ##########
 @@ -485,76 +481,151 @@ 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;
+
+                       final CompletableFuture<PendingCheckpoint> 
pendingCheckpointCompletableFuture =
+                               initializeCheckpoint(props, 
externalSavepointLocation)
+                                       
.thenApplyAsync((checkpointIdAndStorageLocation) -> createPendingCheckpoint(
+                                               timestamp,
+                                               props,
+                                               ackTasks,
+                                               isPeriodic,
+                                               
checkpointIdAndStorageLocation.checkpointId,
+                                               
checkpointIdAndStorageLocation.checkpointStorageLocation,
+                                               onCompletionPromise), timer);
+
+                       pendingCheckpointCompletableFuture
+                               .thenCompose(this::snapshotMasterState)
+                               .whenComplete((ignored, throwable) -> {
+                                       final PendingCheckpoint checkpoint =
+                                               
FutureUtils.getWithoutException(pendingCheckpointCompletableFuture);
+
+                                       if (throwable == null && checkpoint != 
null && !checkpoint.isDiscarded()) {
+                                               // no exception, no discarding, 
everything is OK
+                                               snapshotTaskState(
+                                                       timestamp,
+                                                       
checkpoint.getCheckpointId(),
+                                                       
checkpoint.getCheckpointStorageLocation(),
+                                                       props,
+                                                       executions,
+                                                       advanceToEndOfTime);
+                                               onTriggerSuccess();
+                                       } else {
+                                               // error handling must be 
executed in timer thread
+                                               timer.execute(() -> {
+                                                       // the initialization 
might not be finished yet
+                                                       if (checkpoint == null) 
{
+                                                               
onTriggerFailure(onCompletionPromise, throwable);
+                                                       } else {
+                                                               
onTriggerFailure(checkpoint, throwable);
+                                                       }
+                                               });
+                                       }
+                               });
 
 Review comment:
   > For example isn't the whole whenComplete executed already in the timer 
thread, since CheckpointCoordinator#snapshotMasterState returns a future that 
will be completed from the timer thread?
   
   Yes, all these chain calls should be executed in timer thread, except the 
error handling part. The exceptional future might be completed in IO thread in 
`initializeCheckpoint`. 
   
   > Maybe always explicitly use the executor, timer parameter when 
scheduling/chaining something, so that the executor is clearly visible? 
thenCompose(...) -> thenComposeAsync(..., timer/executor)? 
   
   Yes, it is definitely clearer. Actually I had implemented it like this at 
the beginning. But it's not efficient for most scenarios, they are already in 
timer thread. Especially for the final implementation (replace timer thread by 
main thread), it's an unnecessary burden for main thread.
   
   I think the perfect solution here is checking the current thread. If it's 
already in timer (or main) thread, it would be executed directly. Otherwise it 
would be scheduled in timer (or main) thread. However it's not easy (or a bit 
hacky) to do so for now, because timer is a `ScheduledExecutor`. My plan is to 
implement this after replacing the timer thread by main thread, through 
providing a method maybe named 
`ComponentMainThreadExecutor#isRunningInMainThread`.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to