Hi Mátyás,

Checkpoint are meant to be atomic in nature, i.e. everything is checkpointed at 
the more or less same time.
What you can do in newer Flink versions is to enable the Change Log Feature 
(see [1]) which spreads the actual I/O for writing checkpoint files to a longer 
period and to keep an additional change log file with the running updates.
What you get is a little more overall I/O but a quite flat I/O rate.


Hope this helps

Thias


[1] 
https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/ops/state/state_backends/#enabling-changelog


From: Őrhidi Mátyás <matyas.orh...@gmail.com>
Sent: Wednesday, September 13, 2023 2:47 PM
To: Gyula Fóra <gyula.f...@gmail.com>
Cc: Hangxiang Yu <master...@gmail.com>; user@flink.apache.org
Subject: Re: Checkpoint jitter?

Correct, thanks for the clarification Gyula!

On Wed, Sep 13, 2023 at 1:39 AM Gyula Fóra 
<gyula.f...@gmail.com<mailto:gyula.f...@gmail.com>> wrote:
No, I think what he means is to trigger the checkpoint at slightly different 
times at the different sources so the different parts of the pipeline would not 
checkpoint at the same time.

Gyula

On Wed, Sep 13, 2023 at 10:32 AM Hangxiang Yu 
<master...@gmail.com<mailto:master...@gmail.com>> wrote:
Hi, Matyas.
Do you mean something like adjusting checkpoint intervals dynamically or 
frequency of uploading files according to the pressure of the durable storage ?

On Wed, Sep 13, 2023 at 9:12 AM Őrhidi Mátyás 
<matyas.orh...@gmail.com<mailto:matyas.orh...@gmail.com>> wrote:
Hey folks,

Is it possible to add some sort of jitter to the checkpointing logic for 
massively parallel jobs to mitigate the burst impact on the durable storage 
when a checkpoint is triggered?

Thanks,
Matyas


--
Best,
Hangxiang.
Diese Nachricht ist ausschliesslich für den Adressaten bestimmt und beinhaltet 
unter Umständen vertrauliche Mitteilungen. Da die Vertraulichkeit von 
e-Mail-Nachrichten nicht gewährleistet werden kann, übernehmen wir keine 
Haftung für die Gewährung der Vertraulichkeit und Unversehrtheit dieser 
Mitteilung. Bei irrtümlicher Zustellung bitten wir Sie um Benachrichtigung per 
e-Mail und um Löschung dieser Nachricht sowie eventueller Anhänge. Jegliche 
unberechtigte Verwendung oder Verbreitung dieser Informationen ist streng 
verboten.

This message is intended only for the named recipient and may contain 
confidential or privileged information. As the confidentiality of email 
communication cannot be guaranteed, we do not accept any responsibility for the 
confidentiality and the intactness of this message. If you have received it in 
error, please advise the sender by return e-mail and delete this message and 
any attachments. Any unauthorised use or dissemination of this information is 
strictly prohibited.

Reply via email to