[
https://issues.apache.org/jira/browse/FLINK-2976?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15045313#comment-15045313
]
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_r46850717
--- Diff:
flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
---
@@ -753,7 +841,11 @@ class JobManager(
* @param jobInfo the job info
* @param isRecovery Flag indicating whether this is a recovery or
initial submission
*/
- private def submitJob(jobGraph: JobGraph, jobInfo: JobInfo, isRecovery:
Boolean = false): Unit = {
+ private def submitJob(
+ jobGraph: JobGraph,
+ jobInfo: JobInfo,
+ isRecovery: Boolean = false): Unit = {
--- End diff --
two level indentation of parameters and one level indentation of return
type:
```
private def submitJob(
a: B
c: D)
: R = {
}
```
> 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)