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

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

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

    https://github.com/apache/flink/pull/3522#discussion_r108477718
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java
 ---
    @@ -0,0 +1,556 @@
    +/*
    + * 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.filesystem;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +import org.apache.flink.api.common.JobID;
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.configuration.ConfigOption;
    +import org.apache.flink.configuration.ConfigOptions;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.configuration.IllegalConfigurationException;
    +import org.apache.flink.core.fs.FileStatus;
    +import org.apache.flink.core.fs.FileSystem;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.CheckpointMetadataStreamFactory;
    +import org.apache.flink.runtime.state.StateBackendGlobalHooks;
    +import org.apache.flink.runtime.state.StreamStateHandle;
    +import org.apache.flink.util.FileUtils;
    +import org.apache.flink.util.FlinkException;
    +import org.apache.flink.util.FlinkRuntimeException;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * A base class for all state backends that store their metadata (and 
data) in files.
    + * Examples that inherit from this are the {@link FsStateBackend}, the
    + * {@link org.apache.flink.runtime.state.memory.MemoryStateBackend 
MemoryStateBackend}, or the
    + * {@code RocksDBStateBackend}.
    + * 
    + * <p>This class takes the base checkpoint- and savepoint directory paths, 
but also accepts null
    + * for both of then, in which case creating externalized checkpoint is not 
possible, and it is not
    + * possible to create a savepoint with a default path. Null is accepted to 
enable implementations
    + * that only optionally support default savepoints and externalized 
checkpoints.
    + * 
    + * <h1>Checkpoint Layout</h1>
    + * 
    + * The state backend is configured with a base directory and persists the 
checkpoint data of specific
    + * checkpoints in specific subdirectories. For example, if the base 
directory was set to 
    + * {@code hdfs://namenode:port/flink-checkpoints/}, the state backend will 
create a subdirectory with
    + * the job's ID that will contain the actual checkpoints:
    + * ({@code 
hdfs://namenode:port/flink-checkpoints/1b080b6e710aabbef8993ab18c6de98b})
    + *
    + * <p>Each checkpoint individually will store all its files in a 
subdirectory that includes the
    + * checkpoint number, such as {@code 
hdfs://namenode:port/flink-checkpoints/1b080b6e710aabbef8993ab18c6de98b/chk-17/}.
    + * 
    + * <h1>Savepoint Layout</h1>
    + * 
    + * A savepoint that is set to be stored in path {@code 
hdfs://namenode:port/flink-savepoints/}, will create
    + * a subdirectory {@code savepoint-jobId(0, 6)-randomDigits} in which it 
stores all savepoint data.
    + * The random digits are added as "entropy" to avoid directory collisions.
    + */
    +@PublicEvolving
    +public abstract class AbstractFileStateBackend extends 
AbstractStateBackend implements StateBackendGlobalHooks {
    +
    +   private static final long serialVersionUID = 1L;
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(AbstractFileStateBackend.class);
    +
    +   // 
------------------------------------------------------------------------
    +   //  Configuration
    +   // 
------------------------------------------------------------------------
    +
    +   /**
    +    * The configuration parameter for the checkpoint path/directory.
    +    */
    +   public static final ConfigOption<String> CHECKPOINT_PATH = ConfigOptions
    +                   .key(ConfigConstants.CHECKPOINTS_DIRECTORY_KEY)
    +                   .noDefaultValue()
    +                   .withDeprecatedKeys("state.backend.fs.checkpointdir");
    +
    +   /**
    +    * The configuration parameter for the savepoint path/directory.
    +    */
    +   public static final ConfigOption<String> SAVEPOINT_PATH = ConfigOptions
    +                   .key(ConfigConstants.SAVEPOINT_DIRECTORY_KEY)
    +                   .noDefaultValue()
    +                   .withDeprecatedKeys("state.backend.fs.savepointdir");
    --- End diff --
    
    Shall we move these `ConfigOptions` to a dedicated `StateOptions` class 
which is put under `org.apache.flink.configuration`? I think most of the 
options live there, currently.


> Store Checkpoint Root Metadata in StateBackend (not in HA custom store)
> -----------------------------------------------------------------------
>
>                 Key: FLINK-5823
>                 URL: https://issues.apache.org/jira/browse/FLINK-5823
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>             Fix For: 1.3.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to