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_r368263483
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -485,76 +481,152 @@ 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)
+ .whenCompleteAsync(
+ (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 {
+ // the
initialization might not be finished yet
+ if (checkpoint
== null) {
+
onTriggerFailure(onCompletionPromise, throwable);
+ } else {
+
onTriggerFailure(checkpoint, throwable);
+ }
+ }
+ },
+ timer);
+ } catch (Throwable throwable) {
+ onTriggerFailure(onCompletionPromise, throwable);
Review comment:
Yes, `startTriggeringCheckpoint` must be executed in timer thread. We can
guarantee that now.
----------------------------------------------------------------
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