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

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

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

    https://github.com/apache/flink/pull/2194#discussion_r69753840
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
 ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.checkpoint.savepoint;
    +
    +/**
    + * Savepoint store used to persist {@link Savepoint} instances.
    + *
    + * <p>The main implementation is the {@link FsSavepointStore}. We also 
have the
    + * {@link HeapSavepointStore} for historical reasons (introduced in Flink 
1.0).
    + */
    +public interface SavepointStore {
    +
    +   /**
    +    * Stores the savepoint.
    +    *
    +    * @param savepoint Savepoint to be stored
    +    * @param <T>       Savepoint type
    +    * @return Path of stored savepoint
    +    * @throws Exception Failures during store are forwarded
    +    */
    +   <T extends Savepoint> String storeSavepoint(T savepoint) throws 
Exception;
    +
    +   /**
    +    * Loads the savepoint at the specified path.
    +    *
    +    * @param path Path of savepoint to load
    +    * @return The loaded savepoint
    +    * @throws Exception Failures during load are forwared
    +    */
    +   Savepoint loadSavepoint(String path) throws Exception;
    +
    +   /**
    +    * Disposes the savepoint at the specified path.
    +    *
    +    * <p>The class loader is needed, because savepoints can currently 
point to
    +    * arbitrary snapshot {@link org.apache.flink.runtime.state.StateHandle}
    +    * instances, which need the user code class loader for deserialization.
    +    *
    +    * @param path        Path of savepoint to dispose
    +    * @param classLoader Class loader for disposal
    +    * @throws Exception Failures during diposal are forwarded
    +    */
    +   void disposeSavepoint(String path, ClassLoader classLoader) throws 
Exception;
    +
    +   /**
    +    * Shut downs the savepoint store.
    +    *
    +    * <p>Only necessary for implementations where the savepoint life-cycle 
is
    +    * bound to the cluster life-cycle.
    +    *
    +    * @throws Exception Failures during shut down are forwarded
    +    */
    +   void shutDown() throws Exception;
    --- End diff --
    
    I think *shutdown* is a verb in fact.


> Add version header to savepoints
> --------------------------------
>
>                 Key: FLINK-4067
>                 URL: https://issues.apache.org/jira/browse/FLINK-4067
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.0.3
>            Reporter: Ufuk Celebi
>            Assignee: Ufuk Celebi
>             Fix For: 1.1.0
>
>
> Adding a header with version information to savepoints ensures that we can 
> migrate savepoints between Flink versions in the future (for example when 
> changing internal serialization formats between versions).
> After talking with Till, we propose to add the following meta data:
> - Magic number (int): identify data as savepoint
> - Version (int): savepoint version (independent of Flink version)
> - Data Offset (int): specifies at which point the actual savepoint data 
> starts. With this, we can allow future Flink versions to add fields to the 
> header without breaking stuff, e.g. Flink 1.1 could read savepoints of Flink 
> 2.0.
> For Flink 1.0 savepoint support, we have to try reading the savepoints 
> without a header before failing if we don't find the magic number.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to