flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Congxian Qiu <qcx978132...@gmail.com>
Subject Re: S3 Checkpointing taking long time with stateful operations
Date Sun, 21 Jun 2020 02:19:25 GMT
Hi

>From the description and the given figure. the e2e time for one task is
longer than $time{sync-snapshot} + $time{async-snapshot}.
For at least once mode, could you please try to enable the debug log to
track the barrier align process?
you can find the debug log such as
"Received barrier for checkpoint {} from channel {}"
"Received cancellation barrier for checkpoint {} "
"Received all barriers for checkpoint {}"

Best,
Congxian


Yun Tang <myasuka@live.com> 于2020年6月19日周五 上午11:48写道:

> Hi Sandeep
>
> At-least-once checkpoint mode would not need to align barrier and the
> longer end-to-end duration is mainly due to barrier cannot be processed by
> operator as soon as possible.
> Operator will only start checkpoint after processed checkpoint barrier, I
> think you might need to check the back-pressure status of your job[1].
> Back-pressure would make the checkpoint barrier move to downstream more
> slowly in the network channels.
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/back_pressure.html#back-pressure
>
> Best
> Yun Tang
> ------------------------------
> *From:* Kathula, Sandeep <Sandeep_Kathula@intuit.com>
> *Sent:* Friday, June 19, 2020 9:19
> *To:* user@flink.apache.org <user@flink.apache.org>
> *Cc:* Vora, Jainik <Jainik_Vora@intuit.com>; Rosensweig, JD <
> JD_Rosensweig@intuit.com>
> *Subject:* S3 Checkpointing taking long time with stateful operations
>
>
> Hi,
>
> We are running a stateful application in Flink with *RocksDB as backend* and
> set *incremental state to true *with checkpoints written to* S3*.
>
>    - 10 task managers each with 2 task slots
>    - Checkpoint interval 3 minutes
>    - Checkpointing mode – At-least once processing
>
>
>
> After running app for 2-3 days, we are seeing end to end checkpoint takes
> almost *2 minutes* with Sync time 2 sec and async time 15 sec max. But
> initially when state is less, it takes 10-15 sec for checkpointing. As
> checkpointing mode is at least once, align duration is 0. We are seeing a
> dip in processing during this time. Couldn’t find out what the actual issue
> is.
>
>
>
> We also tried with remote HDFS for checkpointing but observed similar
> behavior.
>
>
>
> We have couple of questions:
>
>    - *When sync time is max 2 sec and async time is 15 sec why is end to
>    end checkpointing taking almost 2 minutes?*
>    - *How can we reduce the checkpoint time?*
>
> [image: A screenshot of a cell phone Description automatically generated]
>
>
>
> Any help would be appreciated.
>
>
>
>
>
> Thank you
>
> Sandeep Kathula
>
>
>
>
>

Mime
View raw message