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

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

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

    https://github.com/apache/flink/pull/5239#discussion_r162585199
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
 ---
    @@ -132,32 +141,49 @@ public AbstractStreamOperatorTestHarness(
                                1024,
                                new Configuration(),
                                new ExecutionConfig(),
    +                           new TestTaskStateManager(),
                                maxParallelism,
                                parallelism,
                                subtaskIndex));
        }
     
        public AbstractStreamOperatorTestHarness(
                        StreamOperator<OUT> operator,
    -                   final Environment environment) throws Exception {
    +                   Environment env) throws Exception {
                this.operator = operator;
                this.outputList = new ConcurrentLinkedQueue<>();
                this.sideOutputLists = new HashMap<>();
     
    -           Configuration underlyingConfig = 
environment.getTaskConfiguration();
    +           Configuration underlyingConfig = env.getTaskConfiguration();
                this.config = new StreamConfig(underlyingConfig);
                this.config.setCheckpointingEnabled(true);
                this.config.setOperatorID(new OperatorID());
    -           this.executionConfig = environment.getExecutionConfig();
    +           this.executionConfig = env.getExecutionConfig();
                this.closableRegistry = new CloseableRegistry();
                this.checkpointLock = new Object();
     
    -           this.environment = Preconditions.checkNotNull(environment);
    +           Preconditions.checkNotNull(env);
    +
    +           MockUtil mockUtil = new MockUtil();
    +
    +           if (!mockUtil.isMock(env) && !mockUtil.isSpy(env)) {
    +                   env = spy(env);
    +           }
    +
    +           this.environment = env;
    +
    +           this.taskStateManager = new TestTaskStateManager(
    +                   env.getJobID(),
    +                   env.getExecutionId());
    +
    +           
when(this.environment.getTaskStateManager()).thenReturn(this.taskStateManager);
    --- End diff --
    
    👍 I made all tests use the `MockEnvironment`.


> 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