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

fanrui edited comment on FLINK-26049 at 2/18/22, 11:32 AM:
-----------------------------------------------------------

Hi [~akalashnikov] , could we increase numberOfFailedCheckpoints 
[here|https://github.com/apache/flink/blob/ac3ad139fbad02b2de241d5eef7b1e3ce6007b82/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L938]?
 It should be same bug with FLINK-24344.

 

User usually use the metric to monitor checkpoint status.

 


was (Author: fanrui):
Hi [~akalashnikov] , could we increase numberOfFailedCheckpoints 
[here|https://github.com/apache/flink/blob/ac3ad139fbad02b2de241d5eef7b1e3ce6007b82/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L938]?
 It should be same bug with FLINK-24344.

 

 

> The tolerable-failed-checkpoints logic is invalid when checkpoint trigger 
> failed
> --------------------------------------------------------------------------------
>
>                 Key: FLINK-26049
>                 URL: https://issues.apache.org/jira/browse/FLINK-26049
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.13.5, 1.14.3
>            Reporter: fanrui
>            Priority: Major
>             Fix For: 1.15.0
>
>         Attachments: image-2022-02-09-18-08-17-868.png, 
> image-2022-02-09-18-08-34-992.png, image-2022-02-09-18-08-42-920.png, 
> image-2022-02-18-11-28-53-337.png, image-2022-02-18-11-33-28-232.png, 
> image-2022-02-18-11-44-52-745.png
>
>
> After triggerCheckpoint, if checkpoint failed, flink will execute the 
> tolerable-failed-checkpoints logic. But if triggerCheckpoint failed, flink 
> won't execute the tolerable-failed-checkpoints logic.
> h1. How to reproduce this issue?
> In our online env, hdfs sre deletes the flink base dir by mistake, and flink 
> job don't have permission to create checkpoint dir. So cause flink trigger 
> checkpoint failed.
> There are some didn't meet expectations:
>  * JM just log _"Failed to trigger checkpoint for job 
> 6f09d4a15dad42b24d52c987f5471f18 since Trigger checkpoint failure" ._ Don't 
> show the root cause or exception.
>  * user set tolerable-failed-checkpoints=0, but if triggerCheckpoint failed, 
> flink won't execute the tolerable-failed-checkpoints logic. 
>  * When triggerCheckpoint failed, numberOfFailedCheckpoints is always 0
>  * When triggerCheckpoint failed, we can't find checkpoint info in checkpoint 
> history page.
>  
> !image-2022-02-09-18-08-17-868.png!
>  
> !image-2022-02-09-18-08-34-992.png!
> !image-2022-02-09-18-08-42-920.png!
>  
> h3. *All metrics are normal, so the next day we found out that the checkpoint 
> failed, and the checkpoint has been failing for a day. it's not acceptable to 
> the flink user.*
> I have some ideas:
>  # Should tolerable-failed-checkpoints logic be executed when 
> triggerCheckpoint fails?
>  # When triggerCheckpoint failed, should increase numberOfFailedCheckpoints?
>  # When triggerCheckpoint failed, should show checkpoint info in checkpoint 
> history page?
>  # JM just show "Failed to trigger checkpoint", should we show detailed 
> exception to easy find the root cause?
>  
> Masters, could we do these changes? Please correct me if I'm wrong.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to