Myasuka commented on a change in pull request #14662:
URL: https://github.com/apache/flink/pull/14662#discussion_r561753898
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java
##########
@@ -237,19 +241,29 @@ private void handleExecutionException(Exception e) {
+ '.',
e);
- // We only report the exception for the original cause of fail
and cleanup.
- // Otherwise this followup exception could race the original
exception in failing
- // the task.
- try {
- taskEnvironment.declineCheckpoint(
+ if (isTaskRunning.get()) {
+ // We only report the exception for the original cause of
fail and cleanup.
+ // Otherwise this followup exception could race the
original exception in
+ // failing the task.
+ try {
+ taskEnvironment.declineCheckpoint(
+ checkpointMetaData.getCheckpointId(),
+ new CheckpointException(
+
CheckpointFailureReason.CHECKPOINT_ASYNC_EXCEPTION,
+ checkpointException));
+ } catch (Exception unhandled) {
+ AsynchronousException asyncException = new
AsynchronousException(unhandled);
+ asyncExceptionHandler.handleAsyncException(
+ "Failure in asynchronous checkpoint
materialization",
+ asyncException);
+ }
+ } else {
+ // We never decline checkpoint after task is not running
to avoid unexpected job
+ // failover, which caused by exceeding checkpoint
tolerable failure threshold.
+ LOG.warn(
Review comment:
We changed the logic to fail job from task side to JM side with
responsibility of `CheckpointFailureManager` at Flink-1.9.
Previously (take Flink-1.8 for example), if the asynchronous part of
checkpoint failed on task side, it would call
`FailingCheckpointExceptionHandler` to `tryHandleCheckpointException` which
just [rethrow the
exception](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java#L56),
and then `AsyncCheckpointExceptionHandler` [handle the
exception](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L1156)
with [code
below](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L791):
```java
public void handleAsyncException(String message, Throwable exception) {
if (isRunning) {
// only fail if the task is still running
getEnvironment().failExternally(exception);
}
}
```
As you can see, if task is not running, it would not fail externally again.
Thus, after JM decided whether to fail the job, task should not send decline
message to JM if it is not running to avoid unexpected failover from
`CheckpointFailureManager`.
If such task is not running, there should existed two cases:
1. Task is finished. This cannot be stopping with savepoint as we would stop
checkpoint scheduler first to avoid newer checkpoint after final savepoint. And
this could only be the case that task reach to finished stage while async phase
of period checkpoint is still running. In this case, the job will come to
finished status thus we should not failover the job anymore by decline. Or some
tasks are not running, that checkpoint would finally timeout just like previous
behavior. Since FLIP-147 is accepted, I think we could think about what would
happen in this case after FLIP-147. cc @gaoyunhaii
2. Task is canceled. This should be triggered by existing failover, and we
should not decline one more time.
----------------------------------------------------------------
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]