[ https://issues.apache.org/jira/browse/FLINK-23874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17411591#comment-17411591 ]
Youjun Yuan commented on FLINK-23874: ------------------------------------- The first attempt was started by resuming from savepoint 258. But the second attempt was NOT started by me, actually I was canceling the job, but due to a bug of Flink, at the end of shutting down the job, it hit a following error, so YARN restarted it: {code:java} RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@3460b405 rejected from {code} this log can be found at the end of JM log of attempt 1. If you pay attention to the log timestamp, you can find that attempt 2 start immediately after attempt 1, just 2 seconds later. > JM did not store latest checkpiont id into Zookeeper, silently > -------------------------------------------------------------- > > Key: FLINK-23874 > URL: https://issues.apache.org/jira/browse/FLINK-23874 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing > Affects Versions: 1.12.1 > Reporter: Youjun Yuan > Priority: Major > Attachments: container_e04_1628083845581_0254_01_000001_jm.log, > container_e04_1628083845581_0254_01_000050_tm.log, > container_e04_1628083845581_0254_02_000001_jm.log > > > Job manager did not update the latest successful checkpoint id into zookeeper > (with ZK HA setup), at path /flink/\{app_id}/checkpoints/, when JM restart, > the job resumed from a very old position. > > We had a job which was resumed from save point 258, after running for a few > days, the latest successful checkpoint was about chk 686. When something > trigged the JM to restart, it restored state to save point 258, instead of > chk 686. > We checked zookeeper, indeed the stored checkpoint was still 258, which means > JM hasn't stored checkpoint id into zookeeper for few days, and without any > error message. > > below are the relevant logs around the restart: > {quote} > {{2021-08-18 11:09:16,505 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Completed > checkpoint 686 for job 00000000000000000000000000000000 (228296 bytes in 827 > ms).}} > {quote} > > {quote}2021-08-18 11:10:13,066 INFO > org.apache.flink.contrib.streaming.state.restore.RocksDBIncrementalRestoreOperation > [] - Finished restoring from state handle: > IncrementalRemoteKeyedStateHandle\{backendIdentifier=c11d290c-617b-4ea5-b7ed-4853272f32a3, > keyGroupRange=KeyGroupRange{startKeyGroup=47, endKeyGroup=48}, > checkpointId=258, sharedState={}, > privateState=\{OPTIONS-000016=ByteStreamStateHandle{handleName='s3://dp-flink/prd/checkpoints/3beb4dd5-9008-4fd0-9910-f564759b466a/1628912020683/00000000000000000000000000000000/shared/20f35556-5c60-4fca-908c-d05d641c2614', > dataBytes=15818}, > MANIFEST-000006=ByteStreamStateHandle\{handleName='s3://dp-flink/prd/checkpoints/3beb4dd5-9008-4fd0-9910-f564759b466a/1628912020683/00000000000000000000000000000000/shared/3c8e1c2f-616d-4f18-8b07-4a818e3ca110', > dataBytes=336}, > CURRENT=ByteStreamStateHandle\{handleName='s3://dp-flink/prd/checkpoints/3beb4dd5-9008-4fd0-9910-f564759b466a/1628912020683/00000000000000000000000000000000/shared/1dc5f341-8a73-4e69-96fb-4b026653da6d', > dataBytes=16}}, > metaStateHandle=ByteStreamStateHandle\{handleName='s3://dp-flink/prd/checkpoints/3beb4dd5-9008-4fd0-9910-f564759b466a/1628912020683/00000000000000000000000000000000/chk-258/0ef57eb3-0f38-45f5-8f3d-3e7b87f5fd15', > dataBytes=1704}, registered=false} without rescaling. > {quote} > -- This message was sent by Atlassian Jira (v8.3.4#803005)