GitHub user uce opened a pull request:

    https://github.com/apache/flink/pull/2194

    [FLINK-4067] [runtime] Add savepoint headers

    Savepoints were previously persisted without any meta data using default 
Java serialization of a `CompletedCheckpoint`. This PR introduces a savepoint 
class with version-specific serializers and stores savepoints with meta data.
    
    Savepoints expose a version number and a `Collection<TaskState>` for 
savepoint restore.
    
    Currently, there are two savepoint versions:
    
    - `SavepointV0` (Flink 1.0): This is a wrapper around the Flink 1.0 
`CompletedCheckpoint`. Because this class changed since Flink 1.0 I had to 
re-introduce it with this PR in order to allow restoring from Flink 1.0 
savepoints (otherwise the old `StateForTask` state can not be deserialized via 
default Java serialization). Therefore, I've renamed the Flink 1.1 
`CompletedCheckpoint` to `Checkpoint`. I've annotated the re-introduced classes 
with `@Deprecated`. We can remove those with Flink 1.2.
    - `SavepointV1` (Flink 1.1): This is the current savepoint version, which 
holds a reference to the `Checkpoint` task state collection, but is serialized 
with a custom serializater not relying on default Java serialization. 
Therefore, it should not happen again that we need to stick to certain classes 
in future Flink versions.
    
    The savepoints are stored in `FsSavepointStore` with the following format:
    
    ```
    MagicNumber SavepointVersion Savepoint
      - MagicNumber => int
      - SavepointVersion => int (returned by Savepoint#getVersion())
      - Savepoint => bytes (serialized via version-specific SavepointSerializer)
    ```
    
    The header is minimal (magic number, version). All savepoint-specific meta 
data can be moved to the savepoint itself. This is also were we would have to 
add new meta data in future versions, allowing us to differentiate between 
different savepoint versions when we change the serialization stack etc.
    
    I've tested both manually and via a unit test that it works to trigger a 
savepoint via Flink 1.0 and resume from it with Flink 1.1.
    
    All savepoint related classes have been moved from `checkpoint` to a new 
sub package `checkpoint.savepoint`.
    
    The main classes to look at to review this PR are `Savepoint` and 
subclasses, `SavepointSerializer` and subclasses, and `FsSavepointStore`. 
@tillrohrmann do you have time to do this?

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/uce/flink 4067-savepoint_header

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2194.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2194
    
----
commit af0d418ba2bc3591f0526476d1ad7dfc4160c205
Author: Ufuk Celebi <[email protected]>
Date:   2016-06-30T14:16:41Z

    [FLINK-4067] [runtime] Add savepoint headers

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to