Skip to content

Commit

Permalink
[FLINK-9751] [filesystem] Add PersistentResumableWriter interface.
Browse files Browse the repository at this point in the history
  • Loading branch information
StephanEwen committed Jul 15, 2018
1 parent 66e0a27 commit e296094
Show file tree
Hide file tree
Showing 14 changed files with 1,425 additions and 6 deletions.
18 changes: 18 additions & 0 deletions flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
Original file line number Diff line number Diff line change
Expand Up @@ -493,6 +493,24 @@ public static URI getDefaultFsUri() {
*/
public abstract FSDataInputStream open(Path f) throws IOException;

/**
* Creates a new {@link RecoverableWriter}. A recoverable writer creates streams that can
* persist and recover their intermediate state.
* Persisting and recovering intermediate state is a core building block for writing to
* files that span multiple checkpoints.
*
* <p>The returned object can act as a shared factory to open and recover multiple streams.
*
* <p>This method is optional on file systems and various file system implementations may
* not support this method, throwing an {@code UnsupportedOperationException}.
*
* @return A RecoverableWriter for this file system.
* @throws IOException Thrown, if the recoverable writer cannot be instantiated.
*/
public RecoverableWriter createRecoverableWriter() throws IOException {
throw new UnsupportedOperationException("This file system does not support recoverable writers.");
}

/**
* Return the number of bytes that large input files should be optimally be split into to minimize I/O time.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* 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.core.fs;

import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable;
import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable;

import java.io.IOException;

/**
* An output stream to a file system that can be recovered at well defined points.
* The stream initially writes to hidden files or temp files and only creates the
* target file once it is closed and "committed".
*/
public abstract class RecoverableFsDataOutputStream extends FSDataOutputStream {

/**
* Ensures all data so far is persistent (similar to {@link #sync()}) and returns
* a handle to recover the stream at the current position.
*/
public abstract ResumeRecoverable persist() throws IOException;

/**
* Closes the stream, ensuring persistence of all data (similar to {@link #sync()}).
* This returns a Committer that can be used to publish (make visible) the file
* that the stream was writing to.
*/
public abstract Committer closeForCommit() throws IOException;

/**
* Closes the stream releasing all local resources, but not finalizing the
* file that the stream writes to.
*
* <p>This method should be understood as "close to dispose on failure".
*/
@Override
public abstract void close() throws IOException;

// ------------------------------------------------------------------------

/**
* A committer can publish the file of a stream that was closed.
* The Committer can be recovered via a {@link CommitRecoverable}.
*/
public interface Committer {

/**
* Commits the file, making it visible. The file will contain the exact data
* as when the committer was created.
*
* @throws IOException Thrown if committing fails.
*/
void commit() throws IOException;

/**
* Commits the file, making it visible. The file will contain the exact data
* as when the committer was created.
*
* <p>This method tolerates situations where the file was already committed and
* will not raise an exception in that case. This is important for idempotent
* commit retries as they need to happen after recovery.
*
* @throws IOException Thrown if committing fails.
*/
void commitAfterRecovery() throws IOException;

/**
* Gets a recoverable object to recover the committer. The recovered committer
* will commit the file with the exact same data as this committer would commit
* it.
*/
CommitRecoverable getRecoverable();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* 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.core.fs;

import org.apache.flink.core.io.SimpleVersionedSerializer;

import java.io.IOException;

/**
* The RecoverableWriter creates and recovers {@link RecoverableFsDataOutputStream}.
* It can be used to write data to a file system in a way that the writing can be
* resumed consistently after a failure and recovery without loss of data or possible
* duplication of bytes.
*
* <p>The streams do not make the files they write to immediately visible, but instead write
* to temp files or other temporary storage. To publish the data atomically in the
* end, the stream offers the {@link RecoverableFsDataOutputStream#closeForCommit()} method
* to create a committer that publishes the result.
*
* <p>These writers are useful in the context of checkpointing. The example below illustrates
* how to use them:
*
* <pre>{@code
* // --------- initial run --------
* RecoverableWriter writer = fileSystem.createRecoverableWriter();
* RecoverableFsDataOutputStream out = writer.open(path);
* out.write(...);
*
* // persist intermediate state
* ResumeRecoverable intermediateState = out.persist();
* storeInCheckpoint(intermediateState);
*
* // --------- recovery --------
* ResumeRecoverable lastCheckpointState = ...; // get state from checkpoint
* RecoverableWriter writer = fileSystem.createRecoverableWriter();
* RecoverableFsDataOutputStream out = writer.recover(lastCheckpointState);
*
* out.write(...); // append more data
*
* out.closeForCommit().commit(); // close stream and publish all the data
*
* // --------- recovery without appending --------
* ResumeRecoverable lastCheckpointState = ...; // get state from checkpoint
* RecoverableWriter writer = fileSystem.createRecoverableWriter();
* Committer committer = writer.recoverForCommit(lastCheckpointState);
* committer.commit(); // publish the state as of the last checkpoint
* }</pre>
*
* <h3>Recovery</h3>
*
* <p>Recovery relies on data persistence in the target file system or object store. While the
* code itself works with the specific primitives that the target storage offers, recovery will
* fail if the data written so far was deleted by an external factor.
* For example, some implementations stage data in temp files or object parts. If these
* were deleted by someone or by an automated cleanup policy, then resuming
* may fail. This is not surprising and should be expected, but we want to explicitly point
* this out here.
*
* <p>Specific care is needed for systems like S3, where the implementation uses Multipart Uploads
* to incrementally upload and persist parts of the result. Timeouts for Multipart Uploads
* and life time of Parts in unfinished Multipart Uploads need to be set in the bucket policy
* high enough to accommodate the recovery. These values are typically in the days, so regular
* recovery is typically not a problem. What can become an issue is situations where a Flink
* application is hard killed (all processes or containers removed) and then one tries to
* manually recover the application from an externalized checkpoint some days later. In that
* case, systems like S3 may have removed uncommitted parts and recovery will not succeed.
*
* <h3>Implementer's Note</h3>
*
* <p>From the perspective of the implementer, it would be desirable to make this class
* generic with respect to the concrete types of 'CommitRecoverable' and 'ResumeRecoverable'.
* However, we found that this makes the code more clumsy to use and we hence dropped the
* generics at the cost of doing some explicit casts in the implementation that would
* otherwise have been implicitly generated by the generics compiler.
*/
public interface RecoverableWriter {

/**
* Opens a new recoverable stream to write to the given path.
* Whether existing files will be overwritten is implementation specific and should
* not be relied upon.
*
* @param path The path of the file/object to write to.
* @return A new RecoverableFsDataOutputStream writing a new file/object.
*
* @throws IOException Thrown if the stream could not be opened/initialized.
*/
RecoverableFsDataOutputStream open(Path path) throws IOException;

/**
* Resumes a recoverable stream consistently at the point indicated by the given ResumeRecoverable.
* Future writes to the stream will continue / append the file as of that point.
*
* <p>This method is optional and whether it is supported is indicated through the
* {@link #supportsResume()} method.
*
* @param resumable The opaque handle with the recovery information.
* @return A recoverable stream writing to the file/object as it was at the point when the
* ResumeRecoverable was created.
*
* @throws IOException Thrown, if resuming fails.
* @throws UnsupportedOperationException Thrown if this optional method is not supported.
*/
RecoverableFsDataOutputStream recover(ResumeRecoverable resumable) throws IOException;

/**
* Recovers a recoverable stream consistently at the point indicated by the given CommitRecoverable
* for finalizing and committing. This will publish the target file with exactly the data
* that was written up to the point then the CommitRecoverable was created.
*
* @param resumable The opaque handle with the recovery information.
* @return A committer that publishes the target file.
*
* @throws IOException Thrown, if recovery fails.
*/
RecoverableFsDataOutputStream.Committer recoverForCommit(CommitRecoverable resumable) throws IOException;

/**
* The serializer for the CommitRecoverable types created in this writer.
* This serializer should be used to store the CommitRecoverable in checkpoint
* state or other forms of persistent state.
*/
SimpleVersionedSerializer<CommitRecoverable> getCommitRecoverableSerializer();

/**
* The serializer for the ResumeRecoverable types created in this writer.
* This serializer should be used to store the ResumeRecoverable in checkpoint
* state or other forms of persistent state.
*/
SimpleVersionedSerializer<ResumeRecoverable> getResumeRecoverableSerializer();

/**
* Checks whether the writer and its streams support resuming (appending to) files after
* recovery (via the {@link #recover(ResumeRecoverable)} method).
*
* <p>If true, then this writer supports the {@link #recover(ResumeRecoverable)} method.
* If false, then that method may not be supported and streams can only be recovered via
* {@link #recoverForCommit(CommitRecoverable)}.
*/
boolean supportsResume();

// ------------------------------------------------------------------------

/**
* A handle to an in-progress stream with a defined and persistent amount of data.
* The handle can be used to recover the stream as of exactly that point and
* publish the result file.
*/
interface CommitRecoverable {}

/**
* A handle to an in-progress stream with a defined and persistent amount of data.
* The handle can be used to recover the stream exactly as of that point and either
* publish the result file or keep appending data to the stream.
*/
interface ResumeRecoverable extends CommitRecoverable {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -142,11 +142,9 @@ public FSDataInputStream open(final Path f) throws IOException {
return new LocalDataInputStream(file);
}

private File pathToFile(Path path) {
if (!path.isAbsolute()) {
path = new Path(getWorkingDirectory(), path);
}
return new File(path.toUri().getPath());
@Override
public LocalRecoverableWriter createRecoverableWriter() throws IOException {
return new LocalRecoverableWriter(this);
}

@Override
Expand Down Expand Up @@ -308,6 +306,20 @@ public FileSystemKind getKind() {

// ------------------------------------------------------------------------

/**
* Converts the given Path to a File for this file system.
*
* <p>If the path is not absolute, it is interpreted relative to this FileSystem's working directory.
*/
public File pathToFile(Path path) {
if (!path.isAbsolute()) {
path = new Path(getWorkingDirectory(), path);
}
return new File(path.toUri().getPath());
}

// ------------------------------------------------------------------------

/**
* Gets the URI that represents the local file system.
* That URI is {@code "file:/"} on Windows platforms and {@code "file:///"} on other
Expand Down
Loading

0 comments on commit e296094

Please sign in to comment.