And now it's happening again -----Original Message----- From: Shai Kaplan [mailto:shai.kap...@microsoft.com] Sent: Wednesday, February 22, 2017 12:02 PM To: user@flink.apache.org Subject: RE: Flink checkpointing gets stuck
I changed the checkpoint interval to 30 minutes, and also switched RocksDB predefined options to FLASH_SSD_OPTIMIZED, as suggested by Vinay. The problem hasn't exactly occurred since yesterday, but perhaps it just takes it longer to happen again because the checkpoints are much less frequent now. I'm now pretty sure that the " Checkpoint Coordinator is suspending" is not related to the other issue, because now I had a checkpoint that failed because of that but then the next ones succeeded. I apologize for diverging from the topic, but I think it's worth mentioning too, the exception was "Could not materialize checkpoint" caused by "Could not flush and close the file system output stream" caused by "com.microsoft.azure.storage.StorageException: The specified blob does not exist". What could cause that? A temporary failure in I/O with Azure blobs? Anyway, back to the topic, I didn't have checkpoints timing out, but I did have one checkpoint that took significantly longer. This is what the checkpoints history look like right now: ID Status Acknowledged Trigger Time Latest Acknowledgement End to End Duration State Size Buffered During Alignment 764 241/241 11:42:50 11:43:56 1m 6s 3.15 GB 30.4 MB 763 241/241 11:12:50 11:28:38 15m 48s 3.11 GB 13.4 MB 762 241/241 10:42:50 10:43:57 1m 7s 3.08 GB 12.8 MB 761 241/241 10:12:50 10:13:58 1m 8s 3.03 GB 5.52 MB 760 241/241 9:42:50 9:43:55 1m 5s 2.99 GB 607 KB 759 241/241 9:12:50 9:13:52 1m 1s 2.94 GB 0 B 758 241/241 8:42:50 8:43:46 56s 2.90 GB 5.55 MB 757 241/241 8:12:50 8:14:20 1m 30s 2.85 GB 0 B 756 121/241 7:41:49 7:42:09 22s 280 MB 0 B 755 241/241 7:11:49 7:12:44 55s 2.81 GB 30 B The status wasn't copied well, but they all succeeded except for #756, which failed for the reason I mentioned above. As you can see, checkpoint #763 took a lot longer for no apparent reason, so I'm guessing it's the same thing that caused the checkpoints to time out at 30 minutes when they were saved every 10 seconds, only now it's less severe because the load is much lower. Any thoughts on what could cause that? -----Original Message----- From: Ufuk Celebi [mailto:u...@apache.org] Sent: Tuesday, February 21, 2017 4:54 PM To: user@flink.apache.org Subject: Re: Flink checkpointing gets stuck Hey Shai! Thanks for reporting this. It's hard to tell what causes this from your email, but could you check the checkpoint interface (https://na01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fci.apache.org%2Fprojects%2Fflink%2Fflink-docs-release-1.3%2Fmonitoring%2Fcheckpoint_monitoring.html&data=02%7C01%7CShai.Kaplan%40microsoft.com%7C1cdb8bde8ee843676a0008d45a6984b7%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C636232856631405599&sdata=Ukf1%2Bahi00NVb3QVrTqUggYOSldheqvgMkJLsCzUWVc%3D&reserved=0) and report how much progress the checkpoints make before timing out? The "Checkpoint Coordinator is suspending" message indicates that the job failed and the checkpoint coordinator is shut down because of that. Can you check the TaskManager and JobManager logs if other errors are reported? Feel free to share them. Then I could help with going over them. – Ufuk On Tue, Feb 21, 2017 at 2:47 PM, Shai Kaplan <shai.kap...@microsoft.com> wrote: > Hi. > > I'm running a Flink 1.2 job with a 10 seconds checkpoint interval. > After some running time (minutes-hours) Flink fails to save > checkpoints, and stops processing records (I'm not sure if the > checkpointing failure is the cause of the problem or just a symptom). > > After several checkpoints that take some seconds each, they start > failing due to 30 minutes timeout. > > When I restart one of the Task Manager services (just to get the job > restarted), the job is recovered from the last successful checkpoint > (the state size continues to grow, so it's probably not the reason for > the failure), advances somewhat, saves some more checkpoints, and then > enters the failing state again. > > One of the times it happened, the first failed checkpoint failed due > to "Checkpoint Coordinator is suspending.", so it might be an > indicator for the cause of the problem, but looking into Flink's code > I can't see how a running job could get to this state. > > I am using RocksDB for state, and the state is saved to Azure Blob > Store, using the NativeAzureFileSystem HDFS connector over the wasbs protocol. > > Any ideas? Possibly a bug in Flink or RocksDB?