|
@@ -18,18 +18,18 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.s3a.commit;
|
|
|
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.Date;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
-import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.stream.Collectors;
|
|
|
|
|
|
import com.amazonaws.services.s3.model.MultipartUpload;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import com.google.common.collect.Lists;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -49,7 +49,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
|
import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.util.DurationInfo;
|
|
|
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS;
|
|
|
import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
|
|
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
|
|
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
|
@@ -66,11 +68,28 @@ import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
|
|
|
* to handle the creation of a committer when the destination is unknown.
|
|
|
*
|
|
|
* Requiring an output directory simplifies coding and testing.
|
|
|
+ *
|
|
|
+ * The original implementation loaded all .pendingset files
|
|
|
+ * before attempting any commit/abort operations.
|
|
|
+ * While straightforward and guaranteeing that no changes were made to the
|
|
|
+ * destination until all files had successfully been loaded -it didn't scale;
|
|
|
+ * the list grew until it exceeded heap size.
|
|
|
+ *
|
|
|
+ * The second iteration builds up an {@link ActiveCommit} class with the
|
|
|
+ * list of .pendingset files to load and then commit; that can be done
|
|
|
+ * incrementally and in parallel.
|
|
|
+ * As a side effect of this change, unless/until changed,
|
|
|
+ * the commit/abort/revert of all files uploaded by a single task will be
|
|
|
+ * serialized. This may slow down these operations if there are many files
|
|
|
+ * created by a few tasks, <i>and</i> the HTTP connection pool in the S3A
|
|
|
+ * committer was large enough for more all the parallel POST requests.
|
|
|
*/
|
|
|
public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(AbstractS3ACommitter.class);
|
|
|
|
|
|
+ public static final String THREAD_PREFIX = "s3a-committer-pool-";
|
|
|
+
|
|
|
/**
|
|
|
* Thread pool for task execution.
|
|
|
*/
|
|
@@ -349,16 +368,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
* @throws IOException IO failure
|
|
|
*/
|
|
|
protected void maybeCreateSuccessMarkerFromCommits(JobContext context,
|
|
|
- List<SinglePendingCommit> pending) throws IOException {
|
|
|
+ ActiveCommit pending) throws IOException {
|
|
|
List<String> filenames = new ArrayList<>(pending.size());
|
|
|
- for (SinglePendingCommit commit : pending) {
|
|
|
- String key = commit.getDestinationKey();
|
|
|
- if (!key.startsWith("/")) {
|
|
|
- // fix up so that FS.makeQualified() sets up the path OK
|
|
|
- key = "/" + key;
|
|
|
- }
|
|
|
- filenames.add(key);
|
|
|
- }
|
|
|
+ // The list of committed objects in pending is size limited in
|
|
|
+ // ActiveCommit.uploadCommitted.
|
|
|
+ filenames.addAll(pending.committedObjects);
|
|
|
maybeCreateSuccessMarker(context, filenames);
|
|
|
}
|
|
|
|
|
@@ -390,22 +404,25 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Base job setup deletes the success marker.
|
|
|
- * TODO: Do we need this?
|
|
|
+ * Base job setup (optionally) deletes the success marker and
|
|
|
+ * always creates the destination directory.
|
|
|
+ * When objects are committed that dest dir marker will inevitably
|
|
|
+ * be deleted; creating it now ensures there is something at the end
|
|
|
+ * while the job is in progress -and if nothing is created, that
|
|
|
+ * it is still there.
|
|
|
* @param context context
|
|
|
* @throws IOException IO failure
|
|
|
*/
|
|
|
-/*
|
|
|
|
|
|
@Override
|
|
|
public void setupJob(JobContext context) throws IOException {
|
|
|
- if (createJobMarker) {
|
|
|
- try (DurationInfo d = new DurationInfo("Deleting _SUCCESS marker")) {
|
|
|
+ try (DurationInfo d = new DurationInfo(LOG, "preparing destination")) {
|
|
|
+ if (createJobMarker){
|
|
|
commitOperations.deleteSuccessMarker(getOutputPath());
|
|
|
}
|
|
|
+ getDestFS().mkdirs(getOutputPath());
|
|
|
}
|
|
|
}
|
|
|
-*/
|
|
|
|
|
|
@Override
|
|
|
public void setupTask(TaskAttemptContext context) throws IOException {
|
|
@@ -430,28 +447,152 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Commit a list of pending uploads.
|
|
|
+ * Commit all the pending uploads.
|
|
|
+ * Each file listed in the ActiveCommit instance is queued for processing
|
|
|
+ * in a separate thread; its contents are loaded and then (sequentially)
|
|
|
+ * committed.
|
|
|
+ * On a failure or abort of a single file's commit, all its uploads are
|
|
|
+ * aborted.
|
|
|
+ * The revert operation lists the files already committed and deletes them.
|
|
|
* @param context job context
|
|
|
- * @param pending list of pending uploads
|
|
|
+ * @param pending pending uploads
|
|
|
* @throws IOException on any failure
|
|
|
*/
|
|
|
- protected void commitPendingUploads(JobContext context,
|
|
|
- List<SinglePendingCommit> pending) throws IOException {
|
|
|
+ protected void commitPendingUploads(
|
|
|
+ final JobContext context,
|
|
|
+ final ActiveCommit pending) throws IOException {
|
|
|
if (pending.isEmpty()) {
|
|
|
LOG.warn("{}: No pending uploads to commit", getRole());
|
|
|
}
|
|
|
- LOG.debug("{}: committing the output of {} task(s)",
|
|
|
- getRole(), pending.size());
|
|
|
- try(CommitOperations.CommitContext commitContext
|
|
|
+ try (DurationInfo ignored = new DurationInfo(LOG,
|
|
|
+ "committing the output of %s task(s)", pending.size());
|
|
|
+ CommitOperations.CommitContext commitContext
|
|
|
= initiateCommitOperation()) {
|
|
|
- Tasks.foreach(pending)
|
|
|
+
|
|
|
+ Tasks.foreach(pending.getSourceFiles())
|
|
|
.stopOnFailure()
|
|
|
+ .suppressExceptions(false)
|
|
|
.executeWith(buildThreadPool(context))
|
|
|
+ .abortWith(path ->
|
|
|
+ loadAndAbort(commitContext, pending, path, true, false))
|
|
|
+ .revertWith(path ->
|
|
|
+ loadAndRevert(commitContext, pending, path))
|
|
|
+ .run(path ->
|
|
|
+ loadAndCommit(commitContext, pending, path));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Run a precommit check that all files are loadable.
|
|
|
+ * This check avoids the situation where the inability to read
|
|
|
+ * a file only surfaces partway through the job commit, so
|
|
|
+ * results in the destination being tainted.
|
|
|
+ * @param context job context
|
|
|
+ * @param pending the pending operations
|
|
|
+ * @throws IOException any failure
|
|
|
+ */
|
|
|
+ protected void precommitCheckPendingFiles(
|
|
|
+ final JobContext context,
|
|
|
+ final ActiveCommit pending) throws IOException {
|
|
|
+
|
|
|
+ FileSystem sourceFS = pending.getSourceFS();
|
|
|
+ try (DurationInfo ignored =
|
|
|
+ new DurationInfo(LOG, "Preflight Load of pending files")) {
|
|
|
+
|
|
|
+ Tasks.foreach(pending.getSourceFiles())
|
|
|
+ .stopOnFailure()
|
|
|
+ .suppressExceptions(false)
|
|
|
+ .executeWith(buildThreadPool(context))
|
|
|
+ .run(path -> PendingSet.load(sourceFS, path));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Load a pendingset file and commit all of its contents.
|
|
|
+ * @param commitContext context to commit through
|
|
|
+ * @param activeCommit commit state
|
|
|
+ * @param path path to load
|
|
|
+ * @throws IOException failure
|
|
|
+ */
|
|
|
+ private void loadAndCommit(
|
|
|
+ final CommitOperations.CommitContext commitContext,
|
|
|
+ final ActiveCommit activeCommit,
|
|
|
+ final Path path) throws IOException {
|
|
|
+
|
|
|
+ try (DurationInfo ignored =
|
|
|
+ new DurationInfo(LOG, false, "Committing %s", path)) {
|
|
|
+ PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), path);
|
|
|
+ Tasks.foreach(pendingSet.getCommits())
|
|
|
+ .stopOnFailure()
|
|
|
+ .suppressExceptions(false)
|
|
|
+ .executeWith(singleCommitThreadPool())
|
|
|
.onFailure((commit, exception) ->
|
|
|
commitContext.abortSingleCommit(commit))
|
|
|
.abortWith(commitContext::abortSingleCommit)
|
|
|
.revertWith(commitContext::revertCommit)
|
|
|
- .run(commitContext::commitOrFail);
|
|
|
+ .run(commit -> {
|
|
|
+ commitContext.commitOrFail(commit);
|
|
|
+ activeCommit.uploadCommitted(
|
|
|
+ commit.getDestinationKey(), commit.getLength());
|
|
|
+ });
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Load a pendingset file and revert all of its contents.
|
|
|
+ * @param commitContext context to commit through
|
|
|
+ * @param activeCommit commit state
|
|
|
+ * @param path path to load
|
|
|
+ * @throws IOException failure
|
|
|
+ */
|
|
|
+ private void loadAndRevert(
|
|
|
+ final CommitOperations.CommitContext commitContext,
|
|
|
+ final ActiveCommit activeCommit,
|
|
|
+ final Path path) throws IOException {
|
|
|
+
|
|
|
+ try (DurationInfo ignored =
|
|
|
+ new DurationInfo(LOG, false, "Committing %s", path)) {
|
|
|
+ PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), path);
|
|
|
+ Tasks.foreach(pendingSet.getCommits())
|
|
|
+ .suppressExceptions(true)
|
|
|
+ .run(commitContext::revertCommit);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Load a pendingset file and abort all of its contents.
|
|
|
+ * @param commitContext context to commit through
|
|
|
+ * @param activeCommit commit state
|
|
|
+ * @param path path to load
|
|
|
+ * @param deleteRemoteFiles should remote files be deleted?
|
|
|
+ * @throws IOException failure
|
|
|
+ */
|
|
|
+ private void loadAndAbort(
|
|
|
+ final CommitOperations.CommitContext commitContext,
|
|
|
+ final ActiveCommit activeCommit,
|
|
|
+ final Path path,
|
|
|
+ final boolean suppressExceptions,
|
|
|
+ final boolean deleteRemoteFiles) throws IOException {
|
|
|
+
|
|
|
+ try (DurationInfo ignored =
|
|
|
+ new DurationInfo(LOG, false, "Aborting %s", path)) {
|
|
|
+ PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(),
|
|
|
+ path);
|
|
|
+ FileSystem fs = getDestFS();
|
|
|
+ Tasks.foreach(pendingSet.getCommits())
|
|
|
+ .executeWith(singleCommitThreadPool())
|
|
|
+ .suppressExceptions(suppressExceptions)
|
|
|
+ .run(commit -> {
|
|
|
+ try {
|
|
|
+ commitContext.abortSingleCommit(commit);
|
|
|
+ } catch (FileNotFoundException e) {
|
|
|
+ // Commit ID was not known; file may exist.
|
|
|
+ // delete it if instructed to do so.
|
|
|
+ if (deleteRemoteFiles) {
|
|
|
+ fs.delete(commit.destinationPath(), false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -465,44 +606,15 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
return getCommitOperations().initiateCommitOperation(getOutputPath());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Try to read every pendingset file and build a list of them/
|
|
|
- * In the case of a failure to read the file, exceptions are held until all
|
|
|
- * reads have been attempted.
|
|
|
- * @param context job context
|
|
|
- * @param suppressExceptions whether to suppress exceptions.
|
|
|
- * @param fs job attempt fs
|
|
|
- * @param pendingCommitFiles list of files found in the listing scan
|
|
|
- * @return the list of commits
|
|
|
- * @throws IOException on a failure when suppressExceptions is false.
|
|
|
- */
|
|
|
- protected List<SinglePendingCommit> loadPendingsetFiles(
|
|
|
- JobContext context,
|
|
|
- boolean suppressExceptions,
|
|
|
- FileSystem fs,
|
|
|
- Iterable<? extends FileStatus> pendingCommitFiles) throws IOException {
|
|
|
-
|
|
|
- final List<SinglePendingCommit> pending = Collections.synchronizedList(
|
|
|
- Lists.newArrayList());
|
|
|
- Tasks.foreach(pendingCommitFiles)
|
|
|
- .suppressExceptions(suppressExceptions)
|
|
|
- .executeWith(buildThreadPool(context))
|
|
|
- .run(pendingCommitFile ->
|
|
|
- pending.addAll(
|
|
|
- PendingSet.load(fs, pendingCommitFile.getPath()).getCommits())
|
|
|
- );
|
|
|
- return pending;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Internal Job commit operation: where the S3 requests are made
|
|
|
* (potentially in parallel).
|
|
|
* @param context job context
|
|
|
- * @param pending pending request
|
|
|
+ * @param pending pending commits
|
|
|
* @throws IOException any failure
|
|
|
*/
|
|
|
protected void commitJobInternal(JobContext context,
|
|
|
- List<SinglePendingCommit> pending)
|
|
|
+ ActiveCommit pending)
|
|
|
throws IOException {
|
|
|
|
|
|
commitPendingUploads(context, pending);
|
|
@@ -523,6 +635,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
* This must clean up operations; it is called when a commit fails, as
|
|
|
* well as in an {@link #abortJob(JobContext, JobStatus.State)} call.
|
|
|
* The base implementation calls {@link #cleanup(JobContext, boolean)}
|
|
|
+ * so cleans up the filesystems and destroys the thread pool.
|
|
|
+ * Subclasses must always invoke this superclass method after their
|
|
|
+ * own operations.
|
|
|
* @param context job context
|
|
|
* @param suppressExceptions should exceptions be suppressed?
|
|
|
* @throws IOException any IO problem raised when suppressExceptions is false.
|
|
@@ -536,13 +651,15 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
/**
|
|
|
* Abort all pending uploads to the destination directory during
|
|
|
* job cleanup operations.
|
|
|
+ * Note: this instantiates the thread pool if required -so
|
|
|
+ * {@link #destroyThreadPool()} must be called after this.
|
|
|
* @param suppressExceptions should exceptions be suppressed
|
|
|
* @throws IOException IO problem
|
|
|
*/
|
|
|
protected void abortPendingUploadsInCleanup(
|
|
|
boolean suppressExceptions) throws IOException {
|
|
|
Path dest = getOutputPath();
|
|
|
- try (DurationInfo d =
|
|
|
+ try (DurationInfo ignored =
|
|
|
new DurationInfo(LOG, "Aborting all pending commits under %s",
|
|
|
dest);
|
|
|
CommitOperations.CommitContext commitContext
|
|
@@ -565,13 +682,18 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Subclass-specific pre commit actions.
|
|
|
+ * Subclass-specific pre-Job-commit actions.
|
|
|
+ * The staging committers all load the pending files to verify that
|
|
|
+ * they can be loaded.
|
|
|
+ * The Magic committer does not, because of the overhead of reading files
|
|
|
+ * from S3 makes it too expensive.
|
|
|
* @param context job context
|
|
|
* @param pending the pending operations
|
|
|
* @throws IOException any failure
|
|
|
*/
|
|
|
- protected void preCommitJob(JobContext context,
|
|
|
- List<SinglePendingCommit> pending) throws IOException {
|
|
|
+ @VisibleForTesting
|
|
|
+ public void preCommitJob(JobContext context,
|
|
|
+ ActiveCommit pending) throws IOException {
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -584,7 +706,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
* <p>
|
|
|
* Commit internal: do the final commit sequence.
|
|
|
* <p>
|
|
|
- * The final commit action is to build the {@code __SUCCESS} file entry.
|
|
|
+ * The final commit action is to build the {@code _SUCCESS} file entry.
|
|
|
* </p>
|
|
|
* @param context job context
|
|
|
* @throws IOException any failure
|
|
@@ -594,7 +716,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
String id = jobIdString(context);
|
|
|
try (DurationInfo d = new DurationInfo(LOG,
|
|
|
"%s: commitJob(%s)", getRole(), id)) {
|
|
|
- List<SinglePendingCommit> pending
|
|
|
+ ActiveCommit pending
|
|
|
= listPendingUploadsToCommit(context);
|
|
|
preCommitJob(context, pending);
|
|
|
commitJobInternal(context, pending);
|
|
@@ -629,12 +751,13 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
* @return a list of pending uploads.
|
|
|
* @throws IOException Any IO failure
|
|
|
*/
|
|
|
- protected abstract List<SinglePendingCommit> listPendingUploadsToCommit(
|
|
|
+ protected abstract ActiveCommit listPendingUploadsToCommit(
|
|
|
JobContext context)
|
|
|
throws IOException;
|
|
|
|
|
|
/**
|
|
|
- * Cleanup the job context, including aborting anything pending.
|
|
|
+ * Cleanup the job context, including aborting anything pending
|
|
|
+ * and destroying the thread pool.
|
|
|
* @param context job context
|
|
|
* @param suppressExceptions should exceptions be suppressed?
|
|
|
* @throws IOException any failure if exceptions were not suppressed.
|
|
@@ -645,6 +768,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
"Cleanup job %s", jobIdString(context))) {
|
|
|
abortPendingUploadsInCleanup(suppressExceptions);
|
|
|
} finally {
|
|
|
+ destroyThreadPool();
|
|
|
cleanupStagingDirs();
|
|
|
}
|
|
|
}
|
|
@@ -715,7 +839,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
|
|
|
/**
|
|
|
* Returns an {@link ExecutorService} for parallel tasks. The number of
|
|
|
- * threads in the thread-pool is set by s3.multipart.committer.num-threads.
|
|
|
+ * threads in the thread-pool is set by fs.s3a.committer.threads.
|
|
|
* If num-threads is 0, this will return null;
|
|
|
*
|
|
|
* @param context the JobContext for this commit
|
|
@@ -730,10 +854,10 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
DEFAULT_COMMITTER_THREADS);
|
|
|
LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads);
|
|
|
if (numThreads > 0) {
|
|
|
- threadPool = Executors.newFixedThreadPool(numThreads,
|
|
|
+ threadPool = HadoopExecutors.newFixedThreadPool(numThreads,
|
|
|
new ThreadFactoryBuilder()
|
|
|
.setDaemon(true)
|
|
|
- .setNameFormat("s3-committer-pool-%d")
|
|
|
+ .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d")
|
|
|
.build());
|
|
|
} else {
|
|
|
return null;
|
|
@@ -742,6 +866,40 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
return threadPool;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Destroy any thread pools; wait for that to finish,
|
|
|
+ * but don't overreact if it doesn't finish in time.
|
|
|
+ */
|
|
|
+ protected synchronized void destroyThreadPool() {
|
|
|
+ if (threadPool != null) {
|
|
|
+ LOG.debug("Destroying thread pool");
|
|
|
+ HadoopExecutors.shutdown(threadPool, LOG,
|
|
|
+ THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
|
|
|
+ threadPool = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the thread pool for executing the single file commit/revert
|
|
|
+ * within the commit of all uploads of a single task.
|
|
|
+ * This is currently null; it is here to allow the Tasks class to
|
|
|
+ * provide the logic for execute/revert.
|
|
|
+ * Why not use the existing thread pool? Too much fear of deadlocking,
|
|
|
+ * and tasks are being committed in parallel anyway.
|
|
|
+ * @return null. always.
|
|
|
+ */
|
|
|
+ protected final synchronized ExecutorService singleCommitThreadPool() {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Does this committer have a thread pool?
|
|
|
+ * @return true if a thread pool exists.
|
|
|
+ */
|
|
|
+ public synchronized boolean hasThreadPool() {
|
|
|
+ return threadPool != null;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Delete the task attempt path without raising any errors.
|
|
|
* @param context task context
|
|
@@ -755,6 +913,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
|
|
|
/**
|
|
|
* Abort all pending uploads in the list.
|
|
|
+ * This operation is used by the magic committer as part of its
|
|
|
+ * rollback after a failure during task commit.
|
|
|
* @param context job context
|
|
|
* @param pending pending uploads
|
|
|
* @param suppressExceptions should exceptions be suppressed
|
|
@@ -779,4 +939,172 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Abort all pending uploads in the list.
|
|
|
+ * @param context job context
|
|
|
+ * @param pending pending uploads
|
|
|
+ * @param suppressExceptions should exceptions be suppressed?
|
|
|
+ * @param deleteRemoteFiles should remote files be deleted?
|
|
|
+ * @throws IOException any exception raised
|
|
|
+ */
|
|
|
+ protected void abortPendingUploads(
|
|
|
+ final JobContext context,
|
|
|
+ final ActiveCommit pending,
|
|
|
+ final boolean suppressExceptions,
|
|
|
+ final boolean deleteRemoteFiles) throws IOException {
|
|
|
+
|
|
|
+ if (pending.isEmpty()) {
|
|
|
+ LOG.info("{}: no pending commits to abort", getRole());
|
|
|
+ } else {
|
|
|
+ try (DurationInfo d = new DurationInfo(LOG,
|
|
|
+ "Aborting %s uploads", pending.size());
|
|
|
+ CommitOperations.CommitContext commitContext
|
|
|
+ = initiateCommitOperation()) {
|
|
|
+ Tasks.foreach(pending.getSourceFiles())
|
|
|
+ .executeWith(buildThreadPool(context))
|
|
|
+ .suppressExceptions(suppressExceptions)
|
|
|
+ .run(path ->
|
|
|
+ loadAndAbort(commitContext,
|
|
|
+ pending,
|
|
|
+ path,
|
|
|
+ suppressExceptions,
|
|
|
+ deleteRemoteFiles));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * State of the active commit operation.
|
|
|
+ *
|
|
|
+ * It contains a list of all pendingset files to load as the source
|
|
|
+ * of outstanding commits to complete/abort,
|
|
|
+ * and tracks the files uploaded.
|
|
|
+ *
|
|
|
+ * To avoid running out of heap by loading all the source files
|
|
|
+ * simultaneously:
|
|
|
+ * <ol>
|
|
|
+ * <li>
|
|
|
+ * The list of files to load is passed round but
|
|
|
+ * the contents are only loaded on demand.
|
|
|
+ * </li>
|
|
|
+ * <li>
|
|
|
+ * The number of written files tracked for logging in
|
|
|
+ * the _SUCCESS file are limited to a small amount -enough
|
|
|
+ * for testing only.
|
|
|
+ * </li>
|
|
|
+ * </ol>
|
|
|
+ */
|
|
|
+ public static class ActiveCommit {
|
|
|
+
|
|
|
+ private static final AbstractS3ACommitter.ActiveCommit EMPTY
|
|
|
+ = new ActiveCommit(null, new ArrayList<>());
|
|
|
+
|
|
|
+ /** All pendingset files to iterate through. */
|
|
|
+ private final List<Path> sourceFiles;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Filesystem for the source files.
|
|
|
+ */
|
|
|
+ private final FileSystem sourceFS;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * List of committed objects; only built up until the commit limit is
|
|
|
+ * reached.
|
|
|
+ */
|
|
|
+ private final List<String> committedObjects = new ArrayList<>();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The total number of committed objects.
|
|
|
+ */
|
|
|
+ private int committedObjectCount;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Total number of bytes committed.
|
|
|
+ */
|
|
|
+ private long committedBytes;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Construct from a source FS and list of files.
|
|
|
+ * @param sourceFS filesystem containing the list of pending files
|
|
|
+ * @param sourceFiles .pendingset files to load and commit.
|
|
|
+ */
|
|
|
+ public ActiveCommit(
|
|
|
+ final FileSystem sourceFS,
|
|
|
+ final List<Path> sourceFiles) {
|
|
|
+ this.sourceFiles = sourceFiles;
|
|
|
+ this.sourceFS = sourceFS;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create an active commit of the given pending files.
|
|
|
+ * @param pendingFS source filesystem.
|
|
|
+ * @param statuses list of file status or subclass to use.
|
|
|
+ * @return the commit
|
|
|
+ */
|
|
|
+ public static ActiveCommit fromStatusList(
|
|
|
+ final FileSystem pendingFS,
|
|
|
+ final List<? extends FileStatus> statuses) {
|
|
|
+ return new ActiveCommit(pendingFS,
|
|
|
+ statuses.stream()
|
|
|
+ .map(FileStatus::getPath)
|
|
|
+ .collect(Collectors.toList()));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the empty entry.
|
|
|
+ * @return an active commit with no pending files.
|
|
|
+ */
|
|
|
+ public static ActiveCommit empty() {
|
|
|
+ return EMPTY;
|
|
|
+ }
|
|
|
+
|
|
|
+ public List<Path> getSourceFiles() {
|
|
|
+ return sourceFiles;
|
|
|
+ }
|
|
|
+
|
|
|
+ public FileSystem getSourceFS() {
|
|
|
+ return sourceFS;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Note that a file was committed.
|
|
|
+ * Increase the counter of files and total size.
|
|
|
+ * If there is room in the committedFiles list, the file
|
|
|
+ * will be added to the list and so end up in the _SUCCESS file.
|
|
|
+ * @param key key of the committed object.
|
|
|
+ * @param size size in bytes.
|
|
|
+ */
|
|
|
+ public synchronized void uploadCommitted(String key, long size) {
|
|
|
+ if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) {
|
|
|
+ committedObjects.add(
|
|
|
+ key.startsWith("/") ? key : ("/" + key));
|
|
|
+ }
|
|
|
+ committedObjectCount++;
|
|
|
+ committedBytes += size;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized List<String> getCommittedObjects() {
|
|
|
+ return committedObjects;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized int getCommittedFileCount() {
|
|
|
+ return committedObjectCount;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized long getCommittedBytes() {
|
|
|
+ return committedBytes;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int size() {
|
|
|
+ return sourceFiles.size();
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean isEmpty() {
|
|
|
+ return sourceFiles.isEmpty();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void add(Path path) {
|
|
|
+ sourceFiles.add(path);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|