You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by uc...@apache.org on 2017/01/19 13:31:39 UTC

flink git commit: [FLINK-5377] [docs] Stress importance of UIDs for savepoints

Repository: flink
Updated Branches:
  refs/heads/master bf72a7c02 -> 6d8b3f5a1


[FLINK-5377] [docs] Stress importance of UIDs for savepoints


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

Branch: refs/heads/master
Commit: 6d8b3f5a1c28ba846352abd6a2750d1ae020fa5a
Parents: bf72a7c
Author: Ufuk Celebi <uc...@apache.org>
Authored: Wed Dec 21 12:24:35 2016 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Thu Jan 19 14:30:51 2017 +0100

----------------------------------------------------------------------
 docs/fig/savepoints-overview.png    | Bin 62824 -> 0 bytes
 docs/fig/savepoints-program_ids.png | Bin 55492 -> 0 bytes
 docs/setup/savepoints.md            | 177 ++++++++++++++++++++++++-------
 3 files changed, 138 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6d8b3f5a/docs/fig/savepoints-overview.png
----------------------------------------------------------------------
diff --git a/docs/fig/savepoints-overview.png b/docs/fig/savepoints-overview.png
deleted file mode 100644
index c0e7563..0000000
Binary files a/docs/fig/savepoints-overview.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/6d8b3f5a/docs/fig/savepoints-program_ids.png
----------------------------------------------------------------------
diff --git a/docs/fig/savepoints-program_ids.png b/docs/fig/savepoints-program_ids.png
deleted file mode 100644
index cc161ef..0000000
Binary files a/docs/fig/savepoints-program_ids.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/6d8b3f5a/docs/setup/savepoints.md
----------------------------------------------------------------------
diff --git a/docs/setup/savepoints.md b/docs/setup/savepoints.md
index 2a1f631..eb12dd9 100644
--- a/docs/setup/savepoints.md
+++ b/docs/setup/savepoints.md
@@ -22,70 +22,169 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Programs written in the [Data Stream API](index.html) can resume execution from a **savepoint**. Savepoints allow both updating your programs and your Flink cluster without losing any state. This page covers all steps to trigger, restore, and dispose savepoints. For more details on how Flink handles state and failures, check out the [State in Streaming Programs]({{ site.baseurl }}/ops/state_backends.html) and [Fault Tolerance](fault_tolerance.html) pages.
-
 * toc
 {:toc}
 
 ## Overview
 
