[docs] Add docs about externalized checkpoints

Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/36983797
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/36983797
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/36983797

Branch: refs/heads/master
Commit: 3698379715a9373b3de59bd4cd12b6cb69154361
Parents: d965d5a
Author: Ufuk Celebi <[email protected]>
Authored: Tue Dec 13 11:32:43 2016 +0100
Committer: Ufuk Celebi <[email protected]>
Committed: Tue Dec 13 11:58:33 2016 +0100

----------------------------------------------------------------------
 docs/setup/cli.md             |  4 ++--
 docs/setup/config.md          |  2 ++
 docs/setup/fault_tolerance.md | 25 +++++++++++++++++++++++++
 3 files changed, 29 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/36983797/docs/setup/cli.md
----------------------------------------------------------------------
diff --git a/docs/setup/cli.md b/docs/setup/cli.md
index 855757f..ef5adb3 100644
--- a/docs/setup/cli.md
+++ b/docs/setup/cli.md
@@ -146,7 +146,7 @@ This allows the job to finish processing all inflight data.
 
 Returns the path of the created savepoint. You need this path to restore and 
dispose savepoints.
 
-You can optionally specify a `savepointDirectory` when triggering the 
savepoint. If you don't specify one here, you need to configure a default 
savepoint directory for the Flink installation (see 
[[savepoint.html#configuration]]).
+You can optionally specify a `savepointDirectory` when triggering the 
savepoint. If you don't specify one here, you need to configure a default 
savepoint directory for the Flink installation (see 
[Savepoints](savepoints.html#configuration)).
 
 ##### Cancel with a savepoint
 
@@ -156,7 +156,7 @@ You can atomically trigger a savepoint and cancel a job.
 ./bin/flink cancel -s  [savepointDirectory] <jobID>
 {% endhighlight %}
 
-If no savepoint directory is configured, you need to configure a default 
savepoint directory for the Flink installation (see 
[[savepoint.html#configuration]]).
+If no savepoint directory is configured, you need to configure a default 
savepoint directory for the Flink installation (see 
[Savepoints](savepoints.html#configuration)).
 
 The job will only be cancelled if the savepoint succeeds.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/36983797/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index 51ef41c..c24065a 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -155,6 +155,8 @@ will be used under the directory specified by 
jobmanager.web.tmpdir.
 
 - `state.backend.rocksdb.checkpointdir`:  The local directory for storing 
RocksDB files, or a list of directories separated by the systems directory 
delimiter (for example ‘:’ (colon) on Linux/Unix). (DEFAULT value is 
`taskmanager.tmp.dirs`)
 
+- `state.checkpoints.dir`: The target directory for meta data of [externalized 
checkpoints]({{ site.baseurl 
}}/setup/fault_tolerance.md#externalized-checkpoints).
+
 - `high-availability.zookeeper.storageDir`: Required for HA. Directory for 
storing JobManager metadata; this is persisted in the state backend and only a 
pointer to this state is stored in ZooKeeper. Exactly like the checkpoint 
directory it must be accessible from the JobManager and a local filesystem 
should only be used for local deployments. Previously this key was named 
`recovery.zookeeper.storageDir`.
 
 - `blob.storage.directory`: Directory for storing blobs (such as user jar's) 
on the TaskManagers.

http://git-wip-us.apache.org/repos/asf/flink/blob/36983797/docs/setup/fault_tolerance.md
----------------------------------------------------------------------
diff --git a/docs/setup/fault_tolerance.md b/docs/setup/fault_tolerance.md
index 7fb3df4..cef746e 100644
--- a/docs/setup/fault_tolerance.md
+++ b/docs/setup/fault_tolerance.md
@@ -95,6 +95,31 @@ env.getCheckpointConfig.setMaxConcurrentCheckpoints(1)
 
 {% top %}
 
+### Externalized Checkpoints
+
+You can configure periodic checkpoints to be persisted externally. 
Externalized checkpoints write their meta data out to persistent storage and 
are *not* automatically cleaned up when the job fails. This way, you will have 
a checkpoint around to resume from if your job fails.
+
+```java
+CheckpoingConfig config = env.getCheckpointConfig();
+config.enableExternalizedCheckpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
+```
+
+The `ExternalizedCheckpointCleanup` mode configures what happens with 
externalized checkpoints when you cancel the job:
+
+- **`ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION`**: Retain the 
externalized checkpoint when the job is cancelled. Note that you have to 
manually clean up the checkpoint state after cancellation in this case.
+
+- **`ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION`**: Delete the 
externalized checkpoint when the job is cancelled. The checkpoint state will 
only be available if the job fails.
+
+The **target directory** for the checkpoint is determined from the default 
checkpoint directory configuration. This is configured via the configuration 
key `state.checkpoints.dir`, which should point to the desired target directory:
+
+```
+state.checkpoints.dir: hdfs:///checkpoints/
+```
+
+This directory will then contain the checkpoint meta data required to restore 
the checkpoint. The actual checkpoint files will still be available in their 
configured directory. You currently can only set this via the configuration 
files.
+
+Follow the [savepoint guide]({{ site.baseurl }}/setup/cli.html#savepoints) 
when you want to resume from a specific checkpoint.
+
 ### Fault Tolerance Guarantees of Data Sources and Sinks
 
 Flink can guarantee exactly-once state updates to user-defined state only when 
the source participates in the

Reply via email to