[ 
https://issues.apache.org/jira/browse/FLINK-2976?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15045146#comment-15045146
 ] 

ASF GitHub Bot commented on FLINK-2976:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1434#discussion_r46838672
  
    --- Diff: docs/apis/savepoints.md ---
    @@ -0,0 +1,108 @@
    +---
    +title: "Savepoints"
    +is_beta: false
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +Programs written in the [Data Stream API]({{ site.baseurl 
}}/apis/streaming_guide.html) can resume execution from a **savepoint**. 
Savepoints allow both updating your programs and your Flink cluster without 
loosing 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 }}/apis/state_backends.html) 
and [Fault Tolerance]({{ site.baseurl }}/apis/fault_tolerance.html) pages.
    +
    +* toc
    +{:toc}
    +
    +## Overview
    +
    +Savepoints are **manually triggered checkpoints**, which take a snapshot 
of the program and write it out to a state backend. They rely on the regular 
checkpointing mechanism for this. During execution programs are periodically 
snapshotted on the worker nodes and produce checkpoints. For recovery only the 
last completed checkpoint is needed and older checkpoints can be safely 
discarded as soon as a new one is completed.
    +
    +Savepoints are similar to these periodic checkpoints except that they are 
**triggered by the user** and **don't automatically expire** when newer 
checkpoints are completed.
    +
    +<img src="fig/savepoints-overview.png" class="center" />
    +
    +In the above example 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).
    +
    +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.
    +
    +## Configuration
    +
    +Savepoints point to regular checkpoints and store their state in a 
configured [state backend]({{ site.baseurl }}/apis/state_backends.html). 
Currently, the supported state backends are **jobmanager** and **filesystem**. 
The state backend configuration for the regular periodic checkpoints is 
**independent** of the savepoint state backend configuration. Checkpoint data 
is **not copied** for savepoints, but points to the configured checkpoint state 
backend.
    +
    +### JobManager
    +
    +This is the **default backend** for savepoints.
    +
    +Savepoints are stored on the heap of the job manager. They are *lost* 
after the job manager is shut down. This mode is only useful if you want to 
*stop* and *resume* your program while the **same cluster** keeps running. It 
is *not recommended* for production use. Savepoints are *not* part the [job 
manager's highly availabile]({{ site.baseurl 
}}/setup/jobmanager_high_availability.html) state.
    --- End diff --
    
    Typos: *not* part **of** the
    
    highly available


> Save and load checkpoints manually
> ----------------------------------
>
>                 Key: FLINK-2976
>                 URL: https://issues.apache.org/jira/browse/FLINK-2976
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Runtime
>    Affects Versions: 0.10.0
>            Reporter: Ufuk Celebi
>             Fix For: 1.0.0
>
>
> Currently, all checkpointed state is bound to a job. After the job finishes 
> all state is lost. In case of an HA cluster, jobs can live longer than the 
> cluster, but they still suffer from the same issue when they finish.
> Multiple users have requested the feature to manually save a checkpoint in 
> order to resume from it at a later point. This is especially important for 
> production environments. As an example, consider upgrading your existing 
> production Flink program. Currently, you loose all the state of your program. 
> With the proposed mechanism, it will be possible to save a checkpoint, stop 
> and update your program, and then continue your program with the  checkpoint.
> The required operations can be simple:
> saveCheckpoint(JobID) => checkpointID: long
> loadCheckpoint(JobID, long) => void
> For the initial version, I would apply the following restriction:
> - The topology needs to stay the same (JobGraph parallelism, etc.)
> A user can configure this behaviour via the environment like the 
> checkpointing interval. Furthermore, the user can trigger the save operation 
> via the command line at arbitrary times and load a checkpoint when submitting 
> a job, e.g.
> bin/flink checkpoint <JobID> => checkpointID: long 
> and
> bin/flink run --loadCheckpoint JobID [latest saved checkpoint]
> bin/flink run --loadCheckpoint (JobID,long) [specific saved checkpoint]
> As far as I can tell, the required mechanisms are similar to the ones 
> implemented for JobManager high availability. We need to make sure to persist 
> the CompletedCheckpoint instances as a pointer to the checkpoint state and to 
> *not* remove saved checkpoint state.
> On the client side, we need to give the job and its vertices the same IDs to 
> allow mapping the checkpoint state.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to