[
https://issues.apache.org/jira/browse/FLINK-2976?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15044988#comment-15044988
]
ASF GitHub Bot commented on FLINK-2976:
---------------------------------------
Github user aljoscha commented on the pull request:
https://github.com/apache/flink/pull/1434#issuecomment-162532182
Very nice, I haven't yet gone through the code in detail but from the
description and doc it looks very good.
One thing I was confused about is the naming of the config options:
```
state.backend.savepoints: filesystem
state.backend.savepoints.fs.dir: hdfs:///flink/savepoints
```
I think this could confuse users into thinking that a different state
backend is used on the task manager to store the actual checkpoint data. If I'm
not mistaken this is purely a JobManager thing, though. Also, before the naming
scheme for stuff in `state.backend` was that `state.backend.x.y` was the
setting of option `y` for state backend `x`. Now it looks like there is a state
backend `savepoints`. Maybe we could change this to:
```
savepoints.state.backend: filesystem
savepoints.state.backend.fs.dir: hdfs:///flink/savepoints
```
to clarify that this is changing the state backend settings for the
savepoints on the JobManager.
> 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)