Savepoints are externally stored self-contained checkpoints that you can use to stop-and-resume or update your Flink programs. They use Flink’s checkpointing mechanism to create a (non-incremental) snapshot of the state of your streaming program and write the checkpoint data and meta data out to an external file system.
This page covers all steps involved in triggering, restoring, and disposing savepoints. For more details on how Flink handles state and failures in general, check out the State in Streaming Programs page.
It is highly recommended that you adjust your programs as described in this section in order to be able to upgrade your programs in the future. The main required change is to manually specify operator IDs via the uid(String)
method. These IDs are used to scope the state of each operator.
If you don’t specify the IDs manually they will be generated automatically. You can automatically restore from the savepoint as long as these IDs do not change. The generated IDs depend on the structure of your program and are sensitive to program changes. Therefore, it is highly recommended to assign these IDs manually.
You can think of a savepoint as holding a map of Operator ID -> State
for each stateful operator:
In the above example, the print sink is stateless and hence not part of the savepoint state. By default, we try to map each entry of the savepoint back to the new program.
You can use the command line client to trigger savepoints, cancel a job with a savepoint, resume from savepoints, and dispose savepoints.
With Flink >= 1.2.0 it is also possible to resume from savepoints using the webui.
When triggering a savepoint, a new savepoint directory is created where the data as well as the meta data will be stored. The location of this directory can be controlled by configuring a default target directory or by specifying a custom target directory with the trigger commands (see the :targetDirectory
argument).
For example with a FsStateBackend
or RocksDBStateBackend
:
_metadata
file.
Please follow FLINK-5778 for progress on lifting this restriction.
Note that if you use the MemoryStateBackend
, metadata and savepoint state will be stored in the _metadata
file. Since it is self-contained, you may move the file and restore from any location.
This will trigger a savepoint for the job with ID :jobId
, and returns the path of the created savepoint. You need this path to restore and dispose savepoints.
This will trigger a savepoint for the job with ID :jobId
and YARN application ID :yarnAppId
, and returns the path of the created savepoint.
This will atomically trigger a savepoint for the job with ID :jobid
and cancel the job. Furthermore, you can specify a target file system directory to store the savepoint in. The directory needs to be accessible by the JobManager(s) and TaskManager(s).
This submits a job and specifies a savepoint to resume from. You may give a path to either the savepoint’s directory or the _metadata
file.
By default the resume operation will try to map all state of the savepoint back to the program you are restoring with. If you dropped an operator, you can allow to skip state that cannot be mapped to the new program via --allowNonRestoredState
(short: -n
) option:
This disposes the savepoint stored in :savepointPath
.
Note that it is possible to also manually delete a savepoint via regular file system operations without affecting other savepoints or checkpoints (recall that each savepoint is self-contained). Up to Flink 1.2, this was a more tedious task which was performed with the savepoint command above.
You can configure a default savepoint target directory via the state.savepoints.dir
key. When triggering savepoints, this directory will be used to store the savepoint. You can overwrite the default by specifying a custom target directory with the trigger commands (see the :targetDirectory
argument).
If you neither configure a default nor specify a custom target directory, triggering the savepoint will fail.
As a rule of thumb, yes. Strictly speaking, it is sufficient to only assign IDs via the uid
method to the stateful operators in your job. The savepoint only contains state for these operators and stateless operator are not part of the savepoint.
In practice, it is recommended to assign it to all operators, because some of Flink’s built-in operators like the Window operator are also stateful and it is not obvious which built-in operators are actually stateful and which are not. If you are absolutely certain that an operator is stateless, you can skip the uid
method.
When you add a new operator to your job it will be initialized without any state. Savepoints contain the state of each stateful operator. Stateless operators are simply not part of the savepoint. The new operator behaves similar to a stateless operator.
By default, a savepoint restore will try to match all state back to the restored job. If you restore from a savepoint that contains state for an operator that has been deleted, this will therefore fail.
You can allow non restored state by setting the --allowNonRestoredState
(short: -n
) with the run command:
If you assigned IDs to these operators, they will be restored as usual.
If you did not assign IDs, the auto generated IDs of the stateful operators will most likely change after the reordering. This would result in you not being able to restore from a previous savepoint.
If you assigned IDs to your stateful operators, the stateless operators will not influence the savepoint restore.
If you did not assign IDs, the auto generated IDs of the stateful operators will most likely change after the reordering. This would result in you not being able to restore from a previous savepoint.
If the savepoint was triggered with Flink >= 1.2.0 and using no deprecated state API like Checkpointed
, you can simply restore the program from a savepoint and specify a new parallelism.
If you are resuming from a savepoint triggered with Flink < 1.2.0 or using now deprecated APIs you first have to migrate your job and savepoint to Flink >= 1.2.0 before being able to change the parallelism. See the upgrading jobs and Flink versions guide.