[ 
https://issues.apache.org/jira/browse/FLINK-10074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16623777#comment-16623777
 ] 

ASF GitHub Bot commented on FLINK-10074:
----------------------------------------

azagrebin edited a comment on issue #6567: [FLINK-10074] Allowable number of 
checkpoint failures
URL: https://github.com/apache/flink/pull/6567#issuecomment-423573937
 
 
   Thanks for the update, @yanghua. Looking at the checkpoint coordinator more 
deeply, I think we firstly have to work a bit more on design for this kind of 
change. We have to take into account at least the following points (roughly):
   - Introduce the separate component/class responsible for failure management, 
counting e.g. `CheckpointFailureManager` or something
   - Job manager should 
       - construct `CheckpointFailureManager`
       - configure with the max failure count and a proper action how to fail.
       - pass it to `CheckpointCoordinator`
   - `CheckpointCoordinator`
       - should give callbacks to `CheckpointFailureManager` about failures and 
successes of checkpoints
       - needs some refactoring to distinguish better failures of 
`PendingCheckpoint`. Now it is `abortXXX()` methods which do not provide enough 
information to count it as a failure for `CheckpointFailureManager` or not 
(like we have for `triggerSavepoint` in `CheckpointDeclineReason`). At the end 
there should be clear place in the `CheckpointCoordinator` where to give 
callbacks to `CheckpointFailureManager`, e.g.:
           - `CheckpointDeclineReason.EXCEPTION` result of `triggerSavepoint`
           - some cases of `PendingCheckpoint.abortDeclined()`, `abortError()`, 
maybe `abortExpired()`
   - Consider having only `DecliningCheckpointExceptionHandler` on 
`TaskExecutor` side and letting now to handle all failure cases only in 
`CheckpointCoordinator`
   
   There might be more points. I suggest we step back and continue discussion 
in the jira issue. Once we have clear design, a PR can be opened again.
   

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

Reply via email to