flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4512) Add option for persistent checkpoints
Date Tue, 11 Oct 2016 14:17:21 GMT

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

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

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

    https://github.com/apache/flink/pull/2608#discussion_r82796870
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
---
    @@ -43,24 +125,62 @@
     	 * @return The loaded savepoint
     	 * @throws Exception Failures during load are forwared
     	 */
    -	Savepoint loadSavepoint(String path) throws Exception;
    +	public static Savepoint loadSavepoint(String path) throws IOException {
    +		Preconditions.checkNotNull(path, "Path");
     
    -	/**
    -	 * Disposes the savepoint at the specified path.
    -	 *
    -	 * @param path        Path of savepoint to dispose
    -	 * @throws Exception Failures during diposal are forwarded
    -	 */
    -	void disposeSavepoint(String path) throws Exception;
    +		try (DataInputStream dis = new DataInputViewStreamWrapper(createFsInputStream(new Path(path))))
{
    +			int magicNumber = dis.readInt();
    +
    +			if (magicNumber == MAGIC_NUMBER) {
    +				int version = dis.readInt();
    +
    +				SavepointSerializer<?> serializer = SavepointSerializers.getSerializer(version);
    +				return serializer.deserialize(dis);
    +			} else {
    +				throw new RuntimeException("Unexpected magic number. This is most likely " +
    +						"caused by trying to load a Flink 1.0 savepoint. You cannot load a " +
    +						"savepoint triggered by Flink 1.0 with this version of Flink. If it is " +
    +						"_not_ a Flink 1.0 savepoint, this error indicates that the specified " +
    +						"file is not a proper savepoint or the file has been corrupted.");
    +			}
    +		}
    +	}
     
     	/**
    -	 * Shut downs the savepoint store.
    +	 * Removes the savepoint meta data w/o loading and disposing it.
     	 *
    -	 * <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
    +	 * @param path Path of savepoint to remove
    +	 * @throws Exception Failures during disposal are forwarded
     	 */
    -	void shutdown() throws Exception;
    +	public static void removeSavepoint(String path) throws Exception {
    +		Preconditions.checkNotNull(path, "Path");
    +
    +		try {
    +			LOG.info("Removing savepoint: " + path);
    --- End diff --
    
    {}


> Add option for persistent checkpoints
> -------------------------------------
>
>                 Key: FLINK-4512
>                 URL: https://issues.apache.org/jira/browse/FLINK-4512
>             Project: Flink
>          Issue Type: Sub-task
>          Components: State Backends, Checkpointing
>            Reporter: Ufuk Celebi
>            Assignee: Ufuk Celebi
>
> Allow periodic checkpoints to be persisted by writing out their meta data. This is what
we currently do for savepoints, but in the future checkpoints and savepoints are likely to
diverge with respect to guarantees they give for updatability, etc.
> This means that the difference between persistent checkpoints and savepoints in the long
term will be that persistent checkpoints can only be restored with the same job settings (like
parallelism, etc.)
> Regular and persisted checkpoints should behave differently with respect to disposal
in *globally* terminal job states (FINISHED, CANCELLED, FAILED): regular checkpoints are cleaned
up in all of these cases whereas persistent checkpoints only on FINISHED. Maybe with the option
to customize behaviour on CANCELLED or FAILED.



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

Mime
View raw message