This thread is also useful in this context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/difference-between-checkpoints-amp-savepoints-td14787.html <http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/difference-between-checkpoints-amp-savepoints-td14787.html>
> On 24 Aug 2018, at 14:49, Andrey Zagrebin <and...@data-artisans.com> wrote: > > Hi Henry, > > In addition to Vino’s answer, there are several things to keep in mind about > “checkpoints vs savepoints". > > Checkpoints are designed mostly for fault tolerance of running Flink job and > automatic recovery > that is why by default Flink manages their storage itself. Though it is > correct that you can configure the checkpoints to be retained (externalised), > have control over their storage and resume a failed/canceled job from them. > > But their format might be optimised for any of new Flink versions and change > between them. > It means that in general you might not be able to upgrade Flink version or > the running job structure using only checkpoints. > > Moreover, currently, it is not guaranteed that you will be always able to > rescale your job from the checkpoint (change parallelism). Although, it is > technically possible for Flink 1.6.0 at the moment, even for incremental > checkpoints. > > Savepoints are designed for manual intervention of the user for maintenance > operations > that is why their storage is under control of the user in the first place. > They have more stable internal format which allows manual migration between > Flink or job versions and rescaling. > > Cheers, > Andrey > >> On 24 Aug 2018, at 12:55, vino yang <yanghua1...@gmail.com >> <mailto:yanghua1...@gmail.com>> wrote: >> >> Hi Henry, >> >> A good answer from stackoverflow: >> >> Apache Flink's Checkpoints and Savepoints are similar in that way they both >> are mechanisms for preserving internal state of Flink's applications. >> >> Checkpoints are taken automatically and are used for automatic restarting >> job in case of a failure. >> >> Savepoints on the other hand are taken manually, are always stored >> externally and are used for starting a "new" job with previous internal >> state in case of e.g. >> >> Bug fixing >> Flink version upgrade >> A/B testing, etc. >> Underneath they are in fact the same mechanism/code path with some subtle >> nuances. >> >> About your question: >> >> 1) No problem, The main purpose of checkpoint itself is to automatically >> restart the recovery when the job fails. >> 2) You can also use REST client to trigger savepoint. >> 3) I don't know, But it seems that their usage scenarios and purposes are >> still different. May Till and Chesnay can answer this question. >> >> Thanks, vino. >> >> 徐涛 <happydexu...@gmail.com <mailto:happydexu...@gmail.com>> 于2018年8月24日周五 >> 下午3:19写道: >> Hi All, >> I check the documentation of Flink release 1.6, find that I can use >> checkpoints to resume the program either. As I encountered some problems >> when using savepoints, I have the following questions: >> 1. Can I use checkpoints only, but not use savepoints, because it can >> also use to resume programs. If I do so, is there any problem? >> 2. Checkpoint can be generated automatically, but savepoints seems can >> only be generated manually. I have to write a crontab to generate the >> savepoint, more than this, my Flink program is run on Yarn, and on the >> machines, only the Hadoop and Yarn are installed, so I can not use flink >> savepoint command to generate savepoint, and I have no authorization to >> install Flink on the machines. >> 3. Will checkpoint and savepoint merged in later releases? >> Thanks very much. >> >> Best, >> Henry >