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

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_r165309957
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java
 ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
    +import org.apache.flink.runtime.state.KeyedStateHandle;
    +import org.apache.flink.runtime.state.OperatorStateHandle;
    +import org.apache.flink.runtime.state.SnapshotResult;
    +import org.apache.flink.runtime.state.StateObject;
    +import org.apache.flink.util.FutureUtil;
    +
    +import java.util.concurrent.ExecutionException;
    +
    +/**
    + * TODO write comment.
    + */
    +public class OperatorSnapshotFinalizer {
    +
    +   private final OperatorSubtaskState jobManagerOwnedState;
    +   private final OperatorSubtaskState taskLocalState;
    +
    +   public OperatorSnapshotFinalizer(
    +           OperatorSnapshotFutures snapshotFutures) throws 
ExecutionException, InterruptedException {
    +
    +           SnapshotResult<KeyedStateHandle> keyedManaged =
    +                   
FutureUtil.runIfNotDoneAndGet(snapshotFutures.getKeyedStateManagedFuture());
    +
    +           SnapshotResult<KeyedStateHandle> keyedRaw =
    +                   
FutureUtil.runIfNotDoneAndGet(snapshotFutures.getKeyedStateRawFuture());
    +
    +           SnapshotResult<OperatorStateHandle> operatorManaged =
    +                   
FutureUtil.runIfNotDoneAndGet(snapshotFutures.getOperatorStateManagedFuture());
    +
    +           SnapshotResult<OperatorStateHandle> operatorRaw =
    +                   
FutureUtil.runIfNotDoneAndGet(snapshotFutures.getOperatorStateRawFuture());
    +
    +           jobManagerOwnedState = new OperatorSubtaskState(
    +                   extractJobManagerOwned(operatorManaged),
    +                   extractJobManagerOwned(operatorRaw),
    +                   extractJobManagerOwned(keyedManaged),
    +                   extractJobManagerOwned(keyedRaw)
    +           );
    +
    +           taskLocalState = new OperatorSubtaskState(
    +                   extractTaskLocal(operatorManaged),
    +                   extractTaskLocal(operatorRaw),
    +                   extractTaskLocal(keyedManaged),
    +                   extractTaskLocal(keyedRaw)
    +           );
    +   }
    +
    +   public OperatorSubtaskState getTaskLocalState() {
    +           return taskLocalState;
    +   }
    +
    +   public OperatorSubtaskState getJobManagerOwnedState() {
    +           return jobManagerOwnedState;
    +   }
    +
    +   private <T extends StateObject> T 
extractJobManagerOwned(SnapshotResult<T> snapshotResult) {
    +           return snapshotResult != null ? 
snapshotResult.getJobManagerOwnedSnapshot() : null;
    --- End diff --
    
    We should add `@Nullable` annotation to make sure that this method can 
return a `null` value.


> 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