-In the example below the workers produce checkpoints **c<sub>1</sub>**, **c<sub>2</sub>**,
-**c<sub>3</sub>**, and **c<sub>4</sub>** for job *0xA312Bc*. Periodic checkpoints **c<sub>1</sub>**
-and **c<sub>3</sub>** have already been *discarded* and **c<sub>4</sub>** is the *latest
-checkpoint*. **c<sub>2</sub> is special**. It is the state associated with the savepoint
-**s<sub>1</sub>** and has been triggered by the user and it doesn't expire automatically (as
-c<sub>1</sub> and c<sub>3</sub> did after the completion of newer checkpoints).
+Savepoints are externally stored checkpoints that you can use to stop-and-resume or update your Flink programs. They use Flink's [checkpointing mechanism]({{ site/baseurl }}/internals/stream_checkpointing.html) to create a snapshot of the state of your streaming program and write the checkpoint meta data out to an external file system.
+
+This page covers all steps involved in triggering, restoring, and disposing savepoints. In order to allow upgrades between programs and Flink versions, it is important to check out the section about [assigning IDs to your operators](#assigning-operator-ids).
+
+For more details on how Flink handles state and failures in general, check out the [State in Streaming Programs]({{ site.baseurl }}/dev/stream/state.html) page.
+
+## Assigning Operator IDs
+
+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.
+
+{% highlight java %}
+DataStream<String> stream = env.
+  // Stateful source (e.g. Kafka) with ID
+  .addSource(new StatefulSource())
+  .uid("source-id") // ID for the source operator
+  .shuffle()
+  // Stateful mapper with ID
+  .map(new StatefulMapper())
+  .uid("mapper-id") // ID for the mapper
+  // Stateless printing sink
+  .print(); // Auto-generated ID
+{% endhighlight %}
+
+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.
+
+### Savepoint State
+
+You can think of a savepoint as holding a map of `Operator ID -> State` for each stateful operator:
+
+```
+Operator ID | State
+------------+------------------------
+source-id   | State of StatefulSource
+mapper-id   | State of StatefulMapper
+```
+
+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.
+
+## Operations
+
+You can use the [command line client]({{ site.baseurl }}/setup/cli.html#savepoints) to *trigger savepoints*, *cancel a job with a savepoint*, *resume from savepoints*, and *dispose savepoints*.
+
+### Triggering Savepoints
+
+When triggering a savepoint, a single savepoint file will be created that contains the checkpoint *meta data*. The actual checkpoint state will be kept around in the configured checkpoint directory. For example with a `FsStateBackend` or `RocksDBStateBackend`:
+
+```sh
+# Savepoint file contains the checkpoint meta data
+/savepoints/savepoint-123123
+
+# Checkpoint directory contains the actual state
+/checkpoints/:jobid/chk-:id/...
+```
+
+The savepoint file is usually much smaller than the actual checkpointed state. Note that if you use the `MemoryStateBackend`, the savepoint file will be self-contained and contain all the state.
+
+#### Trigger a Savepoint
+
+```sh
+$ bin/flink savepoint :jobId [:targetDirectory]
+```
+
+This will trigger a savepoint for the job with ID `:jobid`. Furthermore, you can specify a target file system directory to store the savepoint in. The directory needs to be accessible by the JobManager.
+
+If you don't specify a target directory, you need to have [configured a default directory](#configuration). Otherwise, triggering the savepoint will fail.
+
+#### Cancel Job with Savepoint
 
-<img src="{{ site.baseurl }}/fig/savepoints-overview.png" class="center" />
+```sh
+$ bin/flink cancel -s [:targetDirectory] :jobId
+```
 
-Note that **s<sub>1</sub>** is only a **pointer to the actual checkpoint data c<sub>2</sub>**. This means that the actual state is *not copied* for the savepoint and periodic checkpoint data is kept around.
+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.
 
-## Configuration
+If you don't specify a target directory, you need to have [configured a default directory](#configuration). Otherwise, cancelling the job with a savepoint will fail.
 
-Savepoints are stored in a configured **file system directory**. They are available between cluster instances and allow you to move your program to another cluster.
+### Resuming from Savepoints
 
-<pre>
+```sh
+$ bin/flink run -s :savepointPath [:runArgs]
+```
+
+This submits a job and specifies the savepoint path. The execution will resume from the respective savepoint state. The savepoint file holds the meta data of a checkpoint and points to the actual checkpoint files. This is why the savepoint file is usually much smaller than the actual checkpoint state.
+
+#### Allowing Non-Restored State
+
+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:
+
+```sh
+$ bin/flink run -s :savepointPath -n [:runArgs]
+```
+
+### Disposing Savepoints
+
+```sh
+$ bin/flink savepoint -d :savepointPath
+```
+
+This disposes the savepoint stored in `:savepointPath`.
+
+Note that since savepoints always go to a file system it is possible to also manually delete the savepoint via a regular file system operation. Keep in mind though that the savepoint only stores meta data that points to the actual checkpoint data. Therefore, if you manually want to delete a savepoint, you would have to include the checkpoint files as well. Since there is currently no straight forward way to figure out how a savepoint maps to a checkpoint, it is recommended to use the savepoint tool for this as described above.
+
+### Configuration
+
+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 meta data. You can overwrite the default by specifying a custom target directory with the trigger commands (see the [`:targetDirectory` argument](#trigger-a-savepoint)).
+
+```sh
+# Default savepoint target directory
 state.savepoints.dir: hdfs:///flink/savepoints
-</pre>
+```
 
-**Note**: If you don't configure a specific directory, triggering the savepoint will fail.
+If you neither configure a default nor specify a custom target directory, triggering the savepoint will fail.
 
-**Important**: A savepoint is a pointer to a completed checkpoint. That means that the state of a savepoint is not only found in the savepoint file itself, but also needs the actual checkpoint data (e.g. in a set of further files).
+## F.A.Q
 
-## Changes to your program
+### Should I assign IDs to all operators in my job?
 
-Savepoints **work out of the box**, but it is **highly recommended** that you slightly adjust your programs in order to be able to work with savepoints in future versions of your program.
+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.
 
-<img src="{{ site.baseurl }}/fig/savepoints-program_ids.png" class="center" />
+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.
 
-For savepoints **only stateful tasks matter**. In the above example, the source and map tasks are stateful whereas the sink is not stateful. Therefore, only the state of the source and map tasks are part of the savepoint.
+### Why is the savepoint file so small?
 
-Each task is identified by its **generated task IDs** and **subtask index**. In the above example the state of the source (**s<sub>1</sub>**, **s<sub>2</sub>**) and map tasks (**m<sub>1</sub>**, **m<sub>2</sub>**) is identified by their respective task ID (*0xC322EC* for the source tasks and *0x27B3EF* for the map tasks) and subtask index. There is no state for the sinks (**t<sub>1</sub>**, **t<sub>2</sub>**). Their IDs therefore do not matter.
+The savepoint file only contains the meta data of the checkpoint and has pointers to the checkpoint state, which is usually much larger. In case of using the `MemoryStateBackend`, the checkpoint will include all state, but is constrained by the backend to small state.
 
-<span class="label label-danger">Important</span> The IDs are generated **deterministically** from your program structure. This means that as long as your program does not change, the IDs do not change. **The only allowed changes are within the user function, e.g. you can change the implemented `MapFunction` without changing the topology**. In this case, it is straight forward to restore the state from a savepoint by mapping it back to the same task IDs and subtask indexes. This allows you to work with savepoints out of the box, but gets problematic as soon as you make changes to the topology, because they result in changed IDs and the savepoint state cannot be mapped to your program any more.
+### What happens if I add a new operator that requires state to my job?
 
-<span class="label label-info">Recommended</span> In order to be able to change your program and **have fixed IDs**, the *DataStream* API provides a method to manually specify the task IDs. Each operator provides a **`uid(String)`** method to override the generated ID. The ID is a String, which will be deterministically hashed to a 16-byte hash value. It is **important** that the specified IDs are **unique per transformation and job**. If this is not the case, job submission will fail.
+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.
 
-{% highlight scala %}
-DataStream<String> stream = env.
-  // Stateful source (e.g. Kafka) with ID
-  .addSource(new StatefulSource())
-  .uid("source-id")
-  .shuffle()
-  // The stateful mapper with ID
-  .map(new StatefulMapper())
-  .uid("mapper-id")
+### What happens if I delete an operator that has state from my job?
 
-// Stateless sink (no specific ID required)
-stream.print()
-{% endhighlight %}
+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. 
 
-## Command-line client
+You can allow non restored state by setting the `--allowNonRestoredState` (short: `-n`) with the run command:
 
-You control the savepoints via the [command line client]({{ site.baseurl }}/setup/cli.html#savepoints).
+```sh
+$ bin/flink run -s :savepointPath -n [:runArgs]
+```
 
-## Current limitations
+### What happens if I reorder stateful operators in my job?
+
+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.
+
+### What happens if I add or delete or reorder operators that have no state in my job?
 
-- **Parallelism**: When restoring a savepoint, the parallelism of the program has to match the parallelism of the original program from which the savepoint was drawn. There is no mechanism to re-partition the savepoint's state yet.
+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.
+
+### What happens when I change the parallelism of my program when restoring?
+
+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]({{ site.baseurl }}/ops/upgrading.html).
+
+## Current limitations
 
 - **Chaining**: Chained operators are identified by the ID of the first task. It's not possible to manually assign an ID to an intermediate chained task, e.g. in the chain `[  a -> b -> c ]` only **a** can have its ID assigned manually, but not **b** or **c**. To work around this, you can [manually define the task chains](index.html#task-chaining-and-resource-groups). If you rely on the automatic ID assignment, a change in the chaining behaviour will also change the IDs.
+