Hi Paul, I think Robert's idea might be right.
>From the log you pasted, the checkpoint interval is 2m30s. Chk-5 triggered at >16:42:23 and completed at 16:42:42. In the normal case, chk-6 would be triggered near 16:44:53. However, the actual chk-6 triggered at 16:46:02, which is obviously not normal case. I think your analysis is not correct due to the log below: 2022-01-27 16:46:02,693 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 6 (type=CHECKPOINT) @ 1643273162422 for job 3a57fdaa16502c411a46471bba595d7c. If you translate the unix time 1643273162422 to Beijing time, you can see this is actually 2022-01-27 16:46:02. If the Zookeeper is really slow to respond, the unix time should be much earlier than the logged time [1]. Flink has been improved that checkpointing would be backpressured on slow cleanup [2]. [1] https://github.com/apache/flink/blob/90e850301e672fc0da293abc55eb446f7ec68ffa/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L540 [2] https://issues.apache.org/jira/browse/FLINK-17073 Best Yun Tang ________________________________ From: Robert Metzger <metrob...@gmail.com> Sent: Friday, January 28, 2022 21:53 To: Paul Lam <paullin3...@gmail.com> Cc: Yun Tang <myas...@live.com>; user <user@flink.apache.org> Subject: Re: Inaccurate checkpoint trigger time Hi Paul, where are you storing your checkpoints, and what's their size? IIRC, Flink won't trigger a new checkpoint before the old ones haven't been cleaned up, and if your checkpoints are large and stored on S3, it can take a while to clean them up (especially with the Hadoop S3 plugin, using presto s3 is faster). On Thu, Jan 27, 2022 at 10:56 AM Paul Lam <paullin3...@gmail.com<mailto:paullin3...@gmail.com>> wrote: Hi Yun, Sorry for the late reply. I finally found some time to investigate this problem further. I upgraded the job to 1.14.0, but it’s still the same. I’ve checked the debug logs, and I found that Zookeeper notifies watched event of checkpoint id changes very late [1]. Each time a checkpoint finished, it would take minutes before the Zookeeper client notices the checkpoint ID is changed. I suspect the checkpoint coordinator is blocking on incrementing checkpoint ID on Zookeeper [2]. But with no luck, there’s no many relevant logs can help me prove that. What do you think of this? Thanks a lot! [1] https://gist.github.com/link3280/5072a054a43b40ba28891837a8fdf995 [2] https://github.com/apache/flink/blob/90e850301e672fc0da293abc55eb446f7ec68ffa/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java#L743 Best, Paul Lam 2021年11月23日 16:49,Paul Lam <paullin3...@gmail.com<mailto:paullin3...@gmail.com>> 写道: Hi Yun, Thanks a lot for your pointers! I’ll try it out as you suggested and then get back to you. Best, Paul Lam 2021年11月23日 16:32,Yun Tang <myas...@live.com<mailto:myas...@live.com>> 写道: Hi Paul, This is really weird, from what I know, flink-1.11.0 has a problem of handling min-pause time [1] and this should be resolved in flink-1.12.1. Could you open the debug log level for org.apache.flink.runtime.checkpoint and use jmap or byteman to get the field value of CheckpointCoordinator#lastCheckpointCompletionRelativeTime, CheckpointRequestDecider#minPauseBetweenCheckpoints and SystemClock#relativeTimeMillis in method CheckpointRequestDecider#nextTriggerDelayMillis [2] to see any unexpected behavior. [1] https://issues.apache.org/jira/browse/FLINK-18856 [2] https://github.com/apache/flink/blob/90e850301e672fc0da293abc55eb446f7ec68ffa/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRequestDecider.java#L182 Best Yun Tang ________________________________ From: Paul Lam <paullin3...@gmail.com<mailto:paullin3...@gmail.com>> Sent: Tuesday, November 23, 2021 14:35 To: user <user@flink.apache.org<mailto:user@flink.apache.org>> Subject: Inaccurate checkpoint trigger time Hi, Recently I’ve noticed a job has nondeterministic checkpoint trigger time. The jobs is using Flink 1.12.1 with FsStateBackend and is of 650 parallelism. It was configured to trigger checkpoint every 150 seconds with 0 pause time and no concurrent checkpoints. However there’re obvious errors in the checkpoint trigger times, as the actual interval may vary from 30 seconds to 6 minutes. The jobmanager logs are good, and no error logs is found. Some of the output are as follow: 2021-11-23 13:51:46,438 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1446 for job f432b8d90859db54f7a79ff29a563ee4 (47142264825 bytes in 22166 ms). 2021-11-23 13:57:21,021 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1447 (type=CHECKPOINT) @ 1637647040653 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 13:57:43,761 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1447 for job f432b8d90859db54f7a79ff29a563ee4 (46563195101 bytes in 21813 ms). 2021-11-23 13:59:09,387 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1448 (type=CHECKPOINT) @ 1637647149157 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 13:59:31,370 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1448 for job f432b8d90859db54f7a79ff29a563ee4 (45543757702 bytes in 20354 ms). 2021-11-23 14:06:37,916 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1449 (type=CHECKPOINT) @ 1637647597704 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:07:03,157 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1449 for job f432b8d90859db54f7a79ff29a563ee4 (45662471025 bytes in 23779 ms). 2021-11-23 14:07:05,838 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1450 (type=CHECKPOINT) @ 1637647625640 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:07:30,748 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1450 for job f432b8d90859db54f7a79ff29a563ee4 (46916136024 bytes in 22998 ms). 2021-11-23 14:13:09,089 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1451 (type=CHECKPOINT) @ 1637647988831 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:13:38,411 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1451 for job f432b8d90859db54f7a79ff29a563ee4 (47439074367 bytes in 27616 ms). 2021-11-23 14:13:38,676 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1452 (type=CHECKPOINT) @ 1637648018481 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:14:01,937 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1452 for job f432b8d90859db54f7a79ff29a563ee4 (47046200711 bytes in 21869 ms). 2021-11-23 14:20:04,923 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1453 (type=CHECKPOINT) @ 1637648404722 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:20:26,592 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1453 for job f432b8d90859db54f7a79ff29a563ee4 (47481503566 bytes in 20172 ms). 2021-11-23 14:21:54,879 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1454 (type=CHECKPOINT) @ 1637648514668 for job f432b8d90859db54f7a79ff29a563ee4. 2021-11-23 14:22:19,392 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1454 for job f432b8d90859db54f7a79ff29a563ee4 (47106414948 bytes in 22930 ms). It looks pretty weird to me. Please help me narrow down the problem if you have any idea. Best, Paul Lam