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

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

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

    https://github.com/apache/flink/pull/5239#discussion_r169724637
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ---
    @@ -2123,9 +2162,21 @@ void takeSnapshot() throws Exception {
                                checkpointId,
                                sstFiles,
                                miscFiles,
    -                           metaStateHandle);
    +                           metaStateHandle.getJobManagerOwnedSnapshot());
    +
    +                   DirectoryStateHandle directoryStateHandle = 
localBackupDirectory.completeSnapshotAndGetHandle();
    +                   StreamStateHandle taskLocalSnapshotMetaDataStateHandle 
= metaStateHandle.getTaskLocalSnapshot();
    +                   IncrementalLocalKeyedStateHandle 
directoryKeyedStateHandle =
    +                           directoryStateHandle != null && 
taskLocalSnapshotMetaDataStateHandle != null ?
    +                                   new IncrementalLocalKeyedStateHandle(
    +                                           stateBackend.backendUID,
    +                                           checkpointId,
    +                                           directoryStateHandle,
    +                                           stateBackend.keyGroupRange,
    +                                           
taskLocalSnapshotMetaDataStateHandle) :
    +                                   null;
     
    -                   return new 
SnapshotResult<>(incrementalKeyedStateHandle, null);
    +                   return new 
SnapshotResult<>(incrementalKeyedStateHandle, directoryKeyedStateHandle);
    --- End diff --
    
    I guess you changed the creation of the `SnapshotResult` in a fixup commit 
in your local branch, right? Otherwise we might refactor this in order to get 
rid of the many `nulls` in the lines above.


> Implement task-local state recovery
> -----------------------------------
>
>                 Key: FLINK-8360
>                 URL: https://issues.apache.org/jira/browse/FLINK-8360
>             Project: Flink
>          Issue Type: New Feature
>          Components: State Backends, Checkpointing
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>             Fix For: 1.5.0
>
>
> This issue tracks the development of recovery from task-local state. The main 
> idea is to have a secondary, local copy of the checkpointed state, while 
> there is still a primary copy in DFS that we report to the checkpoint 
> coordinator.
> Recovery can attempt to restore from the secondary local copy, if available, 
> to save network bandwidth. This requires that the assignment from tasks to 
> slots is as sticky is possible.
> For starters, we will implement this feature for all managed keyed states and 
> can easily enhance it to all other state types (e.g. operator state) later.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to