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

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

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

    https://github.com/apache/flink/pull/5239#discussion_r159866648
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java ---
    @@ -669,31 +676,14 @@ else if (current == ExecutionState.CANCELING) {
                        Environment env = new RuntimeEnvironment(
                                jobId, vertexId, executionId, executionConfig, 
taskInfo,
                                jobConfiguration, taskConfiguration, 
userCodeClassLoader,
    -                           memoryManager, ioManager, 
broadcastVariableManager,
    +                           memoryManager, ioManager, 
broadcastVariableManager, taskStateManager,
                                accumulatorRegistry, kvStateRegistry, 
inputSplitProvider,
                                distributedCacheEntries, writers, inputGates,
                                checkpointResponder, taskManagerConfig, 
metrics, this);
     
                        // let the task code create its readers and writers
                        invokable.setEnvironment(env);
     
    -                   // the very last thing before the actual execution 
starts running is to inject
    -                   // the state into the task. the state is non-empty if 
this is an execution
    -                   // of a task that failed but had backuped state from a 
checkpoint
    -
    -                   if (null != taskRestore && 
taskRestore.getTaskStateSnapshot() != null) {
    -                           if (invokable instanceof StatefulTask) {
    -                                   StatefulTask op = (StatefulTask) 
invokable;
    -                                   
op.setInitialState(taskRestore.getTaskStateSnapshot());
    --- End diff --
    
    grrr, this cryptic `op` name forced me to look into the source code to 
check whether this is is instance of `StatefullTask` or not :/ could you rename 
it to something that at least is not an abbreviation?


> 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
>             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
(v6.4.14#64029)

Reply via email to