Hi Jinzhong,
Sorry to answer you just now. We have switched from incremental checkpoint
to non-incremental checkpoint before, I think one of the reasons is the
difficulty to handle properly the clean up of checkpoints on S3. But with
the flink operator's periodic savepoint it may change. I'll re-
Hello,
maybe someone can correct me if I'm wrong, but reading through [1], it
seems to me that manually triggered checkpoints were meant for these
scenarios. If the implementation follows the ticket's description, a
user-triggered checkpoint would "break the chain of incremental
checkpoints", whic
Hi Yang,
I think there is no configuration option available that allow users to
disable checkpoint file cleanup at runtime.
Does your flink application use incremental checkpoint?
1) If yes, i think leveraging S3's lifecycle management to clean checkpoint
files is not safe, because it may acciden
Hi Martijn,
We're currently utilizing flink-s3-fs-presto. After reviewing the
flink-s3-fs-hadoop source code, I believe we would encounter similar issues
with it as well.
When we say, 'The purpose of a checkpoint, in principle, is that Flink
manages its lifecycle,' I think it implies that the au
Hi Junrui,
Currently, we have configured our flink cluster with
execution.checkpointing.externalized-checkpoint-retention:
RETAIN_ON_CANCELLATION and state.checkpoints.num-retained: 10. However,
this setup begins to delete the oldest checkpoint once we exceed 10.
Typically, by the time substantia
Ah, I actually misread checkpoint and savepoints, sorry. The purpose
of a checkpoint in principle is that Flink manages its lifecycle.
Which S3 interface are you using for the checkpoint storage?
On Tue, Nov 7, 2023 at 6:39 PM Martijn Visser wrote:
>
> Hi Yang,
>
> If you use the NO_CLAIM mode, F
Hi Yang,
If you use the NO_CLAIM mode, Flink will not assume ownership of a
snapshot and leave it up to the user to delete them. See the blog [1]
for more details.
Best regards,
Martijn
[1]
https://flink.apache.org/2022/05/06/improvements-to-flink-operations-snapshots-ownership-and-savepoint-f
Hi Yang,
You can try configuring
"execution.checkpointing.externalized-checkpoint-retention:
RETAIN_ON_CANCELLATION"[1] and increasing the value of
"state.checkpoints.num-retained"[2] to retain more checkpoints.
Here are the official documentation links for more details:
[1]
https://nightlies.