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

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

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

    https://github.com/apache/flink/pull/2194#discussion_r69454464
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java
 ---
    @@ -41,13 +41,13 @@
        void stop();
     
        /**
    -    * Creates a {@link CompletedCheckpointStore} instance for a job.
    +    * Creates a {@link CheckpointStore} instance for a job.
         *
         * @param jobId           Job ID to recover checkpoints for
         * @param userClassLoader User code class loader of the job
    -    * @return {@link CompletedCheckpointStore} instance for the job
    +    * @return {@link CheckpointStore} instance for the job
         */
    -   CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, 
ClassLoader userClassLoader)
    +   CheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader 
userClassLoader)
    --- End diff --
    
    Maybe we could rename this method to `createCheckpointStore` if not public.


> Add version header to savepoints
> --------------------------------
>
>                 Key: FLINK-4067
>                 URL: https://issues.apache.org/jira/browse/FLINK-4067
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.0.3
>            Reporter: Ufuk Celebi
>            Assignee: Ufuk Celebi
>             Fix For: 1.1.0
>
>
> Adding a header with version information to savepoints ensures that we can 
> migrate savepoints between Flink versions in the future (for example when 
> changing internal serialization formats between versions).
> After talking with Till, we propose to add the following meta data:
> - Magic number (int): identify data as savepoint
> - Version (int): savepoint version (independent of Flink version)
> - Data Offset (int): specifies at which point the actual savepoint data 
> starts. With this, we can allow future Flink versions to add fields to the 
> header without breaking stuff, e.g. Flink 1.1 could read savepoints of Flink 
> 2.0.
> For Flink 1.0 savepoint support, we have to try reading the savepoints 
> without a header before failing if we don't find the magic number.



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

Reply via email to