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

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_r168176306
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java
 ---
    @@ -501,4 +529,53 @@ public String toString() {
                                "', asynchronous: " + asynchronousSnapshots +
                                ", fileStateThreshold: " + fileStateThreshold + 
")";
        }
    +
    +   /**
    +    * This enum represents the different modes for local recovery.
    +    */
    +   public enum LocalRecoveryMode {
    +           DISABLED, ENABLE_FILE_BASED, ENABLE_HEAP_BASED
    +   }
    +
    +   /**
    +    * This class encapsulates the configuration for local recovery of this 
backend.
    +    */
    +   public static final class LocalRecoveryConfig implements Serializable {
    +
    +           private static final long serialVersionUID = 1L;
    +           private static final LocalRecoveryConfig DISABLED_SINGLETON =
    +                   new LocalRecoveryConfig(LocalRecoveryMode.DISABLED, 
null);
    +
    +           private final LocalRecoveryMode localRecoveryMode;
    +           private final File localStateDirectory;
    +
    +           LocalRecoveryConfig(LocalRecoveryMode localRecoveryMode, File 
localStateDirectory) {
    +                   this.localRecoveryMode = 
Preconditions.checkNotNull(localRecoveryMode);
    +                   this.localStateDirectory = localStateDirectory;
    +                   if 
(LocalRecoveryMode.ENABLE_FILE_BASED.equals(localRecoveryMode) && 
localStateDirectory == null) {
    +                           throw new IllegalStateException("Local state 
directory must be specified if local recovery mode is " +
    +                                   LocalRecoveryMode.ENABLE_FILE_BASED);
    +                   }
    +           }
    +
    +           public LocalRecoveryMode getLocalRecoveryMode() {
    +                   return localRecoveryMode;
    +           }
    +
    +           public File getLocalStateDirectory() {
    +                   return localStateDirectory;
    --- End diff --
    
    All this is changed a bit later, bit in essence the backend simply does not 
ask for the directory if nothing file-based is used.


> 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