[ https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16373135#comment-16373135 ]
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_r170037177 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryConfig.java --- @@ -0,0 +1,120 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.io.ParseException; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; + +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +/** + * This class encapsulates the completed configuration for local recovery, i.e. the root + * directories into which all file-based snapshots can be written and the general mode for the local recover feature. + */ +public class LocalRecoveryConfig { + + /** + * Enum over modes of local recovery: + * <p><ul> + * <li>DISABLED: disables local recovery. + * <li>ENABLE_FILE_BASED: enables local recovery in a variant that is based on local files. + * </ul> + */ + public enum LocalRecoveryMode { + DISABLED, + ENABLE_FILE_BASED; + + /** + * Attempts to parses the given string to a {@link LocalRecoveryMode}. + * + * @param input string to parse + * @return the {@link LocalRecoveryMode} that is represented by the input string. + * @throws ParseException if the string cannot be parsed to a {@link LocalRecoveryMode}. + */ + @Nonnull + public static LocalRecoveryMode fromString(String input) throws ParseException { + if (input != null) { + String trimInput = input.trim(); + if (DISABLED.toString().equalsIgnoreCase(trimInput)) { + return DISABLED; + } else if (ENABLE_FILE_BASED.toString().equalsIgnoreCase(trimInput)) { + return ENABLE_FILE_BASED; + } + } + throw new ParseException("Cannot parse input to LocalRecoveryMode: " + input); + } + + /** + * Extracts the {@link LocalRecoveryMode} from the given configuration. Defaults to LocalRecoveryMode.DISABLED + * if no configuration value is specified or parsing the value resulted in an exception. + * + * @param configuration the configuration that specifies the value for the local recovery mode. + * @return the local recovery mode as found in the config, or LocalRecoveryMode.DISABLED if no mode was + * configured or the specified mode could not be parsed. + */ + @Nonnull + public static LocalRecoveryMode fromConfig(@Nonnull Configuration configuration) { + String localRecoveryConfString = configuration.getString(CheckpointingOptions.LOCAL_RECOVERY); + try { + return LocalRecoveryConfig.LocalRecoveryMode.fromString(localRecoveryConfString); + } catch (ParseException ex) { + LoggerFactory.getLogger(LocalRecoveryConfig.class).warn( + "Exception while parsing configuration of local recovery mode. Local recovery will be disabled.", + ex); + return LocalRecoveryConfig.LocalRecoveryMode.DISABLED; + } + } + } + + /** The local recovery mode. */ + @Nonnull + private final LocalRecoveryMode localRecoveryMode; --- End diff -- This is more of a general question than a review comment: How will it look in the future if we want to support a non file based local recovery? E.g. writing some things to a local DB, for example. Then we would not need the `LocalRecoveryDirectoryProvider` and we would have to adapt how we create the duplicating streams. So I'm wondering whether we should not have something like a `LocalStateStreamFactory` instead. For file based recovery it could wrap the `LocalRecoveryDirectoryProvider` to select the next file to write into. What are the plans for that in the future? > 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)