[
https://issues.apache.org/jira/browse/FLINK-4067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15362589#comment-15362589
]
ASF GitHub Bot commented on FLINK-4067:
---------------------------------------
Github user tillrohrmann commented on the issue:
https://github.com/apache/flink/pull/2194
Really good work @uce. The code is well structured and thoroughly tested. I
had only some minor comments.
While testing the code with the streaming state machine job I stumbled
across a problem, though. Recovering from a Flink 1.0 savepoint does not work
if the job contains a `keyBy` operation. The reason is that we had a faulty
murmur hash implementation in Flink 1.0 and due to its correction, the mapping
of keys to sub tasks has changed. Consequently, the restored state no longer
matches the assigned key spaces for each operator. This is the problematic
[commit](https://github.com/apache/flink/commit/641a0d436c9b7a34ff33ceb370cf29962cac4dee).
Thus, this change is actually breaking our backwards compatibility with
respect to savepoints. In order to solve the problem I see three possibilities:
- Revert the changes of this commit. But we don't know how the flawed
murmur hash performs.
- Develop a tool which can repartition savepoints
- Don't support backwards compatibility between version 1.0 and 1.1
I think that option 3 is not doable given our backwards compatibility
promise. Furthermore, option 2 is not really straight forward, if the user has
a keyed stream where he uses the `Checkpointed` interface. Given that the
release is upcoming, I think option 1 would be the best way to solve the
problem.
> 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)