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

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

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

    https://github.com/apache/flink/pull/3870#discussion_r116175297
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalKeyedStateHandle.java
 ---
    @@ -180,69 +176,66 @@ public long getStateSize() {
     
        @Override
        public void registerSharedStates(SharedStateRegistry stateRegistry) {
    +
                Preconditions.checkState(!registered, "The state handle has 
already registered its shared states.");
     
    -           for (Map.Entry<String, StreamStateHandle> newSstFileEntry : 
newSstFiles.entrySet()) {
    -                   SstFileStateHandle stateHandle = new 
SstFileStateHandle(newSstFileEntry.getKey(), newSstFileEntry.getValue());
    +           for (Map.Entry<String, StreamStateHandle> newSstFileEntry : 
unregisteredSstFiles.entrySet()) {
    --- End diff --
    
    For FLINK-6545 we need to familiarize the savepoint serializer with the new 
incremental handles, but currently they are in the RocksDB package, invisible 
for savepoint classes. 
    
    I am currently already working on making incremental snapshots a concept on 
a higher abstraction level, that is less tightly coupled to RocksDB. I think 
that we can then have incremental snapshots also for other backends, e.g. the 
memory based. The abstraction is simply, that all incremental snapshots can be 
divided into backend meta data, private data, newly created shared data and 
referenced shared data. Also the placeholder handle will become publicly 
available. Would this address comments? I could make those preparations already 
part of this PR.


> Make incremental checkpoints externalizable
> -------------------------------------------
>
>                 Key: FLINK-6545
>                 URL: https://issues.apache.org/jira/browse/FLINK-6545
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.3.0
>            Reporter: Stefan Richter
>            Priority: Blocker
>             Fix For: 1.3.0
>
>
> Incremental checkpoints are currently not externalizible. The missing piece 
> is familiarizing the {{SavepointSerializer}}(s) with the new state handles 
> classes that we added for incremental checkpointing. Currently, some of those 
> (e.g. 
> {{org.apache.flink.contrib.streaming.state.RocksDBIncrementalKeyedStateHandle}})
>  currently live in the contrib.rocksdb package and need to be migrated. We 
> can also push them to a different abstraction level, i.e. 
> {{IncrementalKeyedStateHandle}} with some private data, referenced existing 
> shared data (from previous checkpoints), and (presumably) newly created 
> shared data (first created by the current checkpoint).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to