|
@@ -24,6 +24,7 @@ import java.util.ArrayList;
|
|
|
import java.util.Date;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Future;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
@@ -472,7 +473,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
Tasks.foreach(pending.getSourceFiles())
|
|
|
.stopOnFailure()
|
|
|
.suppressExceptions(false)
|
|
|
- .executeWith(buildThreadPool(context))
|
|
|
+ .executeWith(buildSubmitter(context))
|
|
|
.abortWith(path ->
|
|
|
loadAndAbort(commitContext, pending, path, true, false))
|
|
|
.revertWith(path ->
|
|
@@ -502,7 +503,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
Tasks.foreach(pending.getSourceFiles())
|
|
|
.stopOnFailure()
|
|
|
.suppressExceptions(false)
|
|
|
- .executeWith(buildThreadPool(context))
|
|
|
+ .executeWith(buildSubmitter(context))
|
|
|
.run(path -> PendingSet.load(sourceFS, path));
|
|
|
}
|
|
|
}
|
|
@@ -525,7 +526,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
Tasks.foreach(pendingSet.getCommits())
|
|
|
.stopOnFailure()
|
|
|
.suppressExceptions(false)
|
|
|
- .executeWith(singleCommitThreadPool())
|
|
|
+ .executeWith(singleThreadSubmitter())
|
|
|
.onFailure((commit, exception) ->
|
|
|
commitContext.abortSingleCommit(commit))
|
|
|
.abortWith(commitContext::abortSingleCommit)
|
|
@@ -580,7 +581,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
path);
|
|
|
FileSystem fs = getDestFS();
|
|
|
Tasks.foreach(pendingSet.getCommits())
|
|
|
- .executeWith(singleCommitThreadPool())
|
|
|
+ .executeWith(singleThreadSubmitter())
|
|
|
.suppressExceptions(suppressExceptions)
|
|
|
.run(commit -> {
|
|
|
try {
|
|
@@ -674,7 +675,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
return;
|
|
|
}
|
|
|
Tasks.foreach(pending)
|
|
|
- .executeWith(buildThreadPool(getJobContext()))
|
|
|
+ .executeWith(buildSubmitter(getJobContext()))
|
|
|
.suppressExceptions(suppressExceptions)
|
|
|
.run(u -> commitContext.abortMultipartCommit(
|
|
|
u.getKey(), u.getUploadId()));
|
|
@@ -838,44 +839,116 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Returns an {@link ExecutorService} for parallel tasks. The number of
|
|
|
+ * Returns an {@link Tasks.Submitter} for parallel tasks. The number of
|
|
|
* threads in the thread-pool is set by fs.s3a.committer.threads.
|
|
|
* If num-threads is 0, this will return null;
|
|
|
+ * this is used in Tasks as a cue
|
|
|
+ * to switch to single-threaded execution.
|
|
|
*
|
|
|
* @param context the JobContext for this commit
|
|
|
- * @return an {@link ExecutorService} or null for the number of threads
|
|
|
+ * @return a submitter or null
|
|
|
*/
|
|
|
- protected final synchronized ExecutorService buildThreadPool(
|
|
|
+ protected Tasks.Submitter buildSubmitter(
|
|
|
JobContext context) {
|
|
|
+ if (getThreadCount(context) > 0) {
|
|
|
+ return new PoolSubmitter(context);
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns an {@link ExecutorService} for parallel tasks. The number of
|
|
|
+ * threads in the thread-pool is set by fs.s3a.committer.threads.
|
|
|
+ * If num-threads is 0, this will raise an exception.
|
|
|
+ *
|
|
|
+ * @param context the JobContext for this commit
|
|
|
+ * @param numThreads threads
|
|
|
+ * @return an {@link ExecutorService} for the number of threads
|
|
|
+ */
|
|
|
+ private synchronized ExecutorService buildThreadPool(
|
|
|
+ JobContext context, int numThreads) {
|
|
|
+ Preconditions.checkArgument(numThreads > 0,
|
|
|
+ "Cannot create a thread pool with no threads");
|
|
|
if (threadPool == null) {
|
|
|
- int numThreads = context.getConfiguration().getInt(
|
|
|
- FS_S3A_COMMITTER_THREADS,
|
|
|
- DEFAULT_COMMITTER_THREADS);
|
|
|
LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads);
|
|
|
- if (numThreads > 0) {
|
|
|
- threadPool = HadoopExecutors.newFixedThreadPool(numThreads,
|
|
|
- new ThreadFactoryBuilder()
|
|
|
- .setDaemon(true)
|
|
|
- .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d")
|
|
|
- .build());
|
|
|
- } else {
|
|
|
- return null;
|
|
|
- }
|
|
|
+ threadPool = HadoopExecutors.newFixedThreadPool(numThreads,
|
|
|
+ new ThreadFactoryBuilder()
|
|
|
+ .setDaemon(true)
|
|
|
+ .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d")
|
|
|
+ .build());
|
|
|
}
|
|
|
return threadPool;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the thread count for this job's commit operations.
|
|
|
+ * @param context the JobContext for this commit
|
|
|
+ * @return a possibly zero thread count.
|
|
|
+ */
|
|
|
+ private int getThreadCount(final JobContext context) {
|
|
|
+ return context.getConfiguration().getInt(
|
|
|
+ FS_S3A_COMMITTER_THREADS,
|
|
|
+ DEFAULT_COMMITTER_THREADS);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Submit a runnable.
|
|
|
+ * This will demand-create the thread pool if needed.
|
|
|
+ * <p></p>
|
|
|
+ * This is synchronized to ensure the thread pool is always valid when
|
|
|
+ * work is synchronized. See HADOOP-16798.
|
|
|
+ * @param context the JobContext for this commit
|
|
|
+ * @param task task to execute
|
|
|
+ * @return the future of the submitted task.
|
|
|
+ */
|
|
|
+ private synchronized Future<?> submitRunnable(
|
|
|
+ final JobContext context,
|
|
|
+ final Runnable task) {
|
|
|
+ return buildThreadPool(context, getThreadCount(context)).submit(task);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The real task submitter, which hands off the work to
|
|
|
+ * the current thread pool.
|
|
|
+ */
|
|
|
+ private final class PoolSubmitter implements Tasks.Submitter {
|
|
|
+
|
|
|
+ private final JobContext context;
|
|
|
+
|
|
|
+ private final int numThreads;
|
|
|
+
|
|
|
+ private PoolSubmitter(final JobContext context) {
|
|
|
+ this.numThreads = getThreadCount(context);
|
|
|
+ Preconditions.checkArgument(numThreads > 0,
|
|
|
+ "Cannot create a thread pool with no threads");
|
|
|
+ this.context = context;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Future<?> submit(final Runnable task) {
|
|
|
+ return submitRunnable(context, task);
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* 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) {
|
|
|
+ protected void destroyThreadPool() {
|
|
|
+ ExecutorService pool;
|
|
|
+ // reset the thread pool in a sync block, then shut it down
|
|
|
+ // afterwards. This allows for other threads to create a
|
|
|
+ // new thread pool on demand.
|
|
|
+ synchronized(this) {
|
|
|
+ pool = this.threadPool;
|
|
|
+ threadPool = null;
|
|
|
+ }
|
|
|
+ if (pool != null) {
|
|
|
LOG.debug("Destroying thread pool");
|
|
|
- HadoopExecutors.shutdown(threadPool, LOG,
|
|
|
+ HadoopExecutors.shutdown(pool, LOG,
|
|
|
THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
|
|
|
- threadPool = null;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -884,11 +957,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
* 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() {
|
|
|
+ protected final synchronized Tasks.Submitter singleThreadSubmitter() {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
@@ -932,7 +1003,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
CommitOperations.CommitContext commitContext
|
|
|
= initiateCommitOperation()) {
|
|
|
Tasks.foreach(pending)
|
|
|
- .executeWith(buildThreadPool(context))
|
|
|
+ .executeWith(buildSubmitter(context))
|
|
|
.suppressExceptions(suppressExceptions)
|
|
|
.run(commitContext::abortSingleCommit);
|
|
|
}
|
|
@@ -961,7 +1032,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
|
|
CommitOperations.CommitContext commitContext
|
|
|
= initiateCommitOperation()) {
|
|
|
Tasks.foreach(pending.getSourceFiles())
|
|
|
- .executeWith(buildThreadPool(context))
|
|
|
+ .executeWith(buildSubmitter(context))
|
|
|
.suppressExceptions(suppressExceptions)
|
|
|
.run(path ->
|
|
|
loadAndAbort(commitContext,
|