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

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_r168514168
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java
 ---
    @@ -166,20 +190,38 @@ public TaskStateSnapshot retrieveLocalState(long 
checkpointID) {
         * Disposes the state of all local snapshots managed by this object.
         */
        public void dispose() {
    +
    +           Collection<Map.Entry<Long, TaskStateSnapshot>> statesCopy;
    +
                synchronized (lock) {
    -                   for (Map.Entry<Long, TaskStateSnapshot> entry : 
storedTaskStateByCheckpointID.entrySet()) {
    -                           discardStateObject(entry.getValue(), 
entry.getKey());
    -                   }
                        discarded = true;
    +                   statesCopy = new 
ArrayList<>(storedTaskStateByCheckpointID.entrySet());
                }
    +
    +           discardExecutor.execute(() -> {
    --- End diff --
    
    I think it would be good for these kind of asynchronous clean up operations 
to return a `CompletableFuture`. This future could then be returned by the 
`dispose` method. The benefit would be that the caller would know when the 
clean up has completed and, thus, would be safe to shut down the 
`discardExecutor`.


> 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