Hi, That’s a good observation! And it is indeed the expected behaviour. There are two parts to understanding this: * "retain checkpoints” tells Flink to retain any checkpoints that it stores when a job is shut down * for recovery purposes (all checkpointing purposes, really) a savepoint counts as a checkpoint. Otherwise, you can have strange behaviour when you do a successful savepoint, then a failure occurs, and you would then restore from the checkpoint before that. See especially [1] about this.
So when you configure Flink to retain only one checkpoint this means that when you do a savepoint it will count as that one checkpoint, i.e. the previous checkpoint is discarded. Thus, you only have that savepoint after the job is canceled. I hope this makes sense. Let me know if you have any questions. Best, Aljoscha [1] https://issues.apache.org/jira/browse/FLINK-10354 > On 21. Mar 2019, at 08:58, Parth Sarathy <parth.sara...@microfocus.com> wrote: > > Hi All, > We are using flink 1.7.2 and have enabled checkpoint with > RocksDB configured as state backend with retain checkpoints on job cancel. > In our scenario we are cancelling the job and while resubmitting the job, we > try to restore the job with latest checkpoint / savepoint available. We are > observing ambiguous behavior based on the way job is being cancelled, below > are the captured observations: > > Observations : > 1. When we cancel the job with a savepoint option, a savepoint is created as > expected but flink is deleting the latest checkpoint directory available for > the running job. Is this an expected behavior even when the configuration > asks to retain checkpoints on job cancellation? > 2. When we cancel the job without the savepoint option, the same latest > checkpoint was retained by flink as opposed to before where it was deleted > as job was cancelled with the savepoint option. > > As we have configured flink to retain only a single > checkpoint at any point of time, could there be any issue wherein when we > cancel the job with a savepoint, the savepoint gets triggered but fails > midway. So now we would end up with an incomplete savepoint and no trace of > checkpoint for the job as it would have been erased. > > Thanks > Parth Sarathy > > > > -- > Sent from: > http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/