Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-20675

Asynchronous checkpoint failure would not fail the job anymore

    XMLWordPrintableJSON

Details

    Description

      After FLINK-12364, no mater how many times of asynchronous part of checkpoint on task failed, the job itself would not fail by default:

       Default behavior Flink-1.5 —> Flink-1.8 Flink-1.9 -> Flink-1.12
      Synchronous part of checkpoint at task failed Job failed Job failed
      Asynchronous part of checkpoint at task failed  Job failed  Job would not fail

       This error was because StreamTask use Exception instead of CheckpointException when async part failed as decline message. Thus checkpoint coordinator would call failPendingCheckpointDueToTaskFailure(pendingCheckpoint, CheckpointFailureReason.JOB_FAILURE, cause, executionAttemptID) to process the declined checkpoint:

      if (cause == null) {
      	failPendingCheckpointDueToTaskFailure(pendingCheckpoint, CheckpointFailureReason.CHECKPOINT_DECLINED, executionAttemptID);
      } else if (cause instanceof CheckpointException) {
      	CheckpointException exception = (CheckpointException) cause;
      	failPendingCheckpointDueToTaskFailure(pendingCheckpoint, exception.getCheckpointFailureReason(), cause, executionAttemptID);
      } else {
      	failPendingCheckpointDueToTaskFailure(pendingCheckpoint, CheckpointFailureReason.JOB_FAILURE, cause, executionAttemptID);
      }
      

      However, CheckpointFailureManager would ignore the JOB_FAILURE reason and not count this failed checkpoint, which causes asynchronous checkpoint failure would not fail the job anymore.

       

      FLINK-16753 corrects the misleading message of JOB_FAILURE but the asynchronous checkpoint failure still cannot fail the job.

       

      As this bug exists too long, I decide to set it as critical instead of blocker level. 

       

      Attachments

        Issue Links

          Activity

            People

              yunta Yun Tang
              yunta Yun Tang
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: