Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/2608#discussion_r82977266 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java --- @@ -18,23 +18,105 @@ package org.apache.flink.runtime.checkpoint.savepoint; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + /** - * Savepoint store used to persist {@link Savepoint} instances. + * A file system based savepoint store. * - * <p>The main implementation is the {@link FsSavepointStore}. We also have the - * {@link HeapSavepointStore} for historical reasons (introduced in Flink 1.0). + * <p>Stored savepoints have the following format: + * <pre> + * MagicNumber SavepointVersion Savepoint + * - MagicNumber => int + * - SavepointVersion => int (returned by Savepoint#getVersion()) + * - Savepoint => bytes (serialized via version-specific SavepointSerializer) + * </pre> */ -public interface SavepointStore { +public class SavepointStore { + + private static final Logger LOG = LoggerFactory.getLogger(SavepointStore.class); + + /** Magic number for sanity checks against stored savepoints. */ + private static final int MAGIC_NUMBER = 0x4960672d; + + /** Prefix for savepoint files. */ + private static final String prefix = "savepoint-"; /** * Stores the savepoint. * + * @param targetDirectory Target directory to store savepoint in * @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; + public static <T extends Savepoint> String storeSavepoint( + String targetDirectory, + T savepoint) throws IOException { + + checkNotNull(targetDirectory, "Target directory"); + checkNotNull(savepoint, "Savepoint"); + + Exception latestException = null; + Path path = null; + FSDataOutputStream fdos = null; + + FileSystem fs = null; + + // Try to create a FS output stream + for (int attempt = 0; attempt < 10; attempt++) { + path = new Path(targetDirectory, FileUtils.getRandomFilename(prefix)); + + if (fs == null) { + fs = FileSystem.get(path.toUri()); + } + + try { + fdos = fs.create(path, false); + break; + } catch (Exception e) { + latestException = e; + } + } + + if (fdos == null) { + throw new IOException("Failed to create file output stream at " + path, latestException); + } + + boolean success = false; + try (DataOutputStream dos = new DataOutputStream(fdos)) { + // Write header + dos.writeInt(MAGIC_NUMBER); + dos.writeInt(savepoint.getVersion()); + + // Write savepoint + SavepointSerializer<T> serializer = SavepointSerializers.getSerializer(savepoint); + serializer.serialize(savepoint, dos); + success = true; + } finally { + if (!success && fs.exists(path)) { + if (!fs.delete(path, true)) { + LOG.warn("Failed to delete file " + path + " after failed write."); --- End diff -- Replaced here and below
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---