cc Bekir Best, Congxian
Congxian Qiu <qcx978132...@gmail.com> 于2019年8月2日周五 下午12:23写道: > Hi Bekir > I’ll first summary the problem here(please correct me if I’m wrong) > 1. The same program runs on 1.6 never encounter such problems > 2. Some checkpoints completed too long (15+ min), but other normal > checkpoints complete less than 1 min > 3. Some bad checkpoint will have a large sync time, async time seems ok > 4. Some bad checkpoint, the e2e duration will much bigger than (sync_time > + async_time) > First, answer the last question, the e2e duration is ack_time - > trigger_time, so it always bigger than (sync_time + async_time), but we > have a big gap here, this may be problematic. > According to all the information, maybe the problem is some task start to > do checkpoint too late and the sync checkpoint part took some time too > long, Could you please share some more information such below: > - A Screenshot of summary for one bad checkpoint(we call it A here) > - The detailed information of checkpoint A(includes all the problematic > subtasks) > - Jobmanager.log and the taskmanager.log for the problematic task and a > health task > - Share the screenshot of subtasks for the problematic task(includes the > `Bytes received`, `Records received`, `Bytes sent`, `Records sent` column), > here wants to compare the problematic parallelism and good parallelism’s > information, please also share the information is there has a data skew > among the parallelisms, > - could you please share some jstacks of the problematic parallelism — > here wants to check whether the task is too busy to handle the barrier. > (flame graph or other things is always welcome here) > > Best, > Congxian > > > Congxian Qiu <qcx978132...@gmail.com> 于2019年8月1日周四 下午8:26写道: > >> Hi Bekir >> >> I'll first comb through all the information here, and try to find out the >> reason with you, maybe need you to share some more information :) >> >> Best, >> Congxian >> >> >> Bekir Oguz <bekir.o...@persgroep.net> 于2019年8月1日周四 下午5:00写道: >> >>> Hi Fabian, >>> Thanks for sharing this with us, but we’re already on version 1.8.1. >>> >>> What I don’t understand is which mechanism in Flink adds 15 minutes to >>> the checkpoint duration occasionally. Can you maybe give us some hints on >>> where to look at? Is there a default timeout of 15 minutes defined >>> somewhere in Flink? I couldn’t find one. >>> >>> In our pipeline, most of the checkpoints complete in less than a minute >>> and some of them completed in 15 minutes+(less than a minute). >>> There’s definitely something which adds 15 minutes. This is happening in >>> one or more subtasks during checkpointing. >>> >>> Please see the screenshot below: >>> >>> Regards, >>> Bekir >>> >>> >>> >>> Op 23 jul. 2019, om 16:37 heeft Fabian Hueske <fhue...@gmail.com> het >>> volgende geschreven: >>> >>> Hi Bekir, >>> >>> Another user reported checkpointing issues with Flink 1.8.0 [1]. >>> These seem to be resolved with Flink 1.8.1. >>> >>> Hope this helps, >>> Fabian >>> >>> [1] >>> >>> https://lists.apache.org/thread.html/991fe3b09fd6a052ff52e5f7d9cdd9418545e68b02e23493097d9bc4@%3Cuser.flink.apache.org%3E >>> >>> Am Mi., 17. Juli 2019 um 09:16 Uhr schrieb Congxian Qiu < >>> qcx978132...@gmail.com>: >>> >>> Hi Bekir >>> >>> First of all, I think there is something wrong. the state size is almost >>> the same, but the duration is different so much. >>> >>> The checkpoint for RocksDBStatebackend is dump sst files, then copy the >>> needed sst files(if you enable incremental checkpoint, the sst files >>> already on remote will not upload), then complete checkpoint. Can you >>> check >>> the network bandwidth usage during checkpoint? >>> >>> Best, >>> Congxian >>> >>> >>> Bekir Oguz <bekir.o...@persgroep.net> 于2019年7月16日周二 下午10:45写道: >>> >>> Hi all, >>> We have a flink job with user state, checkpointing to RocksDBBackend >>> which is externally stored in AWS S3. >>> After we have migrated our cluster from 1.6 to 1.8, we see occasionally >>> that some slots do to acknowledge the checkpoints quick enough. As an >>> example: All slots acknowledge between 30-50 seconds except only one slot >>> acknowledges in 15 mins. Checkpoint sizes are similar to each other, like >>> 200-400 MB. >>> >>> We did not experience this weird behaviour in Flink 1.6. We have 5 min >>> checkpoint interval and this happens sometimes once in an hour sometimes >>> more but not in all the checkpoint requests. Please see the screenshot >>> below. >>> >>> Also another point: For the faulty slots, the duration is consistently 15 >>> mins and some seconds, we couldn’t find out where this 15 mins response >>> time comes from. And each time it is a different task manager, not always >>> the same one. >>> >>> Do you guys aware of any other users having similar issues with the new >>> version and also a suggested bug fix or solution? >>> >>> >>> >>> >>> Thanks in advance, >>> Bekir Oguz >>> >>> >>> >>>