ifndef-SleePy commented on a change in pull request #12670:
URL: https://github.com/apache/flink/pull/12670#discussion_r440786956



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -538,51 +542,61 @@ private void 
startTriggeringCheckpoint(CheckpointTriggerRequest request) {
                                                                        
coordinatorsToCheckpoint, pendingCheckpoint, timer),
                                                        timer);
 
-                       FutureUtils.assertNoException(
-                               CompletableFuture.allOf(masterStatesComplete, 
coordinatorCheckpointsComplete)
-                                       .handleAsync(
-                                               (ignored, throwable) -> {
-                                                       final PendingCheckpoint 
checkpoint =
-                                                               
FutureUtils.getWithoutException(pendingCheckpointCompletableFuture);
-
-                                                       
Preconditions.checkState(
-                                                               checkpoint != 
null || throwable != null,
-                                                               "Either the 
pending checkpoint needs to be created or an error must have been occurred.");
-
-                                                       if (throwable != null) {
-                                                               // the 
initialization might not be finished yet
-                                                               if (checkpoint 
== null) {
-                                                                       
onTriggerFailure(request, throwable);
-                                                               } else {
-                                                                       
onTriggerFailure(checkpoint, throwable);
-                                                               }
+                       FutureUtils.waitForAll(asList(masterStatesComplete, 
coordinatorCheckpointsComplete))
+                               .handleAsync(
+                                       (ignored, throwable) -> {
+                                               final PendingCheckpoint 
checkpoint =
+                                                       
FutureUtils.getWithoutException(pendingCheckpointCompletableFuture);
+
+                                               Preconditions.checkState(
+                                                       checkpoint != null || 
throwable != null,
+                                                       "Either the pending 
checkpoint needs to be created or an error must have been occurred.");
+
+                                               if (throwable != null) {
+                                                       // the initialization 
might not be finished yet
+                                                       if (checkpoint == null) 
{
+                                                               
onTriggerFailure(request, throwable);
                                                        } else {
-                                                               if 
(checkpoint.isDiscarded()) {
-                                                                       
onTriggerFailure(
-                                                                               
checkpoint,
-                                                                               
new CheckpointException(
-                                                                               
        CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE,
-                                                                               
        checkpoint.getFailureCause()));
-                                                               } else {
-                                                                       // no 
exception, no discarding, everything is OK
-                                                                       final 
long checkpointId = checkpoint.getCheckpointId();
-                                                                       
snapshotTaskState(
-                                                                               
timestamp,
-                                                                               
checkpointId,
-                                                                               
checkpoint.getCheckpointStorageLocation(),
-                                                                               
request.props,
-                                                                               
executions,
-                                                                               
request.advanceToEndOfTime);
-
-                                                                       
coordinatorsToCheckpoint.forEach((ctx) -> 
ctx.afterSourceBarrierInjection(checkpointId));
-
-                                                                       
onTriggerSuccess();
-                                                               }
+                                                               
onTriggerFailure(checkpoint, throwable);
                                                        }
+                                               } else {
+                                                       if 
(checkpoint.isDiscarded()) {
+                                                               
onTriggerFailure(
+                                                                       
checkpoint,
+                                                                       new 
CheckpointException(
+                                                                               
CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE,
+                                                                               
checkpoint.getFailureCause()));
+                                                       } else {
+                                                               // no 
exception, no discarding, everything is OK
+                                                               final long 
checkpointId = checkpoint.getCheckpointId();
+                                                               
snapshotTaskState(
+                                                                       
timestamp,
+                                                                       
checkpointId,
+                                                                       
checkpoint.getCheckpointStorageLocation(),
+                                                                       
request.props,
+                                                                       
executions,
+                                                                       
request.advanceToEndOfTime);
+
+                                                               
coordinatorsToCheckpoint.forEach((ctx) -> 
ctx.afterSourceBarrierInjection(checkpointId));
+
+                                                               
onTriggerSuccess();
+                                                       }
+                                               }
 
-                                                       return null;
-                                               },
-                                               timer));
+                                               return null;
+                                       },
+                                       timer)
+                               .whenComplete((unused, error) -> {
+                                       if (error != null) {
+                                               if (!isShutdown()) {
+                                                       
failureManager.handleJobLevelCheckpointException(new 
CheckpointException(EXCEPTION, error), Optional.empty());

Review comment:
       It's a serious issue if it gets here. `CheckpointCoordinator` might be 
never recover even we fail the job. The state of `isTrigger` might not be 
reset, and it couldn't be reset forever. So we should reset the `isTriggering` 
to false first and then call the `failureManager`.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -538,51 +542,61 @@ private void 
startTriggeringCheckpoint(CheckpointTriggerRequest request) {
                                                                        
coordinatorsToCheckpoint, pendingCheckpoint, timer),
                                                        timer);
 
-                       FutureUtils.assertNoException(
-                               CompletableFuture.allOf(masterStatesComplete, 
coordinatorCheckpointsComplete)
-                                       .handleAsync(
-                                               (ignored, throwable) -> {
-                                                       final PendingCheckpoint 
checkpoint =
-                                                               
FutureUtils.getWithoutException(pendingCheckpointCompletableFuture);
-
-                                                       
Preconditions.checkState(
-                                                               checkpoint != 
null || throwable != null,
-                                                               "Either the 
pending checkpoint needs to be created or an error must have been occurred.");
-
-                                                       if (throwable != null) {
-                                                               // the 
initialization might not be finished yet
-                                                               if (checkpoint 
== null) {
-                                                                       
onTriggerFailure(request, throwable);
-                                                               } else {
-                                                                       
onTriggerFailure(checkpoint, throwable);
-                                                               }
+                       FutureUtils.waitForAll(asList(masterStatesComplete, 
coordinatorCheckpointsComplete))
+                               .handleAsync(
+                                       (ignored, throwable) -> {
+                                               final PendingCheckpoint 
checkpoint =
+                                                       
FutureUtils.getWithoutException(pendingCheckpointCompletableFuture);
+
+                                               Preconditions.checkState(
+                                                       checkpoint != null || 
throwable != null,
+                                                       "Either the pending 
checkpoint needs to be created or an error must have been occurred.");
+
+                                               if (throwable != null) {
+                                                       // the initialization 
might not be finished yet
+                                                       if (checkpoint == null) 
{
+                                                               
onTriggerFailure(request, throwable);
                                                        } else {
-                                                               if 
(checkpoint.isDiscarded()) {
-                                                                       
onTriggerFailure(
-                                                                               
checkpoint,
-                                                                               
new CheckpointException(
-                                                                               
        CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE,
-                                                                               
        checkpoint.getFailureCause()));
-                                                               } else {
-                                                                       // no 
exception, no discarding, everything is OK
-                                                                       final 
long checkpointId = checkpoint.getCheckpointId();
-                                                                       
snapshotTaskState(
-                                                                               
timestamp,
-                                                                               
checkpointId,
-                                                                               
checkpoint.getCheckpointStorageLocation(),
-                                                                               
request.props,
-                                                                               
executions,
-                                                                               
request.advanceToEndOfTime);
-
-                                                                       
coordinatorsToCheckpoint.forEach((ctx) -> 
ctx.afterSourceBarrierInjection(checkpointId));
-
-                                                                       
onTriggerSuccess();
-                                                               }
+                                                               
onTriggerFailure(checkpoint, throwable);
                                                        }
+                                               } else {
+                                                       if 
(checkpoint.isDiscarded()) {
+                                                               
onTriggerFailure(
+                                                                       
checkpoint,
+                                                                       new 
CheckpointException(
+                                                                               
CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE,
+                                                                               
checkpoint.getFailureCause()));
+                                                       } else {
+                                                               // no 
exception, no discarding, everything is OK
+                                                               final long 
checkpointId = checkpoint.getCheckpointId();
+                                                               
snapshotTaskState(
+                                                                       
timestamp,
+                                                                       
checkpointId,
+                                                                       
checkpoint.getCheckpointStorageLocation(),
+                                                                       
request.props,
+                                                                       
executions,
+                                                                       
request.advanceToEndOfTime);
+
+                                                               
coordinatorsToCheckpoint.forEach((ctx) -> 
ctx.afterSourceBarrierInjection(checkpointId));
+
+                                                               
onTriggerSuccess();
+                                                       }
+                                               }
 
-                                                       return null;
-                                               },
-                                               timer));
+                                               return null;
+                                       },
+                                       timer)
+                               .whenComplete((unused, error) -> {
+                                       if (error != null) {
+                                               if (!isShutdown()) {
+                                                       
failureManager.handleJobLevelCheckpointException(new 
CheckpointException(EXCEPTION, error), Optional.empty());
+                                               } else if (error instanceof 
RejectedExecutionException) {

Review comment:
       Should we check the `RejectedExecutionException` here? It's kind of case 
by case. Do you think `log.warn` for every exception is better? In the future 
we could shut down timer and `CheckpointCoordinator` in a more elegant way to 
avoid this warning log.




----------------------------------------------------------------
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]


Reply via email to