[ https://issues.apache.org/jira/browse/FLINK-10074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16643078#comment-16643078 ]
ASF GitHub Bot commented on FLINK-10074: ---------------------------------------- azagrebin commented on issue #6567: [FLINK-10074] Allowable number of checkpoint failures URL: https://github.com/apache/flink/pull/6567#issuecomment-428137039 Hi @yanghua, In general, `executionGraph.failGlobal` looks good to me to fail, but I think the `CheckpointFailureManager` should be constructed with a callback which will fail job. It means that `JobMaster` or `ExecutionGraphBuilder` should decide how to fail. We also have to make that checkpointing in Task Executor does not fail the job anymore, only the `CheckpointFailureManager` as a central point of failure. Another point is that inserting failure callback into different places in `CheckpointCoordinator` close to cause looks rather invasive. Ideally, there should be 2 places where `CheckpointFailureManager` should get callback: - in `triggerCheckpoint(long timestamp, boolean isPeriodic)` checking `CheckpointTriggerResult` (sync part) - similar to `PendingCheckpointStats` getting callback in `PendingCheckpoint.reportFailedCheckpoint` (async part) but with clear defined cause like `CheckpointDeclineReason` for sync part. `PendingCheckpoint` does not have such clearly defined failure handling as `triggerCheckpoint` and might need some refactoring to distinguish failure cases. I suggest we firstly describe the full approach in Jira issue where we also include a list of all possible failures which `CheckpointFailureManager` needs to react upon and how we define them in code. It can be also a link to some design doc. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Allowable number of checkpoint failures > ---------------------------------------- > > Key: FLINK-10074 > URL: https://issues.apache.org/jira/browse/FLINK-10074 > Project: Flink > Issue Type: Improvement > Components: State Backends, Checkpointing > Reporter: Thomas Weise > Assignee: vinoyang > Priority: Major > Labels: pull-request-available > > For intermittent checkpoint failures it is desirable to have a mechanism to > avoid restarts. If, for example, a transient S3 error prevents checkpoint > completion, the next checkpoint may very well succeed. The user may wish to > not incur the expense of restart under such scenario and this could be > expressed with a failure threshold (number of subsequent checkpoint > failures), possibly combined with a list of exceptions to tolerate. > -- This message was sent by Atlassian JIRA (v7.6.3#76005)