[
https://issues.apache.org/jira/browse/FLINK-5823?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15999800#comment-15999800
]
ASF GitHub Bot commented on FLINK-5823:
---------------------------------------
Github user StephanEwen commented on a diff in the pull request:
https://github.com/apache/flink/pull/3522#discussion_r115140655
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointMetadataStreamFactory.java
---
@@ -0,0 +1,101 @@
+/*
+ * 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.core.fs.FSDataOutputStream;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link CheckpointMetadataStreamFactory} creates streams to write
checkpoint <i>metadata</i>,
+ * similar as the {@link CheckpointStreamFactory} creates streams to write
checkpoint <i>data</i>.
+ */
+public interface CheckpointMetadataStreamFactory {
+
+ /**
+ * Creates a stream to write the metadata for the particular checkpoint
that this factory
+ * belongs to.
+ *
+ * @return The metadata output stream for this factory's checkpoint.
+ * @throws IOException Thrown, if the stream could not be opened.
+ */
+ CheckpointMetadataOutputStream createCheckpointStateOutputStream()
throws IOException;
+
+ /**
+ * Gets the location (as a string pointer) where the metadata factory
+ * The interpretation of the pointer is up to the implementation of the
state backend.
+ *
+ * <p>In case of high-availability setups, the target location pointer
is stored
+ * in the "ground-truth" store for checkpoint recovery.
+ *
+ * @return The checkpoint location pointer.
+ */
+ String getTargetLocation();
+
+ //
------------------------------------------------------------------------
+
+ /**
+ * A dedicated output stream for persisting checkpoint metadata. Upon
completion, this returns
+ * the state handle to the persisted metadata, plus the external
pointer that can be used
+ * to restore from that checkpoint.
+ */
+ abstract class CheckpointMetadataOutputStream extends
FSDataOutputStream {
+
+ /**
+ * Closes the metadata stream and gets the external pointer to
the checkpoint and a
+ * handle that can create an input stream producing the data
written to this stream.
+ *
+ * @return The pointer and state handle with access to the
written metadata.
+ * @throws IOException Thrown, if the stream cannot be closed.
+ */
+ public abstract StreamHandleAndPointer
closeAndGetPointerHandle() throws IOException;
+ }
+
+ //
------------------------------------------------------------------------
+
+ /**
+ * A combination of a {@code StreamStateHandle} and an external pointer
(in the form of a String).
+ */
+ final class StreamHandleAndPointer {
--- End diff --
The idea of the pointer is to have a state-backend independent string that
indicates a checkpoint location. I picked a string, because the location is
ultimately passed as a string, either from the user command `flink run -s
<savepoint path>` or from ZooKeeper' "last completed checkpoint" entry.
Introducing the `StreamHandleAndPointer` here spares the code from having
to reason about state backend specific file handles. We can change that at some
point, but for the breadth of changes already in this PR, this helped reduce
the scope a bit.
> 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)