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

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_r165307761
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java
 ---
    @@ -355,10 +355,11 @@ public void testSnapshotEmpty() throws Exception {
                CheckpointStreamFactory streamFactory =
                                abstractStateBackend.createStreamFactory(new 
JobID(), "testOperator");
     
    -           RunnableFuture<OperatorStateHandle> snapshot =
    +           RunnableFuture<SnapshotResult<OperatorStateHandle>> snapshot =
                                operatorStateBackend.snapshot(0L, 0L, 
streamFactory, CheckpointOptions.forCheckpoint());
     
    -           OperatorStateHandle stateHandle = 
FutureUtil.runIfNotDoneAndGet(snapshot);
    +           SnapshotResult<OperatorStateHandle> snapshotResult = 
FutureUtil.runIfNotDoneAndGet(snapshot);
    +           OperatorStateHandle stateHandle = snapshotResult != null ? 
snapshotResult.getJobManagerOwnedSnapshot() : null;
    --- End diff --
    
    This seems a bit circumstantial. Can't we make it that `snapshotResult` is 
always not null?


> 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