You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2022/03/17 11:28:36 UTC

[hadoop] 02/03: MAPREDUCE-7341. Add an intermediate manifest committer for Azure and GCS

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 7328c34ba5049441dc8d1c30b2fe15784a82acf9
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Wed Mar 16 15:21:54 2022 +0000

    MAPREDUCE-7341. Add an intermediate manifest committer for Azure and GCS
    
    This is a mapreduce/spark output committer optimized for
    performance and correctness on Azure ADLS Gen 2 storage
    (via the abfs connector) and Google Cloud Storage
    (via the external gcs connector library).
    
    * It is safe to use with HDFS, however it has not been optimized
    for that use.
    * It is *not* safe for use with S3, and will fail if an attempt
    is made to do so.
    
    Contributed by Steve Loughran
    
    Change-Id: I6f3502e79c578b9fd1a8c1485f826784b5421fca
---
 .../hadoop-mapreduce-client-core/pom.xml           |    7 +-
 .../lib/output/PathOutputCommitterFactory.java     |   10 +-
 .../committer/manifest/ManifestCommitter.java      |  761 +++++++++
 .../manifest/ManifestCommitterConfig.java          |  375 ++++
 .../manifest/ManifestCommitterConstants.java       |  240 +++
 .../manifest/ManifestCommitterFactory.java         |   57 +
 .../manifest/ManifestCommitterStatisticNames.java  |  264 +++
 .../manifest/files/AbstractManifestData.java       |  137 ++
 .../committer/manifest/files/DiagnosticKeys.java   |   44 +
 .../output/committer/manifest/files/DirEntry.java  |  202 +++
 .../committer/manifest/files/EntryStatus.java      |   87 +
 .../output/committer/manifest/files/FileEntry.java |  189 ++
 .../committer/manifest/files/ManifestPrinter.java  |  144 ++
 .../manifest/files/ManifestSuccessData.java        |  493 ++++++
 .../committer/manifest/files/TaskManifest.java     |  365 ++++
 .../committer/manifest/files/package-info.java     |   41 +
 .../manifest/impl/AuditingIntegration.java         |   94 +
 .../committer/manifest/impl/InternalConstants.java |  130 ++
 .../manifest/impl/ManifestCommitterSupport.java    |  374 ++++
 .../manifest/impl/ManifestStoreOperations.java     |  291 ++++
 .../ManifestStoreOperationsThroughFileSystem.java  |  187 ++
 .../manifest/impl/OutputValidationException.java   |   40 +
 .../committer/manifest/impl/package-info.java      |   30 +
 .../output/committer/manifest/package-info.java    |   34 +
 .../committer/manifest/stages/AbortTaskStage.java  |   63 +
 .../manifest/stages/AbstractJobOrTaskStage.java    |  942 ++++++++++
 .../committer/manifest/stages/CleanupJobStage.java |  511 ++++++
 .../committer/manifest/stages/CommitJobStage.java  |  245 +++
 .../committer/manifest/stages/CommitTaskStage.java |  115 ++
 .../stages/CreateOutputDirectoriesStage.java       |  423 +++++
 .../committer/manifest/stages/JobOrTaskStage.java  |   33 +
 .../manifest/stages/LoadManifestsStage.java        |  293 ++++
 .../manifest/stages/RenameFilesStage.java          |  173 ++
 .../manifest/stages/SaveSuccessFileStage.java      |   79 +
 .../manifest/stages/SaveTaskManifestStage.java     |   81 +
 .../committer/manifest/stages/SetupJobStage.java   |   63 +
 .../committer/manifest/stages/SetupTaskStage.java  |   54 +
 .../committer/manifest/stages/StageConfig.java     |  556 ++++++
 .../manifest/stages/StageEventCallbacks.java       |   37 +
 .../stages/TaskAttemptScanDirectoryStage.java      |  214 +++
 .../manifest/stages/ValidateRenamedFilesStage.java |  200 +++
 .../committer/manifest/stages/package-info.java    |   30 +
 .../src/main/resources/mapred-default.xml          |   19 +
 .../src/site/markdown/manifest_committer.md        |  605 +++++++
 .../markdown/manifest_committer_architecture.md    |  335 ++++
 .../site/markdown/manifest_committer_protocol.md   |  617 +++++++
 .../manifest/AbstractManifestCommitterTest.java    | 1085 ++++++++++++
 .../committer/manifest/CommitterTestBinding.java   |  152 ++
 .../manifest/ManifestCommitterTestSupport.java     |  420 +++++
 .../committer/manifest/StubStoreOperations.java    |  109 ++
 .../committer/manifest/TaggedFileStatus.java       |   54 +
 .../committer/manifest/TestCleanupStage.java       |  142 ++
 .../committer/manifest/TestCommitTaskStage.java    |  126 ++
 .../manifest/TestCreateOutputDirectoriesStage.java |  307 ++++
 .../manifest/TestJobThroughManifestCommitter.java  |  601 +++++++
 .../committer/manifest/TestLoadManifestsStage.java |  141 ++
 .../manifest/TestManifestCommitProtocol.java       | 1801 ++++++++++++++++++++
 .../committer/manifest/TestRenameStageFailure.java |  379 ++++
 .../committer/manifest/TestTaskManifestFileIO.java |  185 ++
 .../committer/manifest/TextOutputForTests.java     |  136 ++
 .../committer/manifest/ThreadLeakTracker.java      |   83 +
 .../impl/UnreliableManifestStoreOperations.java    |  380 +++++
 .../output/committer/manifest/package-info.java    |   29 +
 .../src/test/resources/contract/localfs.xml        |  138 ++
 .../src/test/resources/core-site.xml               |   33 +
 65 files changed, 16579 insertions(+), 6 deletions(-)

diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index 957202a..7530428 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -103,9 +103,14 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-	<configuration>
+        <configuration>
           <systemPropertyVariables>
             <log4j.configuration>file:///${project.basedir}/src/test/resources/log4j.properties</log4j.configuration>
+
+            <!-- project directory   (target/) on a maven build. -->
+            <project.build.directory>${project.build.directory}</project.build.directory>
+            <!-- this property is used in hadoop-azure with separate paths for parallel runs -->
+            <test.build.dir>${project.build.directory}</test.build.dir>
           </systemPropertyVariables>
         </configuration>
       </plugin>
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java
index f3f0dab..cf83bf2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java
@@ -52,8 +52,8 @@ import org.apache.hadoop.util.ReflectionUtils;
  * attempt context and a possibly null path.
  *
  */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public class PathOutputCommitterFactory extends Configured {
   private static final Logger LOG =
       LoggerFactory.getLogger(PathOutputCommitterFactory.class);
@@ -158,7 +158,7 @@ public class PathOutputCommitterFactory extends Configured {
           scheme);
       if (StringUtils.isNotEmpty(conf.getTrimmed(schemeKey))) {
         // it does, so use that key in the classname lookup
-        LOG.debug("Using schema-specific factory for {}", outputPath);
+        LOG.info("Using schema-specific factory for {}", outputPath);
         key = schemeKey;
       } else {
         LOG.debug("No scheme-specific factory defined in {}", schemeKey);
@@ -171,7 +171,7 @@ public class PathOutputCommitterFactory extends Configured {
     String trimmedValue = conf.getTrimmed(key, "");
     if (StringUtils.isEmpty(trimmedValue)) {
       // empty/null value, use default
-      LOG.debug("No output committer factory defined,"
+      LOG.info("No output committer factory defined,"
           + " defaulting to FileOutputCommitterFactory");
       factory = FileOutputCommitterFactory.class;
     } else {
@@ -179,7 +179,7 @@ public class PathOutputCommitterFactory extends Configured {
       factory = conf.getClass(key,
           FileOutputCommitterFactory.class,
           PathOutputCommitterFactory.class);
-      LOG.debug("Using OutputCommitter factory class {} from key {}",
+      LOG.info("Using OutputCommitter factory class {} from key {}",
           factory, key);
     }
     return ReflectionUtils.newInstance(factory, conf);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java
new file mode 100644
index 0000000..99625e8
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java
@@ -0,0 +1,761 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbortTaskStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitJobStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitTaskStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupTaskStage;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageEventCallbacks;
+import org.apache.hadoop.util.functional.CloseableTaskPoolSubmitter;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_DIAGNOSTICS_MANIFEST_DIR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_COMPLETED_COUNT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_FAILED_COUNT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME_RECOVERED;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_ABORT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.updateCommonContextOnCommitterExit;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.updateCommonContextOnCommitterEntry;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createIOStatisticsStore;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage.cleanupStageOptionsFromConfig;
+
+/**
+ * This is the Intermediate-Manifest committer.
+ * At every entry point it updates the thread's audit context with
+ * the current stage info; this is a placeholder for
+ * adding audit information to stores other than S3A.
+ *
+ * This is tagged as public/stable. This is mandatory
+ * for the classname and PathOutputCommitter implementation
+ * classes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ManifestCommitter extends PathOutputCommitter implements
+    IOStatisticsSource, StageEventCallbacks {
+
+  public static final Logger LOG = LoggerFactory.getLogger(
+      ManifestCommitter.class);
+
+  /**
+   * Role: task committer.
+   */
+  public static final String TASK_COMMITTER = "task committer";
+
+  /**
+   * Role: job committer.
+   */
+  public static final String JOB_COMMITTER = "job committer";
+
+  /**
+   * Committer Configuration as extracted from
+   * the job/task context and set in the constructor.
+   */
+  private final ManifestCommitterConfig baseConfig;
+
+  /**
+   * Destination of the job.
+   */
+  private final Path destinationDir;
+
+  /**
+   * For tasks, the attempt directory.
+   * Null for jobs.
+   */
+  private final Path taskAttemptDir;
+
+  /**
+   * IOStatistics to update.
+   */
+  private final IOStatisticsStore iostatistics;
+
+  /**
+   *  The job Manifest Success data; only valid after a job successfully
+   *  commits.
+   */
+  private ManifestSuccessData successReport;
+
+  /**
+   * The active stage; is updated by a callback from within the stages.
+   */
+  private String activeStage;
+
+  /**
+   * The task manifest of the task commit.
+   * Null unless this is a task attempt and the
+   * task has successfully been committed.
+   */
+  private TaskManifest taskAttemptCommittedManifest;
+
+  /**
+   * Create a committer.
+   * @param outputPath output path
+   * @param context job/task context
+   * @throws IOException failure.
+   */
+  public ManifestCommitter(final Path outputPath,
+      final TaskAttemptContext context) throws IOException {
+    super(outputPath, context);
+    this.destinationDir = resolveDestinationDirectory(outputPath,
+        context.getConfiguration());
+    this.iostatistics = createIOStatisticsStore().build();
+    this.baseConfig = enterCommitter(
+        context.getTaskAttemptID() != null,
+        context);
+
+    this.taskAttemptDir = baseConfig.getTaskAttemptDir();
+    LOG.info("Created ManifestCommitter with JobID {},"
+            + " Task Attempt {} and destination {}",
+        context.getJobID(), context.getTaskAttemptID(), outputPath);
+  }
+
+  /**
+   * Committer method invoked; generates a config for it.
+   * Calls {@code #updateCommonContextOnCommitterEntry()}
+   * to update the audit context.
+   * @param isTask is this a task entry point?
+   * @param context context
+   * @return committer config
+   */
+  private ManifestCommitterConfig enterCommitter(boolean isTask,
+      JobContext context) {
+    ManifestCommitterConfig committerConfig =
+        new ManifestCommitterConfig(
+            getOutputPath(),
+            isTask ? TASK_COMMITTER : JOB_COMMITTER,
+            context,
+            iostatistics,
+            this);
+    updateCommonContextOnCommitterEntry(committerConfig);
+    return committerConfig;
+  }
+
+  /**
+   * Set up a job through a {@link SetupJobStage}.
+   * @param jobContext Context of the job whose output is being written.
+   * @throws IOException IO Failure.
+   */
+  @Override
+  public void setupJob(final JobContext jobContext) throws IOException {
+    ManifestCommitterConfig committerConfig = enterCommitter(false,
+        jobContext);
+    StageConfig stageConfig =
+        committerConfig
+            .createStageConfig()
+            .withOperations(createManifestStoreOperations())
+            .build();
+    // set up the job.
+    new SetupJobStage(stageConfig)
+        .apply(committerConfig.getCreateJobMarker());
+    logCommitterStatisticsAtDebug();
+  }
+
+  /**
+   * Set up a task through a {@link SetupTaskStage}.
+   * Classic FileOutputCommitter is a no-op here, relying
+   * on RecordWriters to create the dir implicitly on file
+   * create().
+   * FileOutputCommitter also uses the existence of that
+   * file as a flag to indicate task commit is needed.
+   * @param context task context.
+   * @throws IOException IO Failure.
+   */
+  @Override
+  public void setupTask(final TaskAttemptContext context)
+      throws IOException {
+    ManifestCommitterConfig committerConfig =
+        enterCommitter(true, context);
+    StageConfig stageConfig =
+        committerConfig
+            .createStageConfig()
+            .withOperations(createManifestStoreOperations())
+            .build();
+    // create task attempt dir; delete if present. Or fail?
+    new SetupTaskStage(stageConfig).apply("");
+    logCommitterStatisticsAtDebug();
+  }
+
+  /**
+   * Always return true.
+   * This way, even if there is no output, stats are collected.
+   * @param context task context.
+   * @return true
+   * @throws IOException IO Failure.
+   */
+  @Override
+  public boolean needsTaskCommit(final TaskAttemptContext context)
+      throws IOException {
+    LOG.info("Probe for needsTaskCommit({})",
+        context.getTaskAttemptID());
+    return true;
+  }
+
+  /**
+   * Failure during Job Commit is not recoverable from.
+   *
+   * @param jobContext
+   *          Context of the job whose output is being written.
+   * @return false, always
+   * @throws IOException never
+   */
+  @Override
+  public boolean isCommitJobRepeatable(final JobContext jobContext)
+      throws IOException {
+    LOG.info("Probe for isCommitJobRepeatable({}): returning false",
+        jobContext.getJobID());
+    return false;
+  }
+
+  /**
+   * Declare that task recovery is not supported.
+   * It would be, if someone added the code *and tests*.
+   * @param jobContext
+   *          Context of the job whose output is being written.
+   * @return false, always
+   * @throws IOException never
+   */
+  @Override
+  public boolean isRecoverySupported(final JobContext jobContext)
+      throws IOException {
+    LOG.info("Probe for isRecoverySupported({}): returning false",
+        jobContext.getJobID());
+    return false;
+  }
+
+  /**
+   *
+   * @param taskContext Context of the task whose output is being recovered
+   * @throws IOException always
+   */
+  @Override
+  public void recoverTask(final TaskAttemptContext taskContext)
+      throws IOException {
+    LOG.warn("Rejecting recoverTask({}) call", taskContext.getTaskAttemptID());
+    throw new IOException("Cannot recover task "
+        + taskContext.getTaskAttemptID());
+  }
+
+  /**
+   * Commit the task.
+   * This is where the task attempt tree list takes place.
+   * @param context task context.
+   * @throws IOException IO Failure.
+   */
+  @Override
+  public void commitTask(final TaskAttemptContext context)
+      throws IOException {
+    ManifestCommitterConfig committerConfig = enterCommitter(true,
+        context);
+    try {
+      StageConfig stageConfig = committerConfig.createStageConfig()
+          .withOperations(createManifestStoreOperations())
+          .build();
+      taskAttemptCommittedManifest = new CommitTaskStage(stageConfig)
+          .apply(null).getTaskManifest();
+      iostatistics.incrementCounter(COMMITTER_TASKS_COMPLETED_COUNT, 1);
+    } catch (IOException e) {
+      iostatistics.incrementCounter(COMMITTER_TASKS_FAILED_COUNT, 1);
+      throw e;
+    } finally {
+      logCommitterStatisticsAtDebug();
+      updateCommonContextOnCommitterExit();
+    }
+
+  }
+
+  /**
+   * Abort a task.
+   * @param context task context
+   * @throws IOException failure during the delete
+   */
+  @Override
+  public void abortTask(final TaskAttemptContext context)
+      throws IOException {
+    ManifestCommitterConfig committerConfig = enterCommitter(true,
+        context);
+    try {
+      new AbortTaskStage(
+          committerConfig.createStageConfig()
+              .withOperations(createManifestStoreOperations())
+              .build())
+          .apply(false);
+    } finally {
+      logCommitterStatisticsAtDebug();
+      updateCommonContextOnCommitterExit();
+    }
+  }
+
+  /**
+   * Get the manifest success data for this job; creating on demand if needed.
+   * @param committerConfig source config.
+   * @return the current {@link #successReport} value; never null.
+   */
+  private ManifestSuccessData getOrCreateSuccessData(
+      ManifestCommitterConfig committerConfig) {
+    if (successReport == null) {
+      successReport = createManifestOutcome(
+          committerConfig.createStageConfig(), activeStage);
+    }
+    return successReport;
+  }
+
+  /**
+   * This is the big job commit stage.
+   * Load the manifests, prepare the destination, rename
+   * the files then cleanup the job directory.
+   * @param jobContext Context of the job whose output is being written.
+   * @throws IOException failure.
+   */
+  @Override
+  public void commitJob(final JobContext jobContext) throws IOException {
+
+    ManifestCommitterConfig committerConfig = enterCommitter(false, jobContext);
+
+    // create the initial success data.
+    // this is overwritten by that created during the operation sequence,
+    // but if the sequence fails before that happens, it
+    // will be saved to the report directory.
+    ManifestSuccessData marker = getOrCreateSuccessData(committerConfig);
+    IOException failure = null;
+    try (CloseableTaskPoolSubmitter ioProcs =
+             committerConfig.createSubmitter();
+         ManifestStoreOperations storeOperations = createManifestStoreOperations()) {
+      // the stage config will be shared across all stages.
+      StageConfig stageConfig = committerConfig.createStageConfig()
+          .withOperations(storeOperations)
+          .withIOProcessors(ioProcs)
+          .build();
+
+      // commit the job, including any cleanup and validation.
+      final Configuration conf = jobContext.getConfiguration();
+      CommitJobStage.Result result = new CommitJobStage(stageConfig).apply(
+          new CommitJobStage.Arguments(
+              committerConfig.getCreateJobMarker(),
+              committerConfig.getValidateOutput(),
+              conf.getTrimmed(OPT_DIAGNOSTICS_MANIFEST_DIR, ""),
+              cleanupStageOptionsFromConfig(
+                  OP_STAGE_JOB_CLEANUP, conf)
+          ));
+      marker = result.getJobSuccessData();
+      // update the cached success with the new report.
+      setSuccessReport(marker);
+
+    } catch (IOException e) {
+      // failure. record it for the summary
+      failure = e;
+      // rethrow
+      throw e;
+    } finally {
+      // save the report summary, even on failure
+      maybeSaveSummary(activeStage,
+          committerConfig,
+          marker,
+          failure,
+          true,
+          true);
+      // print job commit stats
+      LOG.info("{}: Job Commit statistics {}",
+          committerConfig.getName(),
+          ioStatisticsToPrettyString(iostatistics));
+      // and warn of rename problems
+      final Long recoveries = iostatistics.counters().get(OP_COMMIT_FILE_RENAME_RECOVERED);
+      if (recoveries != null && recoveries > 0) {
+        LOG.warn("{}: rename failures were recovered from. Number of recoveries: {}",
+            committerConfig.getName(), recoveries);
+      }
+      updateCommonContextOnCommitterExit();
+    }
+  }
+
+  /**
+   * Abort the job.
+   * Invokes
+   * {@link #executeCleanup(String, JobContext, ManifestCommitterConfig)}
+   * then saves the (ongoing) job report data if reporting is enabled.
+   * @param jobContext Context of the job whose output is being written.
+   * @param state final runstate of the job
+   * @throws IOException failure during cleanup; report failure are swallowed
+   */
+  @Override
+  public void abortJob(final JobContext jobContext,
+      final JobStatus.State state)
+      throws IOException {
+    LOG.info("Aborting Job {} in state {}", jobContext.getJobID(), state);
+    ManifestCommitterConfig committerConfig = enterCommitter(false,
+        jobContext);
+    ManifestSuccessData report = getOrCreateSuccessData(
+        committerConfig);
+    IOException failure = null;
+
+    try {
+      executeCleanup(OP_STAGE_JOB_ABORT, jobContext, committerConfig);
+    } catch (IOException e) {
+      // failure.
+      failure = e;
+    }
+    report.setSuccess(false);
+    // job abort does not overwrite any existing report, so a job commit
+    // failure cause will be preserved.
+    maybeSaveSummary(activeStage, committerConfig, report, failure,
+        true, false);
+    // print job stats
+    LOG.info("Job Abort statistics {}",
+        ioStatisticsToPrettyString(iostatistics));
+    updateCommonContextOnCommitterExit();
+  }
+
+  /**
+   * Execute the {@code CleanupJobStage} to remove the job attempt dir.
+   * This does
+   * @param jobContext Context of the job whose output is being written.
+   * @throws IOException failure during cleanup
+   */
+  @SuppressWarnings("deprecation")
+  @Override
+  public void cleanupJob(final JobContext jobContext) throws IOException {
+    ManifestCommitterConfig committerConfig = enterCommitter(false,
+        jobContext);
+    try {
+      executeCleanup(OP_STAGE_JOB_CLEANUP, jobContext, committerConfig);
+    } finally {
+      logCommitterStatisticsAtDebug();
+      updateCommonContextOnCommitterExit();
+    }
+  }
+
+  /**
+   * Perform the cleanup operation for job cleanup or abort.
+   * @param statisticName statistic/stage name
+   * @param jobContext job context
+   * @param committerConfig committer config
+   * @throws IOException failure
+   * @return the outcome
+   */
+  private CleanupJobStage.Result executeCleanup(
+      final String statisticName,
+      final JobContext jobContext,
+      final ManifestCommitterConfig committerConfig) throws IOException {
+    try (CloseableTaskPoolSubmitter ioProcs =
+             committerConfig.createSubmitter()) {
+
+      return new CleanupJobStage(
+          committerConfig.createStageConfig()
+              .withOperations(createManifestStoreOperations())
+              .withIOProcessors(ioProcs)
+              .build())
+          .apply(cleanupStageOptionsFromConfig(
+              statisticName,
+              jobContext.getConfiguration()));
+    }
+  }
+
+  /**
+   * Output path: destination directory of the job.
+   * @return the overall job destination directory.
+   */
+  @Override
+  public Path getOutputPath() {
+    return getDestinationDir();
+  }
+
+  /**
+   * Work path of the current task attempt.
+   * This is null if the task does not have one.
+   * @return a path.
+   */
+  @Override
+  public Path getWorkPath() {
+    return getTaskAttemptDir();
+  }
+
+  /**
+   * Get the job destination dir.
+   * @return dest dir.
+   */
+  private Path getDestinationDir() {
+    return destinationDir;
+  }
+
+  /**
+   * Get the task attempt dir.
+   * May be null.
+   * @return a path or null.
+   */
+  private Path getTaskAttemptDir() {
+    return taskAttemptDir;
+  }
+
+  /**
+   * Callback on stage entry.
+   * Sets {@link #activeStage} and updates the
+   * common context.
+   * @param stage new stage
+   */
+  @Override
+  public void enterStage(String stage) {
+    activeStage = stage;
+    AuditingIntegration.enterStage(stage);
+  }
+
+  /**
+   * Remove stage from common audit context.
+   * @param stage stage exited.
+   */
+  @Override
+  public void exitStage(String stage) {
+    AuditingIntegration.exitStage();
+  }
+
+  /**
+   * Get the unique ID of this job.
+   * @return job ID (yarn, spark)
+   */
+  public String getJobUniqueId() {
+    return baseConfig.getJobUniqueId();
+  }
+
+  /**
+   * Get the config of the task attempt this instance was constructed
+   * with.
+   * @return a configuration.
+   */
+  public Configuration getConf() {
+    return baseConfig.getConf();
+  }
+
+  /**
+   * Get the manifest Success data; only valid after a job.
+   * @return the job _SUCCESS data, or null.
+   */
+  public ManifestSuccessData getSuccessReport() {
+    return successReport;
+  }
+
+  private void setSuccessReport(ManifestSuccessData successReport) {
+    this.successReport = successReport;
+  }
+
+  /**
+   * Get the manifest of the last committed task.
+   * @return a task manifest or null.
+   */
+  @VisibleForTesting
+  TaskManifest getTaskAttemptCommittedManifest() {
+    return taskAttemptCommittedManifest;
+  }
+
+  /**
+   * Compute the path where the output of a task attempt is stored until
+   * that task is committed.
+   * @param context the context of the task attempt.
+   * @return the path where a task attempt should be stored.
+   */
+  @VisibleForTesting
+  public Path getTaskAttemptPath(TaskAttemptContext context) {
+    return enterCommitter(false, context).getTaskAttemptDir();
+  }
+
+  /**
+   * The path to where the manifest file of a task attempt will be
+   * saved when the task is committed.
+   * This path will be the same for all attempts of the same task.
+   * @param context the context of the task attempt.
+   * @return the path where a task attempt should be stored.
+   */
+  @VisibleForTesting
+  public Path getTaskManifestPath(TaskAttemptContext context) {
+    final Path dir = enterCommitter(false, context).getTaskManifestDir();
+
+    return manifestPathForTask(dir,
+        context.getTaskAttemptID().getTaskID().toString());
+  }
+
+  /**
+   * Compute the path where the output of a task attempt is stored until
+   * that task is committed.
+   * @param context the context of the task attempt.
+   * @return the path where a task attempt should be stored.
+   */
+  @VisibleForTesting
+  public Path getJobAttemptPath(JobContext context) {
+
+    return enterCommitter(false, context).getJobAttemptDir();
+  }
+
+  /**
+   * Get the final output path, including resolving any relative path.
+   * @param outputPath output path
+   * @param conf configuration to create any FS with
+   * @return a resolved path.
+   * @throws IOException failure.
+   */
+  private Path resolveDestinationDirectory(Path outputPath,
+      Configuration conf) throws IOException {
+    return FileSystem.get(outputPath.toUri(), conf).makeQualified(outputPath);
+  }
+
+  /**
+   * Create manifest store operations for the destination store.
+   * This MUST NOT be used for the success report operations, as
+   * they may be to a different filesystem.
+   * This is a point which can be overridden during testing.
+   * @return a new store operations instance bonded to the destination fs.
+   * @throws IOException failure to instantiate.
+   */
+  protected ManifestStoreOperations createManifestStoreOperations() throws IOException {
+    return ManifestCommitterSupport.createManifestStoreOperations(
+        baseConfig.getConf(),
+        baseConfig.getDestinationFileSystem(),
+        baseConfig.getDestinationDir());
+  }
+
+  /**
+   * Log IO Statistics at debug.
+   */
+  private void logCommitterStatisticsAtDebug() {
+    logIOStatisticsAtDebug(LOG, "Committer Statistics", this);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "ManifestCommitter{");
+    sb.append(baseConfig);
+    sb.append(", iostatistics=").append(ioStatisticsToPrettyString(iostatistics));
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Save a summary to the report dir if the config option
+   * is set.
+   * The IOStatistics of the summary will be updated to the latest
+   * snapshot of the committer's statistics, so the report is up
+   * to date.
+   * The report will updated with the current active stage,
+   * and if {@code thrown} is non-null, it will be added to the
+   * diagnistics (and the job tagged as a failure).
+   * Static for testability.
+   * @param activeStage active stage
+   * @param config configuration to use.
+   * @param report summary file.
+   * @param thrown any exception indicting failure.
+   * @param quiet should exceptions be swallowed.
+   * @param overwrite should the existing file be overwritten
+   * @return the path of a file, if successfully saved
+   * @throws IOException if a failure occured and quiet==false
+   */
+  private static Path maybeSaveSummary(
+      String activeStage,
+      ManifestCommitterConfig config,
+      ManifestSuccessData report,
+      Throwable thrown,
+      boolean quiet,
+      boolean overwrite) throws IOException {
+    Configuration conf = config.getConf();
+    String reportDir = conf.getTrimmed(OPT_SUMMARY_REPORT_DIR, "");
+    if (reportDir.isEmpty()) {
+      LOG.debug("No summary directory set in " + OPT_SUMMARY_REPORT_DIR);
+      return null;
+    }
+    LOG.debug("Summary directory set in to {}" + OPT_SUMMARY_REPORT_DIR,
+        reportDir);
+
+    // update to the latest statistics
+    report.snapshotIOStatistics(config.getIOStatistics());
+
+    Path reportDirPath = new Path(reportDir);
+    Path path = new Path(reportDirPath,
+        createJobSummaryFilename(config.getJobUniqueId()));
+
+    if (thrown != null) {
+      report.recordJobFailure(thrown);
+    }
+    report.putDiagnostic(STAGE, activeStage);
+    // the store operations here is explicitly created for the FS where
+    // the reports go, which may not be the target FS of the job.
+
+    final FileSystem fs = path.getFileSystem(conf);
+    try (ManifestStoreOperations operations = new ManifestStoreOperationsThroughFileSystem(fs)) {
+      if (!overwrite) {
+        // check for file existence so there is no need to worry about
+        // precisely what exception is raised when overwrite=false and dest file
+        // exists
+        try {
+          FileStatus st = operations.getFileStatus(path);
+          // get here and the file exists
+          LOG.debug("Report already exists: {}", st);
+          return null;
+        } catch (FileNotFoundException ignored) {
+        }
+      }
+      operations.save(report, path, overwrite);
+      LOG.info("Job summary saved to {}", path);
+      return path;
+    } catch (IOException e) {
+      LOG.debug("Failed to save summary to {}", path, e);
+      if (quiet) {
+        return null;
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  @Override
+  public IOStatisticsStore getIOStatistics() {
+    return iostatistics;
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java
new file mode 100644
index 0000000..40dffee
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java
@@ -0,0 +1,375 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageEventCallbacks;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.apache.hadoop.util.functional.CloseableTaskPoolSubmitter;
+
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.*;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.buildJobUUID;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.getAppAttemptId;
+
+/**
+ * The configuration for the committer as built up from the job configuration
+ * and data passed down from the committer factory.
+ * Isolated for ease of dev/test
+ */
+public final class ManifestCommitterConfig implements IOStatisticsSource {
+
+  /**
+   * Final destination of work.
+   * This is <i>unqualified</i>.
+   */
+  private final Path destinationDir;
+
+  /**
+   * Role: used in log/text messages.
+   */
+  private final String role;
+
+  /**
+   * This is the directory for all intermediate work: where the output
+   * format will write data.
+   * Will be null if built from a job context.
+   */
+  private final Path taskAttemptDir;
+
+  /** Configuration of the job. */
+  private final Configuration conf;
+
+  /** The job context. For a task, this can be cast to a TaskContext. */
+  private final JobContext jobContext;
+
+  /** Should a job marker be created? */
+  private final boolean createJobMarker;
+
+  /**
+   * Job ID Or UUID -without any attempt suffix.
+   * This is expected/required to be unique, though
+   * Spark has had "issues" there until recently
+   * with lack of uniqueness of generated MR Job IDs.
+   */
+  private final String jobUniqueId;
+
+  /**
+   * Where did the job Unique ID come from?
+   */
+  private final String jobUniqueIdSource;
+
+  /**
+   * Number of this attempt; starts at zero.
+   */
+  private final int jobAttemptNumber;
+
+  /**
+   * Job ID + AttemptID.
+   */
+  private final String jobAttemptId;
+
+  /**
+   * Task ID: used as the filename of the manifest.
+   * Will be "" if built from a job context.
+   */
+  private final String taskId;
+
+  /**
+   * Task attempt ID. Determines the working
+   * directory for task attempts to write data into,
+   * and for the task committer to scan.
+   * Will be "" if built from a job context.
+   */
+  private final String taskAttemptId;
+
+  /** Any progressable for progress callbacks. */
+  private final Progressable progressable;
+
+  /**
+   * IOStatistics to update.
+   */
+  private final IOStatisticsStore iostatistics;
+
+
+  /** Should the output be validated after the commit? */
+  private final boolean validateOutput;
+
+  /**
+   * Attempt directory management.
+   */
+  private final ManifestCommitterSupport.AttemptDirectories dirs;
+
+  /**
+   * Callback when a stage is entered.
+   */
+  private final StageEventCallbacks stageEventCallbacks;
+
+  /**
+   * Name for logging.
+   */
+  private final String name;
+
+  /**
+   * Delete target paths on commit? Stricter, but
+   * higher IO cost.
+   */
+  private final boolean deleteTargetPaths;
+
+  /**
+   * Constructor.
+   * @param outputPath destination path of the job.
+   * @param role role for log messages.
+   * @param context job/task context
+   * @param iostatistics IO Statistics
+   * @param stageEventCallbacks stage event callbacks.
+   */
+
+  ManifestCommitterConfig(
+      final Path outputPath,
+      final String role,
+      final JobContext context,
+      final IOStatisticsStore iostatistics,
+      final StageEventCallbacks stageEventCallbacks) {
+    this.role = role;
+    this.jobContext = context;
+    this.conf = context.getConfiguration();
+    this.destinationDir = outputPath;
+    this.iostatistics = iostatistics;
+    this.stageEventCallbacks = stageEventCallbacks;
+
+    Pair<String, String> pair = buildJobUUID(conf, context.getJobID());
+    this.jobUniqueId = pair.getLeft();
+    this.jobUniqueIdSource = pair.getRight();
+    this.jobAttemptNumber = getAppAttemptId(context);
+    this.jobAttemptId = this.jobUniqueId + "_" + jobAttemptNumber;
+
+    // build directories
+    this.dirs = new ManifestCommitterSupport.AttemptDirectories(outputPath,
+        this.jobUniqueId, jobAttemptNumber);
+
+    // read in configuration options
+    this.createJobMarker = conf.getBoolean(
+        SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
+        DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
+    this.validateOutput = conf.getBoolean(
+        OPT_VALIDATE_OUTPUT,
+        OPT_VALIDATE_OUTPUT_DEFAULT);
+    this.deleteTargetPaths = conf.getBoolean(
+        OPT_DELETE_TARGET_FILES,
+        OPT_DELETE_TARGET_FILES_DEFAULT);
+
+    // if constructed with a task attempt, build the task ID and path.
+    if (context instanceof TaskAttemptContext) {
+      // it's a task
+      final TaskAttemptContext tac = (TaskAttemptContext) context;
+      TaskAttemptID taskAttempt = Objects.requireNonNull(
+          tac.getTaskAttemptID());
+      taskAttemptId = taskAttempt.toString();
+      taskId = taskAttempt.getTaskID().toString();
+      // Task attempt dir; must be different across instances
+      taskAttemptDir = dirs.getTaskAttemptPath(taskAttemptId);
+      // the context is also the progress callback.
+      progressable = tac;
+      name = String.format(InternalConstants.NAME_FORMAT_TASK_ATTEMPT, taskAttemptId);
+
+    } else {
+      // it's a job
+      taskId = "";
+      taskAttemptId = "";
+      taskAttemptDir = null;
+      progressable = null;
+      name = String.format(InternalConstants.NAME_FORMAT_JOB_ATTEMPT, jobAttemptId);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "ManifestCommitterConfig{" +
+        "name=" + name +
+        ", destinationDir=" + destinationDir +
+        ", role='" + role + '\'' +
+        ", taskAttemptDir=" + taskAttemptDir +
+        ", createJobMarker=" + createJobMarker +
+        ", jobUniqueId='" + jobUniqueId + '\'' +
+        ", jobUniqueIdSource='" + jobUniqueIdSource + '\'' +
+        ", jobAttemptNumber=" + jobAttemptNumber +
+        ", jobAttemptId='" + jobAttemptId + '\'' +
+        ", taskId='" + taskId + '\'' +
+        ", taskAttemptId='" + taskAttemptId + '\'' +
+        '}';
+  }
+
+  /**
+   * Get the destination filesystem.
+   * @return destination FS.
+   * @throws IOException Problems binding to the destination FS.
+   */
+  FileSystem getDestinationFileSystem() throws IOException {
+    return FileSystem.get(destinationDir.toUri(), conf);
+  }
+
+  /**
+   * Create the stage config from the committer
+   * configuration.
+   * This does not bind the store operations
+   * or processors.
+   * @return a stage config with configuration options passed in.
+   */
+  StageConfig createStageConfig() {
+    StageConfig stageConfig = new StageConfig();
+    stageConfig
+        .withIOStatistics(iostatistics)
+        .withJobAttemptNumber(jobAttemptNumber)
+        .withJobDirectories(dirs)
+        .withJobId(jobUniqueId)
+        .withJobIdSource(jobUniqueIdSource)
+        .withName(name)
+        .withProgressable(progressable)
+        .withStageEventCallbacks(stageEventCallbacks)
+        .withTaskAttemptDir(taskAttemptDir)
+        .withTaskAttemptId(taskAttemptId)
+        .withTaskId(taskId)
+        .withDeleteTargetPaths(deleteTargetPaths);
+
+    return stageConfig;
+  }
+
+  public Path getDestinationDir() {
+    return destinationDir;
+  }
+
+  public String getRole() {
+    return role;
+  }
+
+  public Path getTaskAttemptDir() {
+    return taskAttemptDir;
+  }
+
+  public Path getJobAttemptDir() {
+    return dirs.getJobAttemptDir();
+  }
+
+  public Path getTaskManifestDir() {
+    return dirs.getTaskManifestDir();
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public JobContext getJobContext() {
+    return jobContext;
+  }
+
+  public boolean getCreateJobMarker() {
+    return createJobMarker;
+  }
+
+  public String getJobAttemptId() {
+    return jobAttemptId;
+  }
+
+  public String getTaskAttemptId() {
+    return taskAttemptId;
+  }
+
+  public String getTaskId() {
+    return taskId;
+  }
+
+  public String getJobUniqueId() {
+    return jobUniqueId;
+  }
+
+  public boolean getValidateOutput() {
+    return validateOutput;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public IOStatisticsStore getIOStatistics() {
+    return iostatistics;
+  }
+
+  /**
+   * Create a new submitter task pool from the
+   * {@link ManifestCommitterConstants#OPT_IO_PROCESSORS}
+   * settings.
+   * @return a new thread pool.
+   */
+  public CloseableTaskPoolSubmitter createSubmitter() {
+    return createSubmitter(
+        OPT_IO_PROCESSORS, OPT_IO_PROCESSORS_DEFAULT);
+  }
+
+  /**
+   * Create a new submitter task pool.
+   * @param key config key with pool size.
+   * @param defVal default value.
+   * @return a new task pool.
+   */
+  public CloseableTaskPoolSubmitter createSubmitter(String key, int defVal) {
+    int numThreads = conf.getInt(key, defVal);
+    if (numThreads <= 0) {
+      // ignore the setting if it is too invalid.
+      numThreads = defVal;
+    }
+    return createCloseableTaskSubmitter(numThreads, getJobAttemptId());
+  }
+
+  /**
+   * Create a new submitter task pool.
+   *
+   * @param numThreads thread count.
+   * @param jobAttemptId job ID
+   * @return a new task pool.
+   */
+  public static CloseableTaskPoolSubmitter createCloseableTaskSubmitter(
+      final int numThreads,
+      final String jobAttemptId) {
+    return new CloseableTaskPoolSubmitter(
+        HadoopExecutors.newFixedThreadPool(numThreads,
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("manifest-committer-" + jobAttemptId + "-%d")
+                .build()));
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java
new file mode 100644
index 0000000..eb344e8
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java
@@ -0,0 +1,240 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem;
+
+/**
+ * Public constants for the manifest committer.
+ * This includes all configuration options and their default values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class ManifestCommitterConstants {
+
+  /**
+   * Suffix to use in manifest files in the manifest subdir.
+   * Value: {@value}.
+   */
+  public static final String MANIFEST_SUFFIX = "-manifest.json";
+
+  /**
+   * Prefix for summary files in the report dir. Call
+   */
+  public static final String SUMMARY_FILENAME_PREFIX = "summary-";
+
+  /**
+   * Format string used to build a summary file from a Job ID.
+   */
+  public static final String SUMMARY_FILENAME_FORMAT =
+      SUMMARY_FILENAME_PREFIX + "%s.json";
+
+  /**
+   * Suffix to use for temp files before renaming them.
+   * Value: {@value}.
+   */
+  public static final String TMP_SUFFIX = ".tmp";
+
+  /**
+   * Initial number of all app attempts.
+   * This is fixed in YARN; for Spark jobs the
+   * same number "0" is used.
+   */
+  public static final int INITIAL_APP_ATTEMPT_ID = 0;
+
+  /**
+   * Format string for building a job dir.
+   * Value: {@value}.
+   */
+  public static final String JOB_DIR_FORMAT_STR = "%s";
+
+  /**
+   * Format string for building a job attempt dir.
+   * This uses the job attempt number so previous versions
+   * can be found trivially.
+   * Value: {@value}.
+   */
+  public static final String JOB_ATTEMPT_DIR_FORMAT_STR = "%02d";
+
+  /**
+   * Name of directory under job attempt dir for manifests.
+   */
+  public static final String JOB_TASK_MANIFEST_SUBDIR = "manifests";
+
+  /**
+   * Name of directory under job attempt dir for task attempts.
+   */
+  public static final String JOB_TASK_ATTEMPT_SUBDIR = "tasks";
+
+
+  /**
+   * Committer classname as recorded in the committer _SUCCESS file.
+   */
+  public static final String MANIFEST_COMMITTER_CLASSNAME =
+      ManifestCommitter.class.getName();
+
+  /**
+   * Marker file to create on success: {@value}.
+   */
+  public static final String SUCCESS_MARKER = "_SUCCESS";
+
+  /** Default job marker option: {@value}. */
+  public static final boolean DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER = true;
+
+  /**
+   * The limit to the number of committed objects tracked during
+   * job commits and saved to the _SUCCESS file.
+   * Value: {@value}.
+   */
+  public static final int SUCCESS_MARKER_FILE_LIMIT = 100;
+
+  /**
+   * The UUID for jobs: {@value}.
+   * This was historically created in Spark 1.x's SQL queries,
+   * but "went away".
+   * It has been restored in recent spark releases.
+   * If found: it is used instead of the MR job attempt ID.
+   */
+  public static final String SPARK_WRITE_UUID = "spark.sql.sources.writeJobUUID";
+
+  /**
+   * String to use as source of the job ID.
+   * This SHOULD be kept in sync with that of
+   * {@code AbstractS3ACommitter.JobUUIDSource}.
+   * Value: {@value}.
+   */
+  public static final String JOB_ID_SOURCE_MAPREDUCE = "JobID";
+
+  /**
+   * Prefix to use for config options: {@value}.
+   */
+  public static final String OPT_PREFIX = "mapreduce.manifest.committer.";
+
+  /**
+   * Should dir cleanup do parallel deletion of task attempt dirs
+   * before trying to delete the toplevel dirs.
+   * For GCS this may deliver speedup, while on ABFS it may avoid
+   * timeouts in certain deployments.
+   * Value: {@value}.
+   */
+  public static final String OPT_CLEANUP_PARALLEL_DELETE =
+      OPT_PREFIX + "cleanup.parallel.delete";
+
+  /**
+   * Default value:  {@value}.
+   */
+  public static final boolean OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT = true;
+
+  /**
+   * Threads to use for IO.
+   */
+  public static final String OPT_IO_PROCESSORS = OPT_PREFIX + "io.threads";
+
+  /**
+   * Default value:  {@value}.
+   */
+  public static final int OPT_IO_PROCESSORS_DEFAULT = 64;
+
+  /**
+   * Directory for saving job summary reports.
+   * These are the _SUCCESS files, but are saved even on
+   * job failures.
+   * Value: {@value}.
+   */
+  public static final String OPT_SUMMARY_REPORT_DIR =
+      OPT_PREFIX + "summary.report.directory";
+
+  /**
+   * Directory for moving manifests under for diagnostics.
+   * Value: {@value}.
+   */
+  public static final String OPT_DIAGNOSTICS_MANIFEST_DIR =
+      OPT_PREFIX + "diagnostics.manifest.directory";
+
+  /**
+   * Should the output be validated?
+   * This will check expected vs actual file lengths, and,
+   * if etags can be obtained, etags.
+   * Value: {@value}.
+   */
+  public static final String OPT_VALIDATE_OUTPUT = OPT_PREFIX + "validate.output";
+
+  /**
+   * Default value: {@value}.
+   */
+  public static final boolean OPT_VALIDATE_OUTPUT_DEFAULT = false;
+
+  /**
+   * Should job commit delete for files/directories at the targets
+   * of renames, and, if found, deleting them?
+   *
+   * This is part of the effective behavior of the FileOutputCommitter,
+   * however it adds an extra delete call per file being committed.
+   *
+   * If a job is writing to a directory which has only just been created
+   * or were unique filenames are being used, there is no need to perform
+   * this preparation.
+   * The recognition of newly created dirs is automatic.
+   *
+   * Value: {@value}.
+   */
+  public static final String OPT_DELETE_TARGET_FILES =
+      OPT_PREFIX + "delete.target.files";
+
+  /**
+   * Default value: {@value}.
+   */
+  public static final boolean OPT_DELETE_TARGET_FILES_DEFAULT = false;
+
+  /**
+   * Name of the factory: {@value}.
+   */
+  public static final String MANIFEST_COMMITTER_FACTORY =
+      ManifestCommitterFactory.class.getName();
+
+  /**
+   * Classname of the store operations; filesystems and tests
+   * may override.
+   * Value: {@value}.
+   */
+  public static final String OPT_STORE_OPERATIONS_CLASS = OPT_PREFIX + "store.operations.classname";
+
+  /**
+   * Default classname of the store operations.
+   * Value: {@value}.
+   */
+  public static final String STORE_OPERATIONS_CLASS_DEFAULT =
+      ManifestStoreOperationsThroughFileSystem.class.getName();
+
+  /**
+   * Stage attribute in audit context: {@value}.
+   */
+  public static final String CONTEXT_ATTR_STAGE = "st";
+
+  /**
+   * Task ID attribute in audit context: {@value}.
+   */
+  public static final String CONTEXT_ATTR_TASK_ATTEMPT_ID = "ta";
+
+  private ManifestCommitterConstants() {
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java
new file mode 100644
index 0000000..b07182c
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.UNSUPPORTED_FS_SCHEMAS;
+
+/**
+ * This is the committer factory to register as the source of committers
+ * for the job/filesystem schema.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ManifestCommitterFactory extends PathOutputCommitterFactory {
+
+  /**
+   * Name of this factory.
+   */
+  public static final String NAME = ManifestCommitterFactory.class.getName();
+
+  @Override
+  public ManifestCommitter createOutputCommitter(final Path outputPath,
+      final TaskAttemptContext context) throws IOException {
+    // safety check. S3A does not support this, so fail fast.
+    final String scheme = outputPath.toUri().getScheme();
+    if (UNSUPPORTED_FS_SCHEMAS.contains(scheme)) {
+      throw new PathIOException(outputPath.toString(),
+          "This committer does not work with the filesystem of type " + scheme);
+    }
+    return new ManifestCommitter(outputPath, context);
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java
new file mode 100644
index 0000000..243fd60
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java
@@ -0,0 +1,264 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
+
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_RENAME;
+
+/**
+ * Statistic names for committers.
+ * Please keep in sync with org.apache.hadoop.fs.s3a.Statistic
+ * so that S3A and manifest committers are in sync.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class ManifestCommitterStatisticNames {
+
+
+  /** Amount of data committed: {@value}. */
+  public static final String COMMITTER_BYTES_COMMITTED_COUNT =
+      "committer_bytes_committed";
+
+  /** Duration Tracking of time to commit an entire job: {@value}. */
+  public static final String COMMITTER_COMMIT_JOB =
+      "committer_commit_job";
+
+  /** Number of files committed: {@value}. */
+  public static final String COMMITTER_FILES_COMMITTED_COUNT =
+      "committer_files_committed";
+
+  /** "Count of successful tasks:: {@value}. */
+  public static final String COMMITTER_TASKS_COMPLETED_COUNT =
+      "committer_tasks_completed";
+
+  /** Count of failed tasks: {@value}. */
+  public static final String COMMITTER_TASKS_FAILED_COUNT =
+      "committer_tasks_failed";
+
+  /** Count of commits aborted: {@value}. */
+  public static final String COMMITTER_COMMITS_ABORTED_COUNT =
+      "committer_commits_aborted";
+
+  /** Count of commits reverted: {@value}. */
+  public static final String COMMITTER_COMMITS_REVERTED_COUNT =
+      "committer_commits_reverted";
+
+  /** Count of commits failed: {@value}. */
+  public static final String COMMITTER_COMMITS_FAILED =
+      "committer_commits" + StoreStatisticNames.SUFFIX_FAILURES;
+
+  /**
+   * The number of files in a task. This will be a MeanStatistic.
+   */
+  public static final String COMMITTER_FILE_COUNT_MEAN =
+      "committer_task_file_count";
+
+  /**
+   * File Size.
+   */
+  public static final String COMMITTER_FILE_SIZE_MEAN =
+      "committer_task_file_size";
+
+  /**
+   * What is a task attempt's directory count.
+   */
+  public static final String COMMITTER_TASK_DIRECTORY_COUNT_MEAN =
+      "committer_task_directory_count";
+
+  /**
+   * What is the depth of a task attempt's directory tree.
+   */
+  public static final String COMMITTER_TASK_DIRECTORY_DEPTH_MEAN =
+      "committer_task_directory_depth";
+
+  /**
+   * The number of files in a task. This will be a MeanStatistic.
+   */
+  public static final String COMMITTER_TASK_FILE_COUNT_MEAN =
+      "committer_task_file_count";
+
+  /**
+   * The number of files in a task. This will be a MeanStatistic.
+   */
+  public static final String COMMITTER_TASK_FILE_SIZE_MEAN =
+      "committer_task_file_size";
+
+  /**
+   * The size of manifest files. This will be a MeanStatistic.
+   * Useful to highlight whether this size is too large and
+   * effort to optimise for file IO and memory consumption
+   * justified.
+   */
+  public static final String COMMITTER_TASK_MANIFEST_FILE_SIZE =
+      "committer_task_manifest_file_size";
+
+  /**
+   * A file renamed during the commit operation {@value}.
+   */
+  public static final String OP_COMMIT_FILE_RENAME =
+      "commit_file_rename";
+
+  /**
+   * A file rename during the commit operation was recovered from a failure {@value}.
+   */
+  public static final String OP_COMMIT_FILE_RENAME_RECOVERED =
+      "commit_file_rename_recovered";
+
+  /** Directory creation {@value}. */
+  public static final String OP_CREATE_DIRECTORIES = "op_create_directories";
+
+  /** Creating a single directory {@value}. */
+  public static final String OP_CREATE_ONE_DIRECTORY =
+      "op_create_one_directory";
+
+  /**
+   * A file in the destination directory tree has been deleted.
+   *  {@value}.
+   */
+  public static final String OP_DELETE_FILE_UNDER_DESTINATION =
+      "op_delete_file_under_destination";
+
+  /** Directory scan {@value}. */
+  public static final String OP_DIRECTORY_SCAN = "op_directory_scan";
+
+  /**
+   * Overall job commit {@value}.
+   */
+  public static final String OP_STAGE_JOB_COMMIT = COMMITTER_COMMIT_JOB;
+
+  /** {@value}. */
+  public static final String OP_LOAD_ALL_MANIFESTS = "op_load_all_manifests";
+
+  /**
+   * Load a task manifest: {@value}.
+   */
+  public static final String OP_LOAD_MANIFEST = "op_load_manifest";
+
+  /**
+   * mkdir failure statistic: {@value}.
+   * This is incremented whenever mkdir() returns false, e.g because
+   * there was a file at the end of the path.
+   */
+  public static final String OP_MKDIRS_RETURNED_FALSE = "op_mkdir_returned_false";
+
+  /**
+   * msync statistic: {@value}.
+   * This should be the same as StoreStatisticNames.OP_MSYNC; it's
+   * duplicated just to isolate this committer into a single JAR
+   * for ease of testing.
+   */
+  public static final String OP_MSYNC = "op_msync";
+
+  /**
+   * Preparing ancestor dirs: {@value}.
+   * Probing for paths being files, and if true: deleting them.
+   */
+  public static final String OP_PREPARE_DIR_ANCESTORS = "op_prepare_dir_ancestors";
+
+  /** Rename a dir: {@value}. */
+  public static final String OP_RENAME_DIR = OP_RENAME;
+
+
+  /** Rename a file: {@value}. */
+  public static final String OP_RENAME_FILE = OP_RENAME;
+
+  /**
+   * Save a task manifest: {@value}.
+   */
+  public static final String OP_SAVE_TASK_MANIFEST =
+      "task_stage_save_task_manifest";
+
+  /**
+   * Task abort: {@value}.
+   */
+  public static final String OP_STAGE_TASK_ABORT_TASK
+      = "task_stage_abort_task";
+
+  /**
+   * Job abort: {@value}.
+   */
+  public static final String OP_STAGE_JOB_ABORT = "job_stage_abort";
+
+  /**
+   * Job cleanup: {@value}.
+   */
+  public static final String OP_STAGE_JOB_CLEANUP = "job_stage_cleanup";
+
+  /**
+   * Prepare Directories Stage: {@value}.
+   */
+  public static final String OP_STAGE_JOB_CREATE_TARGET_DIRS =
+      "job_stage_create_target_dirs";
+
+  /**
+   * Load Manifest Stage: {@value}.
+   */
+  public static final String OP_STAGE_JOB_LOAD_MANIFESTS =
+      "job_stage_load_manifests";
+
+  /**
+   * Rename files stage duration: {@value}.
+   */
+  public static final String OP_STAGE_JOB_RENAME_FILES =
+      "job_stage_rename_files";
+
+
+  /**
+   * Job Setup Stage: {@value}.
+   */
+  public static final String OP_STAGE_JOB_SETUP = "job_stage_setup";
+
+  /**
+   * Job saving _SUCCESS marker Stage: {@value}.
+   */
+  public static final String OP_STAGE_JOB_SAVE_SUCCESS =
+      "job_stage_save_success_marker";
+
+  /**
+   * Output Validation (within job commit) Stage: {@value}.
+   */
+  public static final String OP_STAGE_JOB_VALIDATE_OUTPUT =
+      "job_stage_optional_validate_output";
+
+  /**
+   * Task saving manifest file Stage: {@value}.
+   */
+  public static final String OP_STAGE_TASK_SAVE_MANIFEST =
+      "task_stage_save_manifest";
+
+  /**
+   * Task Setup Stage: {@value}.
+   */
+  public static final String OP_STAGE_TASK_SETUP = "task_stage_setup";
+
+  /**
+   * Task Commit Stage: {@value}.
+   */
+  public static final String OP_STAGE_TASK_COMMIT = "task_stage_commit";
+
+  /** Task Scan directory Stage: {@value}. */
+  public static final String OP_STAGE_TASK_SCAN_DIRECTORY
+      = "task_stage_scan_directory";
+
+  private ManifestCommitterStatisticNames() {
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java
new file mode 100644
index 0000000..7020d5c
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java
@@ -0,0 +1,137 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Class for single/multiple commit data structures.
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public abstract class AbstractManifestData<T extends AbstractManifestData>
+    implements Serializable, IOStatisticsSource {
+
+
+  /**
+   * Convert a path to a string which can be included in the JSON.
+   * @param path path
+   * @return a string value, or, if path==null, null.
+   */
+  public static String marshallPath(@Nullable Path path) {
+    return path != null
+        ? path.toUri().toString()
+        : null;
+  }
+
+  /**
+   * Convert a string path to Path type, by way of a URI.
+   * @param path path as a string
+   * @return path value
+   * @throws RuntimeException marshalling failure.
+   */
+  public static Path unmarshallPath(String path) {
+    try {
+      return new Path(new URI(requireNonNull(path, "No path")));
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(
+          "Failed to parse \"" + path + "\" : " + e,
+          e);
+
+    }
+  }
+
+  /**
+   * Validate the data: those fields which must be non empty, must be set.
+   * @return the validated instance.
+   * @throws IOException if the data is invalid
+   */
+  public abstract T validate() throws IOException;
+
+  /**
+   * Serialize to JSON and then to a byte array, after performing a
+   * preflight validation of the data to be saved.
+   * @return the data in a persistable form.
+   * @throws IOException serialization problem or validation failure.
+   */
+  public abstract byte[] toBytes() throws IOException;
+
+  /**
+   * Save to a hadoop filesystem.
+   * @param fs filesystem
+   * @param path path
+   * @param overwrite should any existing file be overwritten
+   * @throws IOException IO exception
+   */
+  public abstract void save(FileSystem fs, Path path, boolean overwrite)
+      throws IOException;
+
+  /**
+   * Get a (usually shared) JSON serializer.
+   * @return a serializer. Call
+   */
+  public abstract JsonSerialization<T> createSerializer();
+
+  /**
+   * Verify that all instances in a collection are of the given class.
+   * @param it iterator
+   * @param classname classname to require
+   * @throws IOException on a failure
+   */
+  void validateCollectionClass(Iterable it, Class classname)
+      throws IOException {
+    for (Object o : it) {
+      verify(o.getClass().equals(classname),
+          "Collection element is not a %s: %s", classname, o.getClass());
+    }
+  }
+
+  /**
+   * Verify that a condition holds.
+   * @param expression expression which must be true
+   * @param message message to raise on a failure
+   * @param args arguments for the message formatting
+   * @throws IOException on a failure
+   */
+
+  static void verify(boolean expression,
+      String message,
+      Object... args) throws IOException {
+    if (!expression) {
+      throw new IOException(String.format(message, args));
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java
new file mode 100644
index 0000000..cb67338
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Diagnostic keys in the manifests.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class DiagnosticKeys {
+  /**
+   * Attribute added to diagnostics in _SUCCESS file.
+   */
+  public static final String PRINCIPAL = "principal";
+  public static final String STAGE = "stage";
+  public static final String EXCEPTION = "exception";
+  public static final String STACKTRACE = "stacktrace";
+
+
+  /** Directory where manifests were renamed: {@value}. */
+  public static final String MANIFESTS = "manifests";
+
+  private DiagnosticKeys() {
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java
new file mode 100644
index 0000000..15e8cac
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java
@@ -0,0 +1,202 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.verify;
+
+/**
+ * A directory entry in the task manifest.
+ * Uses shorter field names for smaller files.
+ * Hash and equals are on dir name only; there's no real expectation
+ * that those operations are needed.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class DirEntry implements Serializable {
+
+  private static final long serialVersionUID = 5658520530209859765L;
+
+  /**
+   * Destination directory.
+   */
+  @JsonProperty("d")
+  private String dir;
+
+  /**
+   * Type of dest entry as found when probed for in task commit.
+   */
+  @JsonProperty("t")
+  private int type;
+
+  /**
+   * Level in the treewalk.
+   */
+  @JsonProperty("l")
+  private int level;
+
+  /**
+   * Constructor only for use by jackson.
+   * Do Not Delete.
+   */
+  private DirEntry() {
+  }
+
+  /**
+   * Construct an entry.
+   *
+   * @param dir destination path.
+   * @param type type of dest entry
+   * @param level Level in the treewalk.
+   *
+   */
+  public DirEntry(
+      final String dir,
+      final int type,
+      final int level) {
+    this.dir = requireNonNull(dir);
+    this.type = type;
+    this.level = level;
+  }
+
+  /**
+   * Construct an entry.
+   *
+   * @param dir destination path.
+   * @param type type of dest entry
+   * @param level Level in the treewalk.
+   *
+   */
+  public DirEntry(
+      final Path dir,
+      final int type,
+      final int level) {
+    this(marshallPath(dir), type, level);
+  }
+
+  public void setDir(final String dir) {
+    this.dir = dir;
+  }
+
+  public String getDir() {
+    return dir;
+  }
+
+  @JsonIgnore
+  public Path getDestPath() {
+    return unmarshallPath(dir);
+  }
+
+  public int getType() {
+    return type;
+  }
+
+  public void setType(final int type) {
+    this.type = type;
+  }
+
+  public void setLevel(final int level) {
+    this.level = level;
+  }
+
+  public int getLevel() {
+    return level;
+  }
+
+  @JsonIgnore
+  public EntryStatus getStatus() {
+    return EntryStatus.toEntryStatus(type);
+  }
+
+  @JsonIgnore
+  public void setStatus(EntryStatus status) {
+    setType(status.ordinal());
+  }
+  public void validate() throws IOException {
+    final String s = toString();
+    verify(dir != null && dir.length() > 0,
+        "destination path is missing from " + s);
+    verify(type >= 0,
+        "Invalid type in " + s);
+    verify(level >= 0,
+        "Invalid level in " + s);
+  }
+
+  @Override
+  public String toString() {
+    return "DirEntry{" +
+        "dir='" + dir + '\'' +
+        ", type=" + type +
+        ", level=" + level +
+        '}';
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    DirEntry dirEntry = (DirEntry) o;
+    return dir.equals(dirEntry.dir);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(dir);
+  }
+
+  /**
+   * A directory entry.
+   * @param dest destination path.
+   * @param type type
+   * @param level Level in the treewalk.
+   * @return an entry
+   */
+  public static DirEntry dirEntry(Path dest, int type, int level) {
+    return new DirEntry(dest, type, level);
+  }
+
+  /**
+   * A directory entry.
+   * @param dest destination path.
+   * @param type type
+   * @param level Level in the treewalk.
+   * @return an entry
+   */
+  public static DirEntry dirEntry(Path dest, EntryStatus type, int level) {
+    return dirEntry(dest, type.ordinal(), level);
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java
new file mode 100644
index 0000000..73ac5d7
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java
@@ -0,0 +1,87 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * Status of a file or dir entry, designed to be marshalled as
+ * an integer -the ordinal value of the enum is the
+ * wire value.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public enum EntryStatus {
+
+  unknown,
+  not_found,
+  file,
+  dir,
+  created_dir;
+
+  /**
+   * Go from a marshalled type to a status value.
+   * Any out of range value is converted to unknown.
+   * @param type type
+   * @return the status value.
+   */
+  public static EntryStatus toEntryStatus(int type) {
+    switch (type) {
+    case 1:
+      return not_found;
+    case 2:
+      return file;
+    case 3:
+      return dir;
+    case 4:
+      return created_dir;
+    case 0:
+    default:
+      return unknown;
+    }
+  }
+
+
+  /**
+   * Go from the result of a getFileStatus call or
+   * listing entry to a status.
+   * A null argument is mapped to {@link #not_found}
+   * @param st file status
+   * @return the status enum.
+   */
+  public static EntryStatus toEntryStatus(@Nullable FileStatus st) {
+
+    if (st == null) {
+      return not_found;
+    }
+    if (st.isDirectory()) {
+      return dir;
+    }
+    if (st.isFile()) {
+      return file;
+    }
+    return unknown;
+  }
+
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java
new file mode 100644
index 0000000..bc6cdd9
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java
@@ -0,0 +1,189 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.verify;
+
+/**
+ * A File entry in the task manifest.
+ * Uses shorter field names for smaller files.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public final class FileEntry implements Serializable {
+
+  private static final long serialVersionUID = -550288489009777867L;
+
+  @JsonProperty("s")
+  private String source;
+
+  @JsonProperty("d")
+  private String dest;
+
+  @JsonProperty("z")
+  private long size;
+
+  /**
+   * Etag value if we can extract this.
+   */
+  @JsonProperty("e")
+  private String etag;
+
+  /**
+   * Constructor only for use by jackson.
+   * Do Not Delete.
+   */
+  private FileEntry() {
+  }
+
+  /**
+   * Construct an entry.
+   * @param source source path.
+   * @param dest destination path.
+   * @param size file size.
+   * @param etag optional etag
+   */
+  public FileEntry(
+      final String source,
+      final String dest,
+      final long size,
+      final String etag) {
+    this.source = source;
+    this.dest = dest;
+    this.size = size;
+    this.etag = etag;
+  }
+
+
+  /**
+   * Construct an entry.
+   * @param source source path.
+   * @param dest destination path.
+   * @param size file size.
+   * @param etag optional etag
+   */
+  public FileEntry(
+      final Path source,
+      final Path dest,
+      final long size,
+      final String etag) {
+    this(marshallPath(source), marshallPath(dest), size, etag);
+  }
+
+
+  public void setSource(final String source) {
+    this.source = source;
+  }
+
+  public String getSource() {
+    return source;
+  }
+
+  @JsonIgnore
+  public Path getSourcePath() {
+    return unmarshallPath(source);
+  }
+
+  public void setDest(final String dest) {
+    this.dest = dest;
+  }
+
+  public String getDest() {
+    return dest;
+  }
+
+  @JsonIgnore
+  public Path getDestPath() {
+    return unmarshallPath(dest);
+  }
+
+  public long getSize() {
+    return size;
+  }
+
+  public void setSize(final long size) {
+    this.size = size;
+  }
+
+  public String getEtag() {
+    return etag;
+  }
+
+  public void setEtag(final String etag) {
+    this.etag = etag;
+  }
+
+  public void validate() throws IOException {
+    final String s = toString();
+    verify(source != null && source.length() > 0,
+        "Source is missing from " + s);
+    verify(dest != null && dest.length() > 0,
+        "Source is missing from " + s);
+    verify(size >= 0,
+        "Invalid size in " + s);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "FileOrDirEntry{");
+    sb.append("source='").append(source).append('\'');
+    sb.append(", dest='").append(dest).append('\'');
+    sb.append(", size=").append(size);
+    sb.append(", etag='").append(etag).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    FileEntry that = (FileEntry) o;
+    return size == that.size && source.equals(that.source) && dest.equals(
+        that.dest) &&
+        Objects.equals(etag, that.etag);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(source, dest);
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java
new file mode 100644
index 0000000..c95ec7b
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java
@@ -0,0 +1,144 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
+
+/**
+ * Tool to print a manifest.
+ */
+public class ManifestPrinter extends Configured implements Tool {
+
+  private static final String USAGE = "ManifestPrinter <success-file>";
+
+  /**
+   * Output for printing.
+   */
+  private final PrintStream out;
+
+  /**
+   * Print to System.out.
+   */
+  public ManifestPrinter() {
+    this(null, System.out);
+  }
+
+  /**
+   * Print to the supplied stream.
+   * @param conf configuration
+   * @param out output
+   */
+  public ManifestPrinter(Configuration conf, PrintStream out) {
+    super(conf);
+    this.out = out;
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length != 1) {
+      printUsage();
+      return -1;
+    }
+    Path path = new Path(args[0]);
+    loadAndPrintManifest(path.getFileSystem(getConf()), path);
+    return 0;
+  }
+
+  /**
+   * Load and print a manifest.
+   * @param fs filesystem.
+   * @param path path
+   * @throws IOException failure to load
+   * @return the manifest
+   */
+  public ManifestSuccessData loadAndPrintManifest(FileSystem fs, Path path)
+      throws IOException {
+    // load the manifest
+    println("Manifest file: %s", path);
+    final ManifestSuccessData success = ManifestSuccessData.load(fs, path);
+
+    printManifest(success);
+    return success;
+  }
+
+  private void printManifest(ManifestSuccessData success) {
+    field("succeeded", success.getSuccess());
+    field("created", success.getDate());
+    field("committer", success.getCommitter());
+    field("hostname", success.getHostname());
+    field("description", success.getDescription());
+    field("jobId", success.getJobId());
+    field("jobIdSource", success.getJobIdSource());
+    field("stage", success.getStage());
+    println("Diagnostics\n%s",
+        success.dumpDiagnostics("  ", " = ", "\n"));
+    println("Statistics:\n%s",
+        ioStatisticsToPrettyString(success.getIOStatistics()));
+    out.flush();
+  }
+
+  private void printUsage() {
+    println(USAGE);
+  }
+
+  /**
+   * Print a line to the output stream.
+   * @param format format string
+   * @param args arguments.
+   */
+  private void println(String format, Object... args) {
+    out.format(format, args);
+    out.println();
+  }
+
+  /**
+   * Print a field, if non-null.
+   * @param name field name.
+   * @param value value.
+   */
+  private void field(String name, Object value) {
+    if (value != null) {
+      println("%s: %s", name, value);
+    }
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] argv) throws Exception {
+
+    try {
+      int res = ToolRunner.run(new ManifestPrinter(), argv);
+      System.exit(res);
+    } catch (ExitUtil.ExitException e) {
+      ExitUtil.terminate(e);
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java
new file mode 100644
index 0000000..73e7307
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java
@@ -0,0 +1,493 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
+import org.apache.hadoop.util.JsonSerialization;
+
+/**
+ * Summary data saved into a {@code _SUCCESS} marker file.
+ *
+ * This is a copy of the S3A committer success data format, with
+ * a goal of being/remaining compatible.
+ * This makes it easier for tests in downstream modules to
+ * be able to parse the success files from any of the committers.
+ *
+ * This should be considered public; it is based on the S3A
+ * format, which has proven stable over time.
+ *
+ * The JSON format SHOULD be considered public and evolving
+ * with compatibility across versions.
+ *
+ * All the Java serialization data is different and may change
+ * across versions with no stability guarantees other than
+ * "manifest summaries MAY be serialized between processes with
+ * the exact same version of this binary on their classpaths."
+ * That is sufficient for testing in Spark.
+ *
+ * To aid with Java serialization, the maps and lists are
+ * exclusively those which serialize well.
+ * IOStatisticsSnapshot has a lot of complexity in marshalling
+ * there; this class doesn't worry about concurrent access
+ * so is simpler.
+ *
+ */
+@SuppressWarnings({"unused", "CollectionDeclaredAsConcreteClass"})
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ManifestSuccessData
+    extends AbstractManifestData<ManifestSuccessData> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ManifestSuccessData.class);
+
+  /**
+   * Supported version value: {@value}.
+   * If this is changed the value of {@link #serialVersionUID} will change,
+   * to avoid deserialization problems.
+   */
+  public static final int VERSION = 1;
+
+  /**
+   * Serialization ID: {@value}.
+   */
+  private static final long serialVersionUID = 4755993198698104084L + VERSION;
+
+  /**
+   * Name to include in persisted data, so as to differentiate from
+   * any other manifests: {@value}.
+   */
+  public static final String NAME
+      = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION;
+
+  /**
+   * Name of file; includes version marker.
+   */
+  private String name;
+
+  /** Timestamp of creation. */
+  private long timestamp;
+
+  /**
+   * Did this succeed?
+   * It is implicitly true in a _SUCCESS file, but if the file
+   * is also saved to a log dir, then it depends on the outcome
+   */
+  private boolean success = true;
+
+  /** Timestamp as date string; no expectation of parseability. */
+  private String date;
+
+  /**
+   * Host which created the file (implicitly: committed the work).
+   */
+  private String hostname;
+
+  /**
+   * Committer name.
+   */
+  private String committer;
+
+  /**
+   * Description text.
+   */
+  private String description;
+
+  /** Job ID, if known. */
+  private String jobId = "";
+
+  /**
+   * Source of the job ID.
+   */
+  private String jobIdSource = "";
+
+  /**
+   * Metrics.
+   * Uses a treemap for serialization.
+   */
+  private TreeMap<String, Long> metrics = new TreeMap<>();
+
+  /**
+   * Diagnostics information.
+   * Uses a treemap for serialization.
+   */
+  private TreeMap<String, String> diagnostics = new TreeMap<>();
+
+  /**
+   * Filenames in the commit.
+   */
+  private ArrayList<String> filenames = new ArrayList<>(0);
+
+  /**
+   * IOStatistics.
+   */
+  @JsonProperty("iostatistics")
+  private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot();
+
+  /**
+   * State (committed, aborted).
+   */
+  private String state;
+
+  /**
+   * Stage: last stage executed.
+   */
+  private String stage;
+
+  @Override
+  public ManifestSuccessData validate() throws IOException {
+    verify(name != null,
+        "Incompatible file format: no 'name' field");
+    verify(NAME.equals(name),
+        "Incompatible file format: " + name);
+    return this;
+  }
+
+  @Override
+  public JsonSerialization<ManifestSuccessData> createSerializer() {
+    return serializer();
+  }
+
+  @Override
+  public byte[] toBytes() throws IOException {
+    return serializer().toBytes(this);
+  }
+
+  /**
+   * To JSON.
+   * @return json string value.
+   * @throws IOException failure
+   */
+  public String toJson() throws IOException {
+    return serializer().toJson(this);
+  }
+
+  @Override
+  public void save(FileSystem fs, Path path, boolean overwrite)
+      throws IOException {
+    // always set the name field before being saved.
+    name = NAME;
+    serializer().save(fs, path, this, overwrite);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "ManifestSuccessData{");
+    sb.append("committer='").append(committer).append('\'');
+    sb.append(", hostname='").append(hostname).append('\'');
+    sb.append(", description='").append(description).append('\'');
+    sb.append(", date='").append(date).append('\'');
+    sb.append(", filenames=[").append(
+        StringUtils.join(filenames, ", "))
+        .append("]");
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Dump the metrics (if any) to a string.
+   * The metrics are sorted for ease of viewing.
+   * @param prefix prefix before every entry
+   * @param middle string between key and value
+   * @param suffix suffix to each entry
+   * @return the dumped string
+   */
+  public String dumpMetrics(String prefix, String middle, String suffix) {
+    return joinMap(metrics, prefix, middle, suffix);
+  }
+
+  /**
+   * Dump the diagnostics (if any) to a string.
+   * @param prefix prefix before every entry
+   * @param middle string between key and value
+   * @param suffix suffix to each entry
+   * @return the dumped string
+   */
+  public String dumpDiagnostics(String prefix, String middle, String suffix) {
+    return joinMap(diagnostics, prefix, middle, suffix);
+  }
+
+  /**
+   * Join any map of string to value into a string, sorting the keys first.
+   * @param map map to join
+   * @param prefix prefix before every entry
+   * @param middle string between key and value
+   * @param suffix suffix to each entry
+   * @return a string for reporting.
+   */
+  protected static String joinMap(Map<String, ?> map,
+      String prefix,
+      String middle, String suffix) {
+    if (map == null) {
+      return "";
+    }
+    List<String> list = new ArrayList<>(map.keySet());
+    Collections.sort(list);
+    StringBuilder sb = new StringBuilder(list.size() * 32);
+    for (String k : list) {
+      sb.append(prefix)
+          .append(k)
+          .append(middle)
+          .append(map.get(k))
+          .append(suffix);
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Load an instance from a file, then validate it.
+   * @param fs filesystem
+   * @param path path
+   * @return the loaded instance
+   * @throws IOException IO failure
+   */
+  public static ManifestSuccessData load(FileSystem fs, Path path)
+      throws IOException {
+    LOG.debug("Reading success data from {}", path);
+    ManifestSuccessData instance = serializer().load(fs, path);
+    instance.validate();
+    return instance;
+  }
+
+  /**
+   * Get a JSON serializer for this class.
+   * @return a serializer.
+   */
+  public static JsonSerialization<ManifestSuccessData> serializer() {
+    return new JsonSerialization<>(ManifestSuccessData.class, false, true);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /** @return timestamp of creation. */
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  /** @return timestamp as date; no expectation of parseability. */
+  public String getDate() {
+    return date;
+  }
+
+  public void setDate(String date) {
+    this.date = date;
+  }
+
+  /**
+   * @return host which created the file (implicitly: committed the work).
+   */
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  /**
+   * @return committer name.
+   */
+  public String getCommitter() {
+    return committer;
+  }
+
+  public void setCommitter(String committer) {
+    this.committer = committer;
+  }
+
+  /**
+   * @return any description text.
+   */
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  /**
+   * @return any metrics.
+   */
+  public Map<String, Long> getMetrics() {
+    return metrics;
+  }
+
+  public void setMetrics(TreeMap<String, Long> metrics) {
+    this.metrics = metrics;
+  }
+
+  /**
+   * @return a list of filenames in the commit.
+   */
+  public List<String> getFilenames() {
+    return filenames;
+  }
+
+  /**
+   * Get the list of filenames as paths.
+   * @return the paths.
+   */
+  @JsonIgnore
+  public List<Path> getFilenamePaths() {
+    return getFilenames().stream()
+        .map(AbstractManifestData::unmarshallPath)
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Set the list of filename paths.
+   */
+  @JsonIgnore
+  public void setFilenamePaths(List<Path> paths) {
+    setFilenames(new ArrayList<>(
+        paths.stream()
+            .map(AbstractManifestData::marshallPath)
+            .collect(Collectors.toList())));
+  }
+
+  public void setFilenames(ArrayList<String> filenames) {
+    this.filenames = filenames;
+  }
+
+  public Map<String, String> getDiagnostics() {
+    return diagnostics;
+  }
+
+  public void setDiagnostics(TreeMap<String, String> diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  /**
+   * Add a diagnostics entry.
+   * @param key name
+   * @param value value
+   */
+  public void putDiagnostic(String key, String value) {
+    diagnostics.put(key, value);
+  }
+
+  /** @return Job ID, if known. */
+  public String getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(String jobId) {
+    this.jobId = jobId;
+  }
+
+  public String getJobIdSource() {
+    return jobIdSource;
+  }
+
+  public void setJobIdSource(final String jobIdSource) {
+    this.jobIdSource = jobIdSource;
+  }
+
+  @Override
+  public IOStatisticsSnapshot getIOStatistics() {
+    return iostatistics;
+  }
+
+  public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
+    this.iostatistics = ioStatistics;
+  }
+
+  /**
+   * Set the IOStatistics to a snapshot of the source.
+   * @param iostats. Statistics; may be null.
+   */
+  public void snapshotIOStatistics(IOStatistics iostats) {
+    setIOStatistics(IOStatisticsSupport.snapshotIOStatistics(iostats));
+  }
+
+  /**
+   * Set the success flag.
+   * @param success did the job succeed?
+   */
+  public void setSuccess(boolean success) {
+    this.success = success;
+  }
+
+  /**
+   * Get the success flag.
+   * @return did the job succeed?
+   */
+  public boolean getSuccess() {
+    return success;
+  }
+
+  public String getState() {
+    return state;
+  }
+
+  public void setState(String state) {
+    this.state = state;
+  }
+
+  public String getStage() {
+    return stage;
+  }
+
+  /**
+   * Note a failure by setting success flag to false,
+   * then add the exception to the diagnostics.
+   * @param thrown throwable
+   */
+  public void recordJobFailure(Throwable thrown) {
+    setSuccess(false);
+    String stacktrace = ExceptionUtils.getStackTrace(thrown);
+    diagnostics.put(DiagnosticKeys.EXCEPTION, thrown.toString());
+    diagnostics.put(DiagnosticKeys.STACKTRACE, stacktrace);
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java
new file mode 100644
index 0000000..a06b837
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java
@@ -0,0 +1,365 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.util.JsonSerialization;
+
+/**
+ * This is the manifest of files which were created by
+ * this task attempt.
+ *
+ * Versioning:
+ * In a rolling cluster update, MR or Spark workers deployed on a newer
+ * node (and/or with a newer version of artifacts in a cluster-FS hosted
+ * tar.gz file) may be a later version of this class than that of
+ * job committer.
+ * If any changes are made to the manifest which are backwards compatible,
+ * this new manifest can still be loaded from JSON and processed.
+ *
+ * If the manifest is no longer compatible, the job output may
+ * be invalid.
+ *
+ * It is CRITICAL that the {@link #VERSION} constant is updated whenever
+ * such an incompatible change is made.
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class TaskManifest extends AbstractManifestData<TaskManifest> {
+
+  /**
+   * Supported version value: {@value}.
+   * If this is changed the value of {@code serialVersionUID} will change,
+   * to avoid deserialization problems.
+   */
+  public static final int VERSION = 1;
+
+  /**
+   * Manifest type.
+   */
+  public static final String TYPE =
+      "org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest/"
+      + VERSION;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TaskManifest.class);
+
+  /**
+   * Serialization version.
+   */
+  private static final long serialVersionUID = 7090285511966046094L + VERSION;
+
+  /**
+   * Manifest type.
+   */
+  @JsonProperty("type")
+  private String type = TYPE;
+
+  /** Version marker. */
+  @JsonProperty("version")
+  private int version = VERSION;
+
+  /**
+   * Job ID; constant over multiple attempts.
+   */
+  @JsonProperty("jobId")
+  private String jobId;
+
+  /**
+   * Number of the job attempt; starts at zero.
+   */
+  @JsonProperty("jobAttemptNumber")
+  private int jobAttemptNumber;
+
+  /**
+   * Task Attempt ID.
+   */
+  @JsonProperty("taskID")
+  private String taskID;
+
+  /**
+   * Task Attempt ID.
+   */
+  @JsonProperty("taskAttemptID")
+  private String taskAttemptID;
+
+  /**
+   * The task attempt directory.
+   */
+  @JsonProperty("taskAttemptDir")
+  private String taskAttemptDir;
+
+  /**
+   * The list of files to commit from this task attempt, including
+   * precalculated destination and size.
+   */
+  @JsonProperty("files")
+  private final List<FileEntry> filesToCommit = new ArrayList<>();
+
+  /**
+   * The list of directories needed by this task attempt, both
+   * source and destination.
+   * All these directories must exist in the destination before any of
+   * the files can be renamed there.
+   */
+  @JsonProperty("directories")
+  private final List<DirEntry> destDirectories = new ArrayList<>();
+
+  /**
+   * Any custom extra data committers may choose to add.
+   */
+  private final Map<String, String> extraData = new HashMap<>(0);
+
+  /**
+   * IOStatistics.
+   */
+  @JsonProperty("iostatistics")
+  private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot();
+
+  /**
+   * Empty constructor; will be used by jackson as well as in application
+   * code.
+   */
+  public TaskManifest() {
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  @Override
+  public IOStatisticsSnapshot getIOStatistics() {
+    return iostatistics;
+  }
+
+  public void setIOStatistics(
+      @Nullable final IOStatisticsSnapshot ioStatistics) {
+    this.iostatistics = ioStatistics;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(final String jobId) {
+    this.jobId = jobId;
+  }
+
+  public int getJobAttemptNumber() {
+    return jobAttemptNumber;
+  }
+
+  public void setJobAttemptNumber(final int jobAttemptNumber) {
+    this.jobAttemptNumber = jobAttemptNumber;
+  }
+
+  public String getTaskID() {
+    return taskID;
+  }
+
+  public void setTaskID(final String taskID) {
+    this.taskID = taskID;
+  }
+
+  public String getTaskAttemptID() {
+    return taskAttemptID;
+  }
+
+  public void setTaskAttemptID(final String taskAttemptID) {
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  public String getTaskAttemptDir() {
+    return taskAttemptDir;
+  }
+
+  public void setTaskAttemptDir(final String taskAttemptDir) {
+    this.taskAttemptDir = taskAttemptDir;
+  }
+
+  /**
+   * Add a file to the list of files to commit.
+   * @param entry entry  to add
+   */
+  public void addFileToCommit(FileEntry entry) {
+    filesToCommit.add(entry);
+  }
+
+  public List<FileEntry> getFilesToCommit() {
+    return filesToCommit;
+  }
+
+  /**
+   * Calculate the total amount of data which will be committed.
+   * @return the sum of sizes of all files to commit.
+   */
+  @JsonIgnore
+  public long getTotalFileSize() {
+    return filesToCommit.stream().mapToLong(FileEntry::getSize).sum();
+  }
+
+  /**
+   * All the destination directories.
+   * @return directory list.
+   */
+  public List<DirEntry> getDestDirectories() {
+    return destDirectories;
+  }
+
+  /**
+   * Add a directory to the list of directories to create.
+   * @param entry entry  to add
+   */
+  public void addDirectory(DirEntry entry) {
+    destDirectories.add(entry);
+  }
+
+  public Map<String, String> getExtraData() {
+    return extraData;
+  }
+
+  @Override
+  public byte[] toBytes() throws IOException {
+    return serializer().toBytes(this);
+  }
+
+  /**
+   * To JSON.
+   * @return json string value.
+   * @throws IOException failure
+   */
+  public String toJson() throws IOException {
+    return serializer().toJson(this);
+  }
+
+  @Override
+  public void save(FileSystem fs, Path path, boolean overwrite)
+      throws IOException {
+    serializer().save(fs, path, this, overwrite);
+  }
+
+  /**
+   * Validate the data: those fields which must be non empty, must be set.
+   * @throws IOException if the data is invalid
+   * @return
+   */
+  public TaskManifest validate() throws IOException {
+    verify(TYPE.equals(type), "Wrong type: %s", type);
+    verify(version == VERSION, "Wrong version: %s", version);
+    validateCollectionClass(extraData.keySet(), String.class);
+    validateCollectionClass(extraData.values(), String.class);
+    Set<String> destinations = new HashSet<>(filesToCommit.size());
+    validateCollectionClass(filesToCommit, FileEntry.class);
+    for (FileEntry c : filesToCommit) {
+      c.validate();
+      verify(!destinations.contains(c.getDest()),
+          "Destination %s is written to by more than one pending commit",
+          c.getDest());
+      destinations.add(c.getDest());
+    }
+    return this;
+  }
+
+  /**
+   * Get a JSON serializer for this class.
+   * @return a serializer.
+   */
+  @Override
+  public JsonSerialization<TaskManifest> createSerializer() {
+    return serializer();
+  }
+
+  /**
+   * Create a JSON serializer for this class.
+   * @return a serializer.
+   */
+  public static JsonSerialization<TaskManifest> serializer() {
+    return new JsonSerialization<>(TaskManifest.class, false, true);
+  }
+
+  /**
+   * Load an instance from a file, then validate it.
+   * @param fs filesystem
+   * @param path path
+   * @return the loaded instance
+   * @throws IOException IO failure/the data is invalid
+   */
+  public static TaskManifest load(FileSystem fs, Path path)
+      throws IOException {
+    LOG.debug("Reading Manifest in file {}", path);
+    return serializer().load(fs, path).validate();
+  }
+
+  /**
+   * Load an instance from a file, then validate it.
+   * If loading through a listing; use this API so that filestatus
+   * hints can be used.
+   * @param serializer serializer.
+   * @param fs filesystem
+   * @param path path to load from
+   * @param status status of file to load
+   * @return the loaded instance
+   * @throws IOException IO failure/the data is invalid
+   */
+  public static TaskManifest load(
+      JsonSerialization<TaskManifest> serializer,
+      FileSystem fs,
+      Path path,
+      FileStatus status)
+      throws IOException {
+    LOG.debug("Reading Manifest in file {}", path);
+    return serializer.load(fs, path, status)
+        .validate();
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java
new file mode 100644
index 0000000..e1fbb4a
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/**
+ * Persistence formats.
+ * These are the persistence formats used for passing data from tasks
+ * to the job committer
+ * {@link org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest},
+ * and for a {@code _SUCCESS} file, which is in
+ * {@link org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData}.
+ * The {@code _SUCCESS} file is a copy of the S3A Committer
+ * {@code org.apache.hadoop.fs.s3a.commit.files.ManifestSuccessData},
+ * the intent being that at the JSON-level they are compatible.
+ * This is to aid testing/validation and support calls, with one single
+ * format to load.
+ *
+ * Consult the individual formats for their declarations of access;
+ * the _SUCCESS file is one which tests may use.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.mapreduce.lib.output.committer.manifest.files;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java
new file mode 100644
index 0000000..de67a2a
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java
@@ -0,0 +1,94 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConfig;
+
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_JOB_ID;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.CONTEXT_ATTR_STAGE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.CONTEXT_ATTR_TASK_ATTEMPT_ID;
+
+/**
+ * Helper class to support integration with Hadoop 3.3.2+ Auditing.
+ * This MUST BE the sole place where fs.audit methods are used, so can be replaced
+ * by a stub class on any backport.
+ */
+@InterfaceAudience.Private
+public final class AuditingIntegration {
+  private AuditingIntegration() {
+  }
+
+  /**
+   * Add jobID to current context; also
+   * task attempt ID if set.
+   */
+  public static void updateCommonContextOnCommitterEntry(
+      ManifestCommitterConfig committerConfig) {
+    CommonAuditContext context = currentAuditContext();
+    context.put(PARAM_JOB_ID,
+        committerConfig.getJobUniqueId());
+    // maybe the task attempt ID.
+    if (!committerConfig.getTaskAttemptId().isEmpty()) {
+      context.put(CONTEXT_ATTR_TASK_ATTEMPT_ID,
+          committerConfig.getTaskAttemptId());
+    }
+  }
+
+  /**
+   * Callback on stage entry.
+   * Sets the activeStage and updates the
+   * common context.
+   * @param stage new stage
+   */
+  public static void enterStage(String stage) {
+    currentAuditContext().put(CONTEXT_ATTR_STAGE, stage);
+  }
+
+  /**
+   * Remove stage from common audit context.
+   */
+  public static void exitStage() {
+    currentAuditContext().remove(CONTEXT_ATTR_STAGE);
+  }
+
+  /**
+   * Remove commit info at the end of the task or job.
+   */
+  public static void updateCommonContextOnCommitterExit() {
+    currentAuditContext().remove(PARAM_JOB_ID);
+    currentAuditContext().remove(CONTEXT_ATTR_TASK_ATTEMPT_ID);
+  }
+
+  /**
+   * Update the thread context with the stage name and
+   * job ID.
+   * This MUST be invoked at the start of methods invoked in helper threads,
+   * to ensure that they are all annotated with job and stage.
+   * @param jobId job ID.
+   * @param stage stage name.
+   */
+  public static void enterStageWorker(String jobId, String stage) {
+    CommonAuditContext context = currentAuditContext();
+    context.put(PARAM_JOB_ID, jobId);
+    context.put(CONTEXT_ATTR_STAGE, stage);
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java
new file mode 100644
index 0000000..15f9899
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java
@@ -0,0 +1,130 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import java.util.Set;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_GET_FILE_STATUS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_DIRECTORY;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_FILE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_LIST_STATUS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_MKDIRS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.*;
+
+/**
+ * Constants internal to the manifest committer.
+ */
+@InterfaceAudience.Private
+public final class InternalConstants {
+  private InternalConstants() {
+  }
+
+  /**
+   * Durations.
+   */
+  public static final String[] DURATION_STATISTICS = {
+
+      /* Job stages. */
+      OP_STAGE_JOB_ABORT,
+      OP_STAGE_JOB_CLEANUP,
+      OP_STAGE_JOB_COMMIT,
+      OP_STAGE_JOB_CREATE_TARGET_DIRS,
+      OP_STAGE_JOB_LOAD_MANIFESTS,
+      OP_STAGE_JOB_RENAME_FILES,
+      OP_STAGE_JOB_SAVE_SUCCESS,
+      OP_STAGE_JOB_SETUP,
+      OP_STAGE_JOB_VALIDATE_OUTPUT,
+
+      /* Task stages. */
+
+      OP_STAGE_TASK_ABORT_TASK,
+      OP_STAGE_TASK_COMMIT,
+      OP_STAGE_TASK_SAVE_MANIFEST,
+      OP_STAGE_TASK_SCAN_DIRECTORY,
+      OP_STAGE_TASK_SETUP,
+
+      /* Lower level store/fs operations. */
+      OP_COMMIT_FILE_RENAME,
+      OP_CREATE_DIRECTORIES,
+      OP_CREATE_ONE_DIRECTORY,
+      OP_DIRECTORY_SCAN,
+      OP_DELETE,
+      OP_DELETE_FILE_UNDER_DESTINATION,
+      OP_GET_FILE_STATUS,
+      OP_IS_DIRECTORY,
+      OP_IS_FILE,
+      OP_LIST_STATUS,
+      OP_LOAD_MANIFEST,
+      OP_LOAD_ALL_MANIFESTS,
+      OP_MKDIRS,
+      OP_MKDIRS_RETURNED_FALSE,
+      OP_MSYNC,
+      OP_PREPARE_DIR_ANCESTORS,
+      OP_RENAME_FILE,
+      OP_SAVE_TASK_MANIFEST,
+
+      OBJECT_LIST_REQUEST,
+      OBJECT_CONTINUE_LIST_REQUEST,
+
+      STORE_IO_RATE_LIMITED
+  };
+
+  /**
+   * Counters.
+   */
+  public static final String[] COUNTER_STATISTICS = {
+      COMMITTER_BYTES_COMMITTED_COUNT,
+      COMMITTER_FILES_COMMITTED_COUNT,
+      COMMITTER_TASKS_COMPLETED_COUNT,
+      COMMITTER_TASKS_FAILED_COUNT,
+      COMMITTER_TASK_DIRECTORY_COUNT_MEAN,
+      COMMITTER_TASK_DIRECTORY_DEPTH_MEAN,
+      COMMITTER_TASK_FILE_COUNT_MEAN,
+      COMMITTER_TASK_FILE_SIZE_MEAN,
+      COMMITTER_TASK_MANIFEST_FILE_SIZE,
+      OP_COMMIT_FILE_RENAME_RECOVERED,
+  };
+
+  /**
+   * Error string from ABFS connector on timeout.
+   */
+  public static final String OPERATION_TIMED_OUT = "OperationTimedOut";
+
+  /**
+   * Format string for task attempt names.
+   */
+  public static final String NAME_FORMAT_TASK_ATTEMPT = "[Task-Attempt %s]";
+
+  /**
+   * Format string for job attempt names.
+   */
+  public static final String NAME_FORMAT_JOB_ATTEMPT = "[Job-Attempt %s]";
+
+  /** Schemas of filesystems we know to not work with this committer. */
+  public static final Set<String> UNSUPPORTED_FS_SCHEMAS =
+      ImmutableSet.of("s3a", "wasb");
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java
new file mode 100644
index 0000000..f6edde5
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java
@@ -0,0 +1,374 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import java.io.IOException;
+import java.time.ZonedDateTime;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.EtagSource;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.PENDING_DIR_NAME;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.INITIAL_APP_ATTEMPT_ID;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ATTEMPT_DIR_FORMAT_STR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_DIR_FORMAT_STR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_ATTEMPT_SUBDIR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_MANIFEST_SUBDIR;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUMMARY_FILENAME_FORMAT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.COUNTER_STATISTICS;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.DURATION_STATISTICS;
+
+/**
+ * Class for manifest committer support util methods.
+ */
+
+@InterfaceAudience.Private
+public final class ManifestCommitterSupport {
+
+  private ManifestCommitterSupport() {
+  }
+
+  /**
+   * Create an IOStatistics Store with the standard statistics
+   * set up.
+   * @return a store builder preconfigured with the standard stats.
+   */
+  public static IOStatisticsStoreBuilder createIOStatisticsStore() {
+
+    final IOStatisticsStoreBuilder store
+        = iostatisticsStore();
+
+    store.withCounters(COUNTER_STATISTICS);
+    store.withMaximums(COUNTER_STATISTICS);
+    store.withMinimums(COUNTER_STATISTICS);
+    store.withMeanStatistics(COUNTER_STATISTICS);
+    store.withDurationTracking(DURATION_STATISTICS);
+    return store;
+  }
+
+  /**
+   * If the object is an IOStatisticsSource, get and add
+   * its IOStatistics.
+   * @param o source object.
+   */
+  public static void maybeAddIOStatistics(IOStatisticsAggregator ios,
+      Object o) {
+    if (o instanceof IOStatisticsSource) {
+      ios.aggregate(((IOStatisticsSource) o).getIOStatistics());
+    }
+  }
+
+  /**
+   * Build a Job UUID from the job conf (if it is
+   * {@link ManifestCommitterConstants#SPARK_WRITE_UUID}
+   * or the MR job ID.
+   * @param conf job/task configuration
+   * @param jobId job ID from YARN or spark.
+   * @return (a job ID, source)
+   */
+  public static Pair<String, String> buildJobUUID(Configuration conf,
+      JobID jobId) {
+    String jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, "");
+    if (jobUUID.isEmpty()) {
+      jobUUID = jobId.toString();
+      return Pair.of(jobUUID, JOB_ID_SOURCE_MAPREDUCE);
+    } else {
+      return Pair.of(jobUUID, SPARK_WRITE_UUID);
+    }
+  }
+
+  /**
+   * Get the location of pending job attempts.
+   * @param out the base output directory.
+   * @return the location of pending job attempts.
+   */
+  public static Path getPendingJobAttemptsPath(Path out) {
+    return new Path(out, PENDING_DIR_NAME);
+  }
+
+  /**
+   * Get the Application Attempt Id for this job.
+   * @param context the context to look in
+   * @return the Application Attempt Id for a given job.
+   */
+  public static int getAppAttemptId(JobContext context) {
+    return getAppAttemptId(context.getConfiguration());
+  }
+
+  /**
+   * Get the Application Attempt Id for this job
+   * by looking for {@link MRJobConfig#APPLICATION_ATTEMPT_ID}
+   * in the configuration, falling back to 0 if unset.
+   * For spark it will always be 0, for MR it will be set in the AM
+   * to the {@code ApplicationAttemptId} the AM is launched with.
+   * @param conf job configuration.
+   * @return the Application Attempt Id for the job.
+   */
+  public static int getAppAttemptId(Configuration conf) {
+    return conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        INITIAL_APP_ATTEMPT_ID);
+  }
+
+  /**
+   * Get the path in the job attempt dir for a manifest for a task.
+   * @param manifestDir manifest directory
+   * @param taskId taskID.
+   * @return the final path to rename the manifest file to
+   */
+  public static Path manifestPathForTask(Path manifestDir, String taskId) {
+
+    return new Path(manifestDir, taskId + MANIFEST_SUFFIX);
+  }
+
+  /**
+   * Get the path in the  manifest subdir for the temp path to save a
+   * task attempt's manifest before renaming it to the
+   * path defined by {@link #manifestPathForTask(Path, String)}.
+   * @param manifestDir manifest directory
+   * @param taskAttemptId task attempt ID.
+   * @return the path to save/load the manifest.
+   */
+  public static Path manifestTempPathForTaskAttempt(Path manifestDir,
+      String taskAttemptId) {
+    return new Path(manifestDir,
+        taskAttemptId + MANIFEST_SUFFIX + TMP_SUFFIX);
+  }
+
+  /**
+   * Create a task attempt dir; stage config must be for a task attempt.
+   * @param stageConfig state config.
+   * @return a manifest with job and task attempt info set up.
+   */
+  public static TaskManifest createTaskManifest(StageConfig stageConfig) {
+    final TaskManifest manifest = new TaskManifest();
+    manifest.setTaskAttemptID(stageConfig.getTaskAttemptId());
+    manifest.setTaskID(stageConfig.getTaskId());
+    manifest.setJobId(stageConfig.getJobId());
+    manifest.setJobAttemptNumber(stageConfig.getJobAttemptNumber());
+    manifest.setTaskAttemptDir(
+        stageConfig.getTaskAttemptDir().toUri().toString());
+    return manifest;
+  }
+
+  /**
+   * Create success/outcome data.
+   * @param stageConfig configuration.
+   * @param stage
+   * @return a _SUCCESS object with some diagnostics.
+   */
+  public static ManifestSuccessData createManifestOutcome(
+      StageConfig stageConfig, String stage) {
+    final ManifestSuccessData outcome = new ManifestSuccessData();
+    outcome.setJobId(stageConfig.getJobId());
+    outcome.setJobIdSource(stageConfig.getJobIdSource());
+    outcome.setCommitter(MANIFEST_COMMITTER_CLASSNAME);
+    // real timestamp
+    outcome.setTimestamp(System.currentTimeMillis());
+    final ZonedDateTime now = ZonedDateTime.now();
+    outcome.setDate(now.toString());
+    outcome.setHostname(NetUtils.getLocalHostname());
+    // add some extra diagnostics which can still be parsed by older
+    // builds of test applications.
+    // Audit Span information can go in here too, in future.
+    try {
+      outcome.putDiagnostic(PRINCIPAL,
+          UserGroupInformation.getCurrentUser().getShortUserName());
+    } catch (IOException ignored) {
+      // don't know who we are? exclude from the diagnostics.
+    }
+    outcome.putDiagnostic(STAGE, stage);
+    return outcome;
+  }
+
+  /**
+   * Create the filename for a report from the jobID.
+   * @param jobId jobId
+   * @return filename for a report.
+   */
+  public static String createJobSummaryFilename(String jobId) {
+    return String.format(SUMMARY_FILENAME_FORMAT, jobId);
+  }
+
+  /**
+   * Get an etag from a FileStatus which MUST BE
+   * an implementation of EtagSource and
+   * whose etag MUST NOT BE null/empty.
+   * @param status the status; may be null.
+   * @return the etag or null if not provided
+   */
+  public static String getEtag(FileStatus status) {
+    if (status instanceof EtagSource) {
+      return ((EtagSource) status).getEtag();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Create the manifest store operations for the given FS.
+   * This supports binding to custom filesystem handlers.
+   * @param conf configuration.
+   * @param filesystem fs.
+   * @param path path under FS.
+   * @return a bonded store operations.
+   * @throws IOException on binding/init problems.
+   */
+  public static ManifestStoreOperations createManifestStoreOperations(
+      final Configuration conf,
+      final FileSystem filesystem,
+      final Path path) throws IOException {
+    try {
+      final Class<? extends ManifestStoreOperations> storeClass = conf.getClass(
+          OPT_STORE_OPERATIONS_CLASS,
+          ManifestStoreOperationsThroughFileSystem.class,
+          ManifestStoreOperations.class);
+      final ManifestStoreOperations operations = storeClass.
+          getDeclaredConstructor().newInstance();
+      operations.bindToFileSystem(filesystem, path);
+      return operations;
+    } catch (Exception e) {
+      throw new PathIOException(path.toString(),
+          "Failed to create Store Operations from configuration option "
+              + OPT_STORE_OPERATIONS_CLASS
+              + ":" + e, e);
+    }
+  }
+
+  /**
+   * Logic to create directory names from job and attempt.
+   * This is self-contained it so it can be used in tests
+   * as well as in the committer.
+   */
+  public static class AttemptDirectories {
+
+    /**
+     * Job output path.
+     */
+    private final Path outputPath;
+
+    /**
+     * Path for the job attempt.
+     */
+    private final Path jobAttemptDir;
+
+    /**
+     * Path for the job.
+     */
+    private final Path jobPath;
+
+    /**
+     * Subdir under the job attempt dir where task
+     * attempts will have subdirectories.
+     */
+    private final Path jobAttemptTaskSubDir;
+
+    /**
+     * temp directory under job dest dir.
+     */
+    private final Path outputTempSubDir;
+
+    /**
+     * Directory to save manifests into.
+     */
+    private final Path taskManifestDir;
+
+    /**
+     * Build the attempt directories.
+     * @param outputPath output path
+     * @param jobUniqueId job ID/UUID
+     * @param jobAttemptNumber job attempt number
+     */
+    public AttemptDirectories(
+        Path outputPath,
+        String jobUniqueId,
+        int jobAttemptNumber) {
+      this.outputPath = requireNonNull(outputPath, "Output path");
+
+      this.outputTempSubDir = new Path(outputPath, PENDING_DIR_NAME);
+      // build the path for the job
+      this.jobPath = new Path(outputTempSubDir,
+          String.format(JOB_DIR_FORMAT_STR, jobUniqueId));
+
+      // then the specific path underneath that for the attempt.
+      this.jobAttemptDir = new Path(jobPath,
+          String.format(JOB_ATTEMPT_DIR_FORMAT_STR, jobAttemptNumber));
+
+      // subdir for task attempts.
+      this.jobAttemptTaskSubDir = new Path(jobAttemptDir, JOB_TASK_ATTEMPT_SUBDIR);
+
+      this.taskManifestDir = new Path(jobAttemptDir, JOB_TASK_MANIFEST_SUBDIR);
+    }
+
+    public Path getOutputPath() {
+      return outputPath;
+    }
+
+    public Path getJobAttemptDir() {
+      return jobAttemptDir;
+    }
+
+    public Path getJobPath() {
+      return jobPath;
+    }
+
+    public Path getJobAttemptTaskSubDir() {
+      return jobAttemptTaskSubDir;
+    }
+
+    public Path getTaskAttemptPath(String taskAttemptId) {
+      return new Path(jobAttemptTaskSubDir, taskAttemptId);
+    }
+
+    public Path getOutputTempSubDir() {
+      return outputTempSubDir;
+    }
+
+    public Path getTaskManifestDir() {
+      return taskManifestDir;
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java
new file mode 100644
index 0000000..b81fa9d
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java
@@ -0,0 +1,291 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.time.Duration;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.JsonSerialization;
+
+/**
+ * FileSystem operations which are needed to generate the task manifest.
+ * The specific choice of which implementation to use is configurable.
+ * Object store implementations MAY subclass if they
+ * need to implement resilient commit operations.
+ * However, the actual API MUST NOT be used outside
+ * the manifest committer and its tests.
+ */
+@InterfaceAudience.LimitedPrivate("mapreduce, object-stores")
+@InterfaceStability.Unstable
+public abstract class ManifestStoreOperations implements Closeable {
+
+  /**
+   * Bind to the filesystem.
+   * This is called by the manifest committer after the operations
+   * have been instantiated.
+   * @param fileSystem target FS
+   * @param path actual path under FS.
+   * @throws IOException if there are binding problems.
+   */
+  public void bindToFileSystem(FileSystem fileSystem, Path path) throws IOException {
+
+  }
+
+  /**
+   * Forward to {@link FileSystem#getFileStatus(Path)}.
+   * @param path path
+   * @return status
+   * @throws IOException failure.
+   */
+  public abstract FileStatus getFileStatus(Path path) throws IOException;
+
+  /**
+   * Is a path a file? Used during directory creation.
+   * The is a copy & paste of FileSystem.isFile();
+   * {@code StoreOperationsThroughFileSystem} calls into
+   * the FS direct so that stores which optimize their probes
+   * can save on IO.
+   * @param path path to probe
+   * @return true if the path exists and resolves to a file
+   * @throws IOException failure other than FileNotFoundException
+   */
+  public boolean isFile(Path path) throws IOException {
+    try {
+      return getFileStatus(path).isFile();
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  /**
+   * Forward to {@link FileSystem#delete(Path, boolean)}.
+   * If it returns without an error: there is nothing at
+   * the end of the path.
+   * @param path path
+   * @param recursive recursive delete.
+   * @return true if the path was deleted.
+   * @throws IOException failure.
+   */
+  public abstract boolean delete(Path path, boolean recursive)
+      throws IOException;
+
+  /**
+   * Forward to {@link FileSystem#mkdirs(Path)}.
+   * Usual "what does 'false' mean" ambiguity.
+   * @param path path
+   * @return true if the directory was created.
+   * @throws IOException failure.
+   */
+  public abstract boolean mkdirs(Path path) throws IOException;
+
+  /**
+   * Forward to {@link FileSystem#rename(Path, Path)}.
+   * Usual "what does 'false' mean" ambiguity.
+   * @param source source file
+   * @param dest destination path -which must not exist.
+   * @return the return value of the rename
+   * @throws IOException failure.
+   */
+  public abstract boolean renameFile(Path source, Path dest)
+      throws IOException;
+
+  /**
+   * Rename a dir; defaults to invoking
+   * Forward to {@link #renameFile(Path, Path)}.
+   * Usual "what does 'false' mean?" ambiguity.
+   * @param source source file
+   * @param dest destination path -which must not exist.
+   * @return true if the directory was created.
+   * @throws IOException failure.
+   */
+  public boolean renameDir(Path source, Path dest)
+      throws IOException {
+    return renameFile(source, dest);
+  }
+
+  /**
+   * List the directory.
+   * @param path path to list.
+   * @return an iterator over the results.
+   * @throws IOException any immediate failure.
+   */
+  public abstract RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException;
+
+  /**
+   * Load a task manifest from the store.
+   * with a real FS, this is done with
+   * {@link TaskManifest#load(JsonSerialization, FileSystem, Path, FileStatus)}
+   *
+   * @param serializer serializer.
+   * @param st status with the path and other data.
+   * @return the manifest
+   * @throws IOException failure to load/parse
+   */
+  public abstract TaskManifest loadTaskManifest(
+      JsonSerialization<TaskManifest> serializer,
+      FileStatus st) throws IOException;
+
+  /**
+   * Save a task manifest by {@code FileSystem.create(path)}.
+   * there's no attempt at renaming anything here.
+   * @param manifestData the manifest/success file
+   * @param path temp path for the initial save
+   * @param overwrite should create(overwrite=true) be used?
+   * @throws IOException failure to load/parse
+   */
+  public abstract <T extends AbstractManifestData<T>> void save(
+      T manifestData,
+      Path path,
+      boolean overwrite) throws IOException;
+
+  /**
+   * Make an msync() call; swallow when unsupported.
+   * @param path path
+   * @throws IOException IO failure
+   */
+  public void msync(Path path) throws IOException {
+
+  }
+
+
+  /**
+   * Extract an etag from a status if the conditions are met.
+   * If the conditions are not met, return null or ""; they will
+   * both be treated as "no etags available"
+   * <pre>
+   *   1. The status is of a type which the implementation recognizes
+   *   as containing an etag.
+   *   2. After casting the etag field can be retrieved
+   *   3. and that value is non-null/non-empty.
+   * </pre>
+   * @param status status, which may be null of any subclass of FileStatus.
+   * @return either a valid etag, or null or "".
+   */
+  public String getEtag(FileStatus status) {
+    return ManifestCommitterSupport.getEtag(status);
+  }
+
+  /**
+   * Does the store preserve etags through renames.
+   * If true, and if the source listing entry has an etag,
+   * it will be used to attempt to validate a failed rename.
+   * @param path path to probe.
+   * @return true if etag comparison is a valid strategy.
+   */
+  public boolean storePreservesEtagsThroughRenames(Path path) {
+    return false;
+  }
+
+  /**
+   * Does the store provide rename resilience through an
+   * implementation of {@link #commitFile(FileEntry)}?
+   * If true then that method will be invoked to commit work
+   * @return true if resilient commit support is available.
+   */
+  public boolean storeSupportsResilientCommit() {
+    return false;
+  }
+
+  /**
+   * Commit one file through any resilient API.
+   * This operation MUST rename source to destination,
+   * else raise an exception.
+   * The result indicates whether or not some
+   * form of recovery took place.
+   *
+   * If etags were collected during task commit, these will be
+   * in the entries passed in here.
+   *
+   * The base implementation always raises
+   * {@code UnsupportedOperationException}
+   * @param entry entry to commit
+   * @return the result of the commit
+   * @throws IOException failure.
+   * @throws UnsupportedOperationException if not available.
+   *
+   */
+  public CommitFileResult commitFile(FileEntry entry) throws IOException {
+    throw new UnsupportedOperationException("Resilient commit not supported");
+  }
+
+  /**
+   * Outcome from the operation {@link #commitFile(FileEntry)}.
+   * As a rename failure MUST raise an exception, this result
+   * only declares whether or not some form of recovery took place.
+   */
+  public static final class CommitFileResult {
+
+    /** Did recovery take place? */
+    private final boolean recovered;
+
+    /** Time waiting for IO capacity, may be null. */
+    @Nullable
+    private final Duration waitTime;
+
+    /**
+     * Full commit result.
+     * @param recovered Did recovery take place?
+     * @param waitTime any time spent waiting for IO capacity.
+     */
+    public static CommitFileResult fromResilientCommit(
+        final boolean recovered,
+        final Duration waitTime) {
+      return new CommitFileResult(recovered, waitTime);
+    }
+
+    /**
+     * Full commit result.
+     * @param recovered Did recovery take place?
+     * @param waitTime any time spent waiting for IO capacity.
+     */
+    public CommitFileResult(final boolean recovered,
+        @Nullable final Duration waitTime) {
+
+      this.recovered = recovered;
+      this.waitTime = waitTime;
+    }
+
+    /**
+     * Did some form of recovery take place?
+     * @return true if the commit succeeded through some form of (etag-based) recovery
+     */
+    public boolean recovered() {
+      return recovered;
+    }
+
+    @Nullable
+    public Duration getWaitTime() {
+      return waitTime;
+    }
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java
new file mode 100644
index 0000000..9a0b972
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java
@@ -0,0 +1,187 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonPathCapabilities;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.JsonSerialization;
+
+/**
+ * Implementation of manifest store operations through the filesystem API.
+ * This class is subclassed in the ABFS module, which does add the resilient
+ * commit method.
+ */
+@InterfaceAudience.LimitedPrivate("mapreduce, object-stores")
+@InterfaceStability.Unstable
+public class ManifestStoreOperationsThroughFileSystem extends ManifestStoreOperations {
+
+  /**
+   * Filesystem; set in {@link #bindToFileSystem(FileSystem, Path)}.
+   */
+  private FileSystem fileSystem;
+
+  /**
+   * Has a call to FileSystem.msync() failed as unsupported?
+   * If so, no new attempts will be made when
+   * (@link {@link #msync(Path)} is invoked.
+   */
+  private boolean msyncUnsupported = false;
+
+  /**
+   * Direct Constructor.
+   * @param fileSystem filesystem to write through.
+   */
+  public ManifestStoreOperationsThroughFileSystem(final FileSystem fileSystem) {
+    this.fileSystem = fileSystem;
+  }
+
+  /**
+   * Constructor used for introspection-based binding.
+   */
+  public ManifestStoreOperationsThroughFileSystem() {
+  }
+
+  @Override
+  public void close() throws IOException {
+    /* no-op; FS is assumed to be shared. */
+
+  }
+
+  /**
+   * Get the filesystem.
+   * @return the filesystem; null until bound.
+   */
+  public FileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  @Override
+  public void bindToFileSystem(FileSystem filesystem, Path path) throws IOException {
+    fileSystem = filesystem;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path path) throws IOException {
+    return fileSystem.getFileStatus(path);
+  }
+
+  /**
+   * Using FileSystem.isFile to offer stores the option to optimize their probes.
+   * @param path path to probe
+   * @return true if the path resolves to a file.
+   * @throws IOException IO failure.
+   */
+  @SuppressWarnings("deprecation")
+  @Override
+  public boolean isFile(Path path) throws IOException {
+    return fileSystem.isFile(path);
+  }
+
+  @Override
+  public boolean delete(Path path, boolean recursive)
+      throws IOException {
+    return fileSystem.delete(path, recursive);
+  }
+
+  @Override
+  public boolean mkdirs(Path path)
+      throws IOException {
+    return fileSystem.mkdirs(path);
+  }
+
+  @Override
+  public boolean renameFile(Path source, Path dest)
+      throws IOException {
+    return fileSystem.rename(source, dest);
+  }
+
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(Path path)
+      throws IOException {
+    return fileSystem.listStatusIterator(path);
+  }
+
+  @Override
+  public TaskManifest loadTaskManifest(
+      JsonSerialization<TaskManifest> serializer,
+      FileStatus st) throws IOException {
+    return TaskManifest.load(serializer, fileSystem, st.getPath(), st);
+  }
+
+  @Override
+  public <T extends AbstractManifestData<T>> void save(
+      final T manifestData,
+      final Path path,
+      final boolean overwrite) throws IOException {
+    manifestData.save(fileSystem, path, overwrite);
+  }
+
+  /**
+   * Probe filesystem capabilities.
+   * @param path path to probe.
+   * @return true if the FS declares its renames work.
+   */
+  @Override
+  public boolean storePreservesEtagsThroughRenames(Path path) {
+    try {
+      return fileSystem.hasPathCapability(path,
+          CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME);
+    } catch (IOException ignored) {
+      return false;
+    }
+  }
+
+  /**
+   * Invokes FileSystem msync(); swallows UnsupportedOperationExceptions.
+   * This ensures client metadata caches are in sync in an HDFS-HA deployment.
+   * No other filesystems support this; in the absence of a hasPathCapability()
+   * probe, after the operation is rejected, an atomic boolean is set
+   * to stop further attempts from even trying.
+   * @param path path
+   * @throws IOException failure to synchronize.
+   */
+  @Override
+  public void msync(Path path) throws IOException {
+    // there's need for atomicity here, as the sole cost of
+    // multiple failures
+    if (msyncUnsupported) {
+      return;
+    }
+    // qualify so we can be confident that the FS being synced
+    // is the one we expect.
+    fileSystem.makeQualified(path);
+    try {
+      fileSystem.msync();
+    } catch (UnsupportedOperationException ignored) {
+      // this exception is the default.
+      // set the unsupported flag so no future attempts are made.
+      msyncUnsupported = true;
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java
new file mode 100644
index 0000000..f1dacc2
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Exception raised during validation.
+ * This can be treated differently from other outcomes.
+ */
+@InterfaceAudience.Private
+public class OutputValidationException extends PathIOException {
+  public OutputValidationException(Path path, String error) {
+    super(path.toUri().toString(), error);
+  }
+
+  public OutputValidationException(Path path,
+      String error,
+      Throwable cause) {
+    super(path.toUri().toString(), error, cause);
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java
new file mode 100644
index 0000000..eb7dda6
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation classes for the manifest committer.
+ * Nothing outside this module should be using these classes,
+ * except where explicitly stated.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java
new file mode 100644
index 0000000..c00ae5a
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/**
+ * Intermediate manifest committer.
+ *
+ * Optimized for object stores where listing is slow, directory renames may not
+ * be atomic, and the output is a deep tree of files intermixed with
+ * the output of (many) other task attempts.
+ *
+ * All classes in this module are private/unstable, except where stated.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.mapreduce.lib.output.committer.manifest;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java
new file mode 100644
index 0000000..c2b44c2
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java
@@ -0,0 +1,63 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_ABORT_TASK;
+
+/**
+ * Abort a task.
+ *
+ * This is done by deleting the task directory.
+ * Exceptions may/may not be suppressed.
+ */
+public class AbortTaskStage extends
+    AbstractJobOrTaskStage<Boolean, Path> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbortTaskStage.class);
+
+  public AbortTaskStage(final StageConfig stageConfig) {
+    super(true, stageConfig, OP_STAGE_TASK_ABORT_TASK, false);
+  }
+
+  /**
+   * Delete the task attempt directory.
+   * @param suppressExceptions should exceptions be ignored?
+   * @return the directory
+   * @throws IOException failure when exceptions were not suppressed
+   */
+  @Override
+  protected Path executeStage(final Boolean suppressExceptions)
+      throws IOException {
+    final Path dir = getTaskAttemptDir();
+    if (dir != null) {
+      LOG.info("{}: Deleting task attempt directory {}", getName(), dir);
+      deleteDir(dir, suppressExceptions);
+    }
+    return dir;
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java
new file mode 100644
index 0000000..05ee7a5
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java
@@ -0,0 +1,942 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations;
+import org.apache.hadoop.util.OperationDuration;
+import org.apache.hadoop.util.Preconditions;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_GET_FILE_STATUS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_FILE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_LIST_STATUS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_MKDIRS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.createTracker;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME_RECOVERED;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_MANIFEST;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MSYNC;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_RENAME_FILE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_SAVE_TASK_MANIFEST;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.enterStageWorker;
+
+/**
+ * A Stage in Task/Job Commit.
+ * A stage can be executed once only, creating the return value of the
+ * {@link #apply(Object)} method, and, potentially, updating the state of the
+ * store via {@link ManifestStoreOperations}.
+ * IOStatistics will also be updated.
+ * Stages are expected to be combined to form the commit protocol.
+ * @param <IN> Type of arguments to the stage.
+ * @param <OUT> Type of result.
+ */
+public abstract class AbstractJobOrTaskStage<IN, OUT>
+    implements JobOrTaskStage<IN, OUT> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractJobOrTaskStage.class);
+
+  /**
+   * Error text on rename failure: {@value}.
+   */
+  public static final String FAILED_TO_RENAME_PREFIX = "Failed to ";
+
+  /**
+   * Is this a task stage? If so, toString() includes task
+   * info..
+   */
+  private final boolean isTaskStage;
+
+  /**
+   * Configuration of all the stages in the ongoing committer
+   * operation.
+   */
+  private final StageConfig stageConfig;
+
+  /**
+   * Name of the stage for statistics and logging.
+   */
+  private final String stageStatisticName;
+
+  /**
+   * Callbacks to update store.
+   * This is not made visible to the stages; they must
+   * go through the wrapper classes in this class, which
+   * add statistics and logging.
+   */
+  private final ManifestStoreOperations operations;
+
+  /**
+   * Submitter for doing IO against the store.
+   */
+  private final TaskPool.Submitter ioProcessors;
+
+  /**
+   * Used to stop any re-entrancy of the rename.
+   * This is an execute-once operation.
+   */
+  private final AtomicBoolean executed = new AtomicBoolean(false);
+
+  /**
+   * Tracker of the duration of the execution of the stage.
+   * set after {@link #executeStage(Object)} completes.
+   */
+  private DurationTracker stageExecutionTracker;
+
+  /**
+   * Name for logging.
+   */
+  private final String name;
+
+  /**
+   * Constructor.
+   * @param isTaskStage Is this a task stage?
+   * @param stageConfig stage-independent configuration.
+   * @param stageStatisticName name of the stage for statistics/logging
+   * @param requireIOProcessors are the IO processors required?
+   */
+  protected AbstractJobOrTaskStage(
+      final boolean isTaskStage,
+      final StageConfig stageConfig,
+      final String stageStatisticName,
+      final boolean requireIOProcessors) {
+    this.isTaskStage = isTaskStage;
+    this.stageStatisticName = stageStatisticName;
+    this.stageConfig = stageConfig;
+    requireNonNull(stageConfig.getDestinationDir(), "Destination Directory");
+    requireNonNull(stageConfig.getJobId(), "Job ID");
+    requireNonNull(stageConfig.getJobAttemptDir(), "Job attempt directory");
+    this.operations = requireNonNull(stageConfig.getOperations(),
+        "Operations callbacks");
+    // and the processors of work if required.
+    this.ioProcessors = bindProcessor(
+        requireIOProcessors,
+        stageConfig.getIoProcessors());
+    String stageName;
+    if (isTaskStage) {
+      // force fast failure.
+      getRequiredTaskId();
+      getRequiredTaskAttemptId();
+      getRequiredTaskAttemptDir();
+      stageName = String.format("[Task-Attempt %s]", getRequiredTaskAttemptId());
+    } else  {
+      stageName = String.format("[Job-Attempt %s/%02d]",
+          stageConfig.getJobId(),
+          stageConfig.getJobAttemptNumber());
+    }
+    name = stageName;
+  }
+
+  /**
+   * Bind to the processor if it is required.
+   * @param required is the processor required?
+   * @param processor processor
+   * @return the processor binding
+   * @throws NullPointerException if required == true and processor is null.
+   */
+  private TaskPool.Submitter bindProcessor(
+      final boolean required,
+      final TaskPool.Submitter processor) {
+    return required
+        ? requireNonNull(processor, "required IO processor is null")
+        : null;
+  }
+
+  /**
+   * Stage entry point.
+   * Verifies that this is the first and only time the stage is invoked,
+   * then calls {@link #executeStage(Object)} for the subclass
+   * to perform its part of the commit protocol.
+   * The duration of the stage is collected as a statistic, and its
+   * entry/exit logged at INFO.
+   * @param arguments arguments to the function.
+   * @return the result.
+   * @throws IOException failures.
+   */
+  @Override
+  public final OUT apply(final IN arguments) throws IOException {
+    executeOnlyOnce();
+    progress();
+    String stageName = getStageName(arguments);
+    getStageConfig().enterStage(stageName);
+    String statisticName = getStageStatisticName(arguments);
+    // The tracker here
+    LOG.info("{}: Executing Stage {}", getName(), stageName);
+    stageExecutionTracker = createTracker(getIOStatistics(), statisticName);
+    try {
+      // exec the input function and return its value
+      final OUT out = executeStage(arguments);
+      LOG.info("{}: Stage {} completed after {}",
+          getName(),
+          stageName,
+          OperationDuration.humanTime(
+              stageExecutionTracker.asDuration().toMillis()));
+      return out;
+    } catch (IOException | RuntimeException e) {
+      LOG.error("{}: Stage {} failed: after {}: {}",
+          getName(),
+          stageName,
+          OperationDuration.humanTime(
+              stageExecutionTracker.asDuration().toMillis()),
+          e.toString());
+      LOG.debug("{}: Stage failure:", getName(), e);
+      // input function failed: note it
+      stageExecutionTracker.failed();
+      // and rethrow
+      throw e;
+    } finally {
+      // update the tracker.
+      // this is called after the catch() call would have
+      // set the failed flag.
+      stageExecutionTracker.close();
+      progress();
+      getStageConfig().exitStage(stageName);
+    }
+  }
+
+  /**
+   * The work of a stage.
+   * Executed exactly once.
+   * @param arguments arguments to the function.
+   * @return the result.
+   * @throws IOException failures.
+   */
+  protected abstract OUT executeStage(IN arguments) throws IOException;
+
+  /**
+   * Check that the operation has not been invoked twice.
+   * This is an atomic check.
+   * @throws IllegalStateException on a second invocation.
+   */
+  private void executeOnlyOnce() {
+    Preconditions.checkState(
+        !executed.getAndSet(true),
+        "Stage attempted twice");
+  }
+
+  /**
+   * The stage statistic name.
+   * @param arguments args to the invocation.
+   * @return stage name.
+   */
+  protected String getStageStatisticName(IN arguments) {
+    return stageStatisticName;
+  }
+
+  /**
+   * Stage name for reporting; defaults to
+   * call {@link #getStageStatisticName(IN)}.
+   * @param arguments args to the invocation.
+   * @return name used in updating reports.
+   */
+  protected String getStageName(IN arguments) {
+    return getStageStatisticName(arguments);
+  }
+
+  /**
+   * Get the execution tracker; non-null
+   * after stage execution.
+   * @return a tracker or null.
+   */
+  public DurationTracker getStageExecutionTracker() {
+    return stageExecutionTracker;
+  }
+
+  /**
+   * Adds the duration of the job to an IOStatistics store
+   * (such as the manifest to be saved).
+   * @param iostats store
+   * @param statistic statistic name.
+   */
+  public void addExecutionDurationToStatistics(IOStatisticsStore iostats,
+      String statistic) {
+    iostats.addTimedOperation(
+        statistic,
+        getStageExecutionTracker().asDuration());
+  }
+
+  /**
+   * Note any rate limiting to the given timing statistic.
+   * If the wait was 0, no statistics are updated.
+   * @param statistic statistic key.
+   * @param wait wait duration.
+   */
+  private void noteAnyRateLimiting(String statistic, Duration wait) {
+    if (!wait.isZero()) {
+      // rate limiting took place
+      getIOStatistics().addTimedOperation(
+          statistic,
+          wait.toMillis());
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AbstractJobOrTaskStage{");
+    sb.append(isTaskStage ? "Task Stage" : "Job Stage");
+    sb.append(" name='").append(name).append('\'');
+    sb.append(" stage='").append(stageStatisticName).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * The stage configuration.
+   * @return the stage configuration used by this stage.
+   */
+  protected StageConfig getStageConfig() {
+    return stageConfig;
+  }
+
+  /**
+   * Update the thread context with the stage name and
+   * job ID.
+   * This MUST be invoked at the start of methods invoked in helper threads,
+   * to ensure that they are all annotated with job and stage.
+   * @param stage stage name.
+   */
+  protected void updateAuditContext(final String stage) {
+    enterStageWorker(stageConfig.getJobId(), stage);
+  }
+
+  /**
+   * The IOStatistics are shared across all uses of the
+   * StageConfig.
+   * @return the (possibly shared) IOStatistics.
+   */
+  @Override
+  public final IOStatisticsStore getIOStatistics() {
+    return stageConfig.getIOStatistics();
+  }
+
+  /**
+   * Call progress() on any Progressable passed in.
+   */
+  protected final void progress() {
+    if (stageConfig.getProgressable() != null) {
+      stageConfig.getProgressable().progress();
+    }
+  }
+
+  /**
+   * Get a file status value or, if the path doesn't exist, return null.
+   * @param path path
+   * @return status or null
+   * @throws IOException IO Failure.
+   */
+  protected final FileStatus getFileStatusOrNull(
+      final Path path)
+      throws IOException {
+    try {
+      return getFileStatus(path);
+    } catch (FileNotFoundException e) {
+      return null;
+    }
+  }
+
+  /**
+   * Get a file status value or, if the path doesn't exist, return null.
+   * @param path path
+   * @return status or null
+   * @throws IOException IO Failure.
+   */
+  protected final FileStatus getFileStatus(
+      final Path path)
+      throws IOException {
+    LOG.trace("{}: getFileStatus('{}')", getName(), path);
+    requireNonNull(path,
+        () -> String.format("%s: Null path for getFileStatus() call", getName()));
+    return trackDuration(getIOStatistics(), OP_GET_FILE_STATUS, () ->
+        operations.getFileStatus(path));
+  }
+
+  /**
+   * Get a file status value or, if the path doesn't exist, return null.
+   * @param path path
+   * @return true if the path resolves to a file
+   * @throws IOException IO Failure.
+   */
+  protected final boolean isFile(
+      final Path path)
+      throws IOException {
+    LOG.trace("{}: isFile('{}')", getName(), path);
+    return trackDuration(getIOStatistics(), OP_IS_FILE, () -> {
+      return operations.isFile(path);
+    });
+  }
+
+  /**
+   * Delete a path.
+   * @param path path
+   * @param recursive recursive delete.
+   * @return status or null
+   * @throws IOException IO Failure.
+   */
+  protected final boolean delete(
+      final Path path,
+      final boolean recursive)
+      throws IOException {
+    LOG.trace("{}: delete('{}, {}')", getName(), path, recursive);
+    return delete(path, recursive, OP_DELETE);
+  }
+
+  /**
+   * Delete a path.
+   * @param path path
+   * @param recursive recursive delete.
+   * @param statistic statistic to update
+   * @return status or null
+   * @throws IOException IO Failure.
+   */
+  protected Boolean delete(
+      final Path path,
+      final boolean recursive,
+      final String statistic)
+      throws IOException {
+    return trackDuration(getIOStatistics(), statistic, () -> {
+      return operations.delete(path, recursive);
+    });
+  }
+
+  /**
+   * Create a directory.
+   * @param path path
+   * @param escalateFailure escalate "false" to PathIOE
+   * @return true if the directory was created/exists.
+   * @throws IOException IO Failure.
+   */
+  protected final boolean mkdirs(
+      final Path path,
+      final boolean escalateFailure)
+      throws IOException {
+    LOG.trace("{}: mkdirs('{}')", getName(), path);
+    return trackDuration(getIOStatistics(), OP_MKDIRS, () -> {
+      boolean success = operations.mkdirs(path);
+      if (!success && escalateFailure) {
+        throw new PathIOException(path.toUri().toString(),
+            stageStatisticName + ": mkdirs() returned false");
+      }
+      return success;
+    });
+
+  }
+
+  /**
+   * List all directly files under a path.
+   * Async implementations may under-report their durations.
+   * @param path path
+   * @return iterator over the results.
+   * @throws IOException IO Failure.
+   */
+  protected final RemoteIterator<FileStatus> listStatusIterator(
+      final Path path)
+      throws IOException {
+    LOG.trace("{}: listStatusIterator('{}')", getName(), path);
+    return trackDuration(getIOStatistics(), OP_LIST_STATUS, () ->
+        operations.listStatusIterator(path));
+  }
+
+  /**
+   * Load a manifest file.
+   * @param status source.
+   * @return the manifest.
+   * @throws IOException IO Failure.
+   */
+  protected final TaskManifest loadManifest(
+      final FileStatus status)
+      throws IOException {
+    LOG.trace("{}: loadManifest('{}')", getName(), status);
+    return trackDuration(getIOStatistics(), OP_LOAD_MANIFEST, () ->
+        operations.loadTaskManifest(
+            stageConfig.currentManifestSerializer(),
+            status));
+  }
+
+  /**
+   * List all the manifests in the task manifest dir.
+   * @return a iterator of manifests.
+   * @throws IOException IO Failure.
+   */
+  protected final RemoteIterator<FileStatus> listManifests()
+      throws IOException {
+    return RemoteIterators.filteringRemoteIterator(
+        listStatusIterator(getTaskManifestDir()),
+        st -> st.getPath().toUri().toString().endsWith(MANIFEST_SUFFIX));
+  }
+
+  /**
+   * Make an msync() call; swallow when unsupported.
+   * @param path path
+   * @throws IOException IO failure
+   */
+  protected final void msync(Path path) throws IOException {
+    LOG.trace("{}: msync('{}')", getName(), path);
+    trackDurationOfInvocation(getIOStatistics(), OP_MSYNC, () ->
+        operations.msync(path));
+  }
+
+  /**
+   * Create a directory -failing if it exists or if
+   * mkdirs() failed.
+   * @param operation operation for error reporting.
+   * @param path path path to create.
+   * @return the path.
+   * @throws IOException failure
+   * @throws PathIOException mkdirs failed.
+   * @throws FileAlreadyExistsException destination exists.
+   */
+  protected final Path createNewDirectory(
+      final String operation,
+      final Path path) throws IOException {
+    LOG.trace("{}: {} createNewDirectory('{}')", getName(), operation, path);
+    requireNonNull(path,
+        () -> String.format("%s: Null path for operation %s", getName(), operation));
+    // check for dir existence before trying to create.
+    try {
+      final FileStatus status = getFileStatus(path);
+      // no exception, so the path exists.
+      throw new FileAlreadyExistsException(operation
+          + ": path " + path
+          + " already exists and has status " + status);
+    } catch (FileNotFoundException e) {
+      // the path does not exist, so create it.
+      mkdirs(path, true);
+      return path;
+    }
+  }
+
+  /**
+   * Assert that a path is a directory which must exist.
+   * @param operation operation for error reporting.
+   * @param path path path to create.
+   * @return the path
+   * @throws IOException failure
+   * @throws PathIOException mkdirs failed.
+   * @throws FileAlreadyExistsException destination exists.
+   */
+  protected final Path directoryMustExist(
+      final String operation,
+      final Path path) throws IOException {
+    final FileStatus status = getFileStatus(path);
+    if (!status.isDirectory()) {
+      throw new PathIOException(path.toString(),
+          operation
+              + ": Path is not a directory; its status is :" + status);
+    }
+    return path;
+  }
+
+  /**
+   * Save a task manifest or summary. This will be done by
+   * writing to a temp path and then renaming.
+   * If the destination path exists: Delete it.
+   * @param manifestData the manifest/success file
+   * @param tempPath temp path for the initial save
+   * @param finalPath final path for rename.
+   * @throws IOException failure to load/parse
+   */
+  @SuppressWarnings("unchecked")
+  protected final <T extends AbstractManifestData> void save(T manifestData,
+      final Path tempPath,
+      final Path finalPath) throws IOException {
+    LOG.trace("{}: save('{}, {}, {}')", getName(), manifestData, tempPath, finalPath);
+    trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () ->
+        operations.save(manifestData, tempPath, true));
+    renameFile(tempPath, finalPath);
+  }
+
+  /**
+   * Get an etag from a FileStatus which MUST BE
+   * an implementation of EtagSource and
+   * whose etag MUST NOT BE null/empty.
+   * @param status the status; may be null.
+   * @return the etag or null if not provided
+   */
+  public String getEtag(FileStatus status) {
+    return operations.getEtag(status);
+  }
+
+  /**
+   * Rename a file from source to dest; if the underlying FS API call
+   * returned false that's escalated to an IOE.
+   * @param source source file.
+   * @param dest dest file
+   * @throws IOException failure
+   * @throws PathIOException if the rename() call returned false.
+   */
+  protected final void renameFile(final Path source, final Path dest)
+      throws IOException {
+    maybeDeleteDest(true, dest);
+    executeRenamingOperation("renameFile", source, dest,
+        OP_RENAME_FILE, () ->
+            operations.renameFile(source, dest));
+  }
+
+  /**
+   * Rename a file from source to dest; if the underlying FS API call
+   * returned false that's escalated to an IOE.
+   * @param source source file.
+   * @param dest dest file
+   * @throws IOException failure
+   * @throws PathIOException if the rename() call returned false.
+   */
+  protected final void renameDir(final Path source, final Path dest)
+      throws IOException {
+
+    maybeDeleteDest(true, dest);
+    executeRenamingOperation("renameDir", source, dest,
+        OP_RENAME_FILE, () ->
+        operations.renameDir(source, dest)
+    );
+  }
+
+  /**
+   * Commit a file from the manifest using rename or, if available, resilient renaming.
+   * @param entry entry from manifest
+   * @throws PathIOException if the rename() call returned false and was uprated.
+   * @throws IOException failure
+   */
+  protected final CommitOutcome commitFile(FileEntry entry,
+      boolean deleteDest)
+      throws IOException {
+
+    final Path source = entry.getSourcePath();
+    final Path dest = entry.getDestPath();
+
+    maybeDeleteDest(deleteDest, dest);
+    if (storeSupportsResilientCommit()) {
+      // get the commit permits
+      final ManifestStoreOperations.CommitFileResult result = trackDuration(getIOStatistics(),
+          OP_COMMIT_FILE_RENAME, () ->
+              operations.commitFile(entry));
+      if (result.recovered()) {
+        // recovery took place.
+        getIOStatistics().incrementCounter(OP_COMMIT_FILE_RENAME_RECOVERED);
+      }
+      if (result.getWaitTime() != null) {
+        // note any delay which took place
+        noteAnyRateLimiting(STORE_IO_RATE_LIMITED, result.getWaitTime());
+      }
+    } else {
+      // commit with a simple rename; failures will be escalated.
+      executeRenamingOperation("renameFile", source, dest,
+          OP_COMMIT_FILE_RENAME, () ->
+              operations.renameFile(source, dest));
+    }
+    return new CommitOutcome();
+  }
+
+  /**
+   * Does this store support resilient commit.
+   * @return true if resilient commit operations are available.
+   */
+  protected boolean storeSupportsResilientCommit() {
+    return operations.storeSupportsResilientCommit();
+  }
+
+  private void maybeDeleteDest(final boolean deleteDest, final Path dest) throws IOException {
+    if (deleteDest) {
+      // delete the destination, always, knowing that it's a no-op if
+      // the data isn't there. Skipping the change saves one round trip
+      // to actually look for the file/object
+      boolean deleted = delete(dest, true);
+      // log the outcome in case of emergency diagnostics traces
+      // being needed.
+      LOG.debug("{}: delete('{}') returned {}'", getName(), dest, deleted);
+    }
+  }
+
+  /**
+   * Execute an operation to rename a file/dir, commit a manifest entry.
+   * The statistic is tracked; returning false from the operation is considered
+   * a failure from the statistics perspective.
+   * @param operation operation name
+   * @param source source path
+   * @param dest dest path
+   * @param statistic statistic to track
+   * @param action callable of the operation
+   * @throws IOException on any failure
+   */
+  private void executeRenamingOperation(String operation,
+      Path source,
+      Path dest,
+      String statistic,
+      CallableRaisingIOE<Boolean> action) throws IOException {
+
+    LOG.debug("{}: {} '{}' to '{}')", getName(), operation, source, dest);
+    requireNonNull(source, "Null source");
+    requireNonNull(dest, "Null dest");
+
+    // duration tracking is a bit convoluted as it
+    // ensures that rename failures as well as IOEs are
+    // treated as failures from a statistics perspective.
+
+    DurationTracker tracker = createTracker(getIOStatistics(), statistic);
+    boolean success;
+    try {
+      success = action.apply();
+      if (!success) {
+        // record failure in the tracker before closing it
+        tracker.failed();
+      }
+    } catch (IOException | RuntimeException e) {
+      LOG.info("{}: {} raised an exception: {}", getName(), operation, e.toString());
+      LOG.debug("{}: {} stack trace", getName(), operation, e);
+      tracker.failed();
+      throw e;
+    } finally {
+      // success
+      // update the tracker.
+      tracker.close();
+    }
+    // escalate the failure; this is done out of the duration tracker
+    // so its file status probes aren't included.
+    if (!success) {
+      throw escalateRenameFailure(operation, source, dest);
+    }
+  }
+
+  /**
+   * Escalate a rename failure to an exception.
+   * Returns an error exception to throw if one was not
+   * triggered when probing for the source.
+   * @param operation operation name
+   * @param source source path
+   * @param dest dest path
+   * @return an exception to throw
+   * @throws IOException raised probing for source or dest
+   */
+  private PathIOException escalateRenameFailure(String operation,
+      Path source, Path dest) throws IOException {
+    // rename just returned false.
+    // collect information for a meaningful error message
+    // and include in an exception raised.
+
+    // get the source status; this will implicitly raise a FNFE.
+    final FileStatus sourceStatus = getFileStatus(source);
+
+    // and look to see if there is anything at the destination
+    final FileStatus destStatus = getFileStatusOrNull(dest);
+
+    LOG.error("{}: failure to {} {} to {} with" +
+            " source status {} " +
+            " and destination status {}",
+        getName(), operation, source, dest,
+        sourceStatus, destStatus);
+
+    return new PathIOException(source.toString(),
+        FAILED_TO_RENAME_PREFIX + operation + " to " + dest);
+  }
+
+  /**
+   * Outcome from the commit.
+   */
+  public static final class CommitOutcome {
+
+  }
+
+  /**
+   * Job ID: never null.
+   */
+  protected final String getJobId() {
+    return stageConfig.getJobId();
+  }
+
+  /**
+   * Job attempt number.
+   */
+  protected final int getJobAttemptNumber() {
+    return stageConfig.getJobAttemptNumber();
+  }
+
+  /**
+   * ID of the task.
+   */
+  protected final String getTaskId() {
+    return stageConfig.getTaskId();
+  }
+
+  /**
+   * Get the task ID; raise an NPE
+   * if it is null.
+   * @return a non-null task ID.
+   */
+  protected final String getRequiredTaskId() {
+    return requireNonNull(getTaskId(),
+        "No Task ID in stage config");
+  }
+
+  /**
+   * ID of this specific attempt at a task.
+   */
+  protected final String getTaskAttemptId() {
+    return stageConfig.getTaskAttemptId();
+  }
+
+  /**
+   * Get the task attempt ID; raise an NPE
+   * if it is null.
+   * @return a non-null task attempt ID.
+   */
+  protected final String getRequiredTaskAttemptId() {
+    return requireNonNull(getTaskAttemptId(),
+        "No Task Attempt ID in stage config");
+  }
+
+  /**
+   * Job attempt dir.
+   */
+  protected final Path getJobAttemptDir() {
+    return stageConfig.getJobAttemptDir();
+  }
+
+  /**
+   * Directory to put task manifests into.
+   * @return a path under the job attempt dir.
+   */
+  protected final Path getTaskManifestDir() {
+    return stageConfig.getTaskManifestDir();
+  }
+
+
+  /**
+   * Task attempt dir.
+   */
+  protected final Path getTaskAttemptDir() {
+    return stageConfig.getTaskAttemptDir();
+  }
+
+  /**
+   * Get the task attemptDir; raise an NPE
+   * if it is null.
+   * @return a non-null task attempt dir.
+   */
+  protected final Path getRequiredTaskAttemptDir() {
+    return requireNonNull(getTaskAttemptDir(),
+        "No Task Attempt Dir");
+  }
+
+  /**
+   * Destination of job.
+   */
+  protected final Path getDestinationDir() {
+    return stageConfig.getDestinationDir();
+  }
+
+  /**
+   * Stage confog name, for logging.
+   * @return name.
+   */
+  public final String getName() {
+    return name;
+  }
+
+  /**
+   * Submitter for doing IO against the store other than
+   * manifest processing.
+   */
+  protected final TaskPool.Submitter getIOProcessors() {
+    return ioProcessors;
+  }
+
+  /**
+   * Submitter for doing IO against the store other than
+   * manifest processing.
+   * The size parameter is used to select between sequential
+   * and parallel runners.
+   * no data, or one entry: serial.
+   * everything else, parallel.
+   * @param size number of items.
+   * @return a submitter or null
+   */
+  protected final TaskPool.Submitter getIOProcessors(int size) {
+    return size > 1
+        ? getIOProcessors()
+        : null;
+  }
+
+  /**
+   * Delete a directory, possibly suppressing exceptions.
+   * @param dir directory.
+   * @param suppressExceptions should exceptions be suppressed?
+   * @throws IOException exceptions raised in delete if not suppressed.
+   * @return any exception caught and suppressed
+   */
+  protected IOException deleteDir(
+      final Path dir,
+      final Boolean suppressExceptions)
+      throws IOException {
+    try {
+      delete(dir, true);
+      return null;
+    } catch (IOException ex) {
+      LOG.info("Error deleting {}: {}", dir, ex.toString());
+      if (!suppressExceptions) {
+        throw ex;
+      } else {
+        return ex;
+      }
+    }
+  }
+
+  /**
+   * Create an entry for a file to rename under the destination.
+   * If the store operations supports extracting etags from file status
+   * entries, that is included in the entry
+   * @param status source file
+   * @param destDir destination directory
+   * @return an entry which includes the rename path
+   */
+  protected FileEntry fileEntry(FileStatus status, Path destDir) {
+    // generate a new path under the dest dir
+    Path dest = new Path(destDir, status.getPath().getName());
+    return new FileEntry(status.getPath(),
+        dest,
+        status.getLen(),
+        getEtag(status));
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java
new file mode 100644
index 0000000..4a5971a
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java
@@ -0,0 +1,511 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.util.DurationInfo;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED;
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT;
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED;
+import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_CLEANUP_PARALLEL_DELETE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP;
+
+/**
+ * Clean up a job's temporary directory through parallel delete,
+ * base _temporary delete.
+ * Returns: the outcome of the overall operation
+ * The result is detailed purely for the benefit of tests, which need
+ * to make assertions about error handling and fallbacks.
+ *
+ * There's a few known issues with the azure and GCS stores which
+ * this stage tries to address.
+ * - Google GCS directory deletion is O(entries), so is slower for big jobs.
+ * - Azure storage directory delete, when using OAuth authentication or
+ *   when not the store owner triggers a scan down the tree to verify the
+ *   caller has the permission to delete each subdir.
+ *   If this scan takes over 90s, the operation can time out.
+ *
+ * The main solution for both of these is that task attempts are
+ * deleted in parallel, in different threads.
+ * This will speed up GCS cleanup and reduce the risk of
+ * abfs related timeouts.
+ * Exceptions during cleanup can be suppressed,
+ * so that these do not cause the job to fail.
+ *
+ * Also, some users want to be able to run multiple independent jobs
+ * targeting the same output directory simultaneously.
+ * If one job deletes the directory `__temporary` all the others
+ * will fail.
+ *
+ * This can be addressed by disabling cleanup entirely.
+ *
+ */
+public class CleanupJobStage extends
+    AbstractJobOrTaskStage<
+            CleanupJobStage.Arguments,
+            CleanupJobStage.Result> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      CleanupJobStage.class);
+
+  /**
+   * Count of deleted directories.
+   */
+  private final AtomicInteger deleteDirCount = new AtomicInteger();
+
+  /**
+   * Count of delete failures.
+   */
+  private final AtomicInteger deleteFailureCount = new AtomicInteger();
+
+  /**
+   * Last delete exception; non null if deleteFailureCount is not zero.
+   */
+  private IOException lastDeleteException;
+
+  /**
+   * Stage name as passed in from arguments.
+   */
+  private String stageName = OP_STAGE_JOB_CLEANUP;
+
+  public CleanupJobStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_CLEANUP, true);
+  }
+
+  /**
+   * Statistic name is extracted from the arguments.
+   * @param arguments args to the invocation.
+   * @return stage name.
+   */
+  @Override
+  protected String getStageStatisticName(Arguments arguments) {
+    return arguments.statisticName;
+  }
+
+  /**
+   * Clean up the job attempt directory tree.
+   * @param args arguments built up.
+   * @return the result.
+   * @throws IOException failure was raised an exceptions weren't surpressed.
+   */
+  @Override
+  protected Result executeStage(
+      final Arguments args)
+      throws IOException {
+    stageName = getStageName(args);
+    // this is $dest/_temporary
+    final Path baseDir = requireNonNull(getStageConfig().getOutputTempSubDir());
+    LOG.debug("{}: Cleaup of directory {} with {}", getName(), baseDir, args);
+    if (!args.enabled) {
+      LOG.info("{}: Cleanup of {} disabled", getName(), baseDir);
+      return new Result(Outcome.DISABLED, baseDir,
+          0, null);
+    }
+    // shortcut of a single existence check before anything else
+    if (getFileStatusOrNull(baseDir) == null) {
+      return new Result(Outcome.NOTHING_TO_CLEAN_UP,
+          baseDir,
+          0, null);
+    }
+
+    Outcome outcome = null;
+    IOException exception;
+
+
+    // to delete.
+    LOG.info("{}: Deleting job directory {}", getName(), baseDir);
+
+    if (args.deleteTaskAttemptDirsInParallel) {
+      // Attempt to do a parallel delete of task attempt dirs;
+      // don't overreact if a delete fails, but stop trying
+      // to delete the others, and fall back to deleting the
+      // job dir.
+      Path taskSubDir
+          = getStageConfig().getJobAttemptTaskSubDir();
+      try (DurationInfo info = new DurationInfo(LOG,
+          "parallel deletion of task attempts in %s",
+          taskSubDir)) {
+        RemoteIterator<FileStatus> dirs =
+            RemoteIterators.filteringRemoteIterator(
+                listStatusIterator(taskSubDir),
+                FileStatus::isDirectory);
+        TaskPool.foreach(dirs)
+            .executeWith(getIOProcessors())
+            .stopOnFailure()
+            .suppressExceptions(false)
+            .run(this::rmTaskAttemptDir);
+        getIOStatistics().aggregate((retrieveIOStatistics(dirs)));
+
+        if (getLastDeleteException() != null) {
+          // one of the task attempts failed.
+          throw getLastDeleteException();
+        }
+        // success: record this as the outcome.
+        outcome = Outcome.PARALLEL_DELETE;
+      } catch (FileNotFoundException ex) {
+        // not a problem if there's no dir to list.
+        LOG.debug("{}: Task attempt dir {} not found", getName(), taskSubDir);
+        outcome = Outcome.DELETED;
+      } catch (IOException ex) {
+        // failure. Log and continue
+        LOG.info(
+            "{}: Exception while listing/deleting task attempts under {}; continuing",
+            getName(),
+            taskSubDir, ex);
+        // not overreacting here as the base delete will still get executing
+        outcome = Outcome.DELETED;
+      }
+    }
+    // Now the top-level deletion; exception gets saved
+    exception = deleteOneDir(baseDir);
+    if (exception != null) {
+      // failure, report and continue
+      // assume failure.
+      outcome = Outcome.FAILURE;
+    } else {
+      // if the outcome isn't already recorded as parallel delete,
+      // mark is a simple delete.
+      if (outcome == null) {
+        outcome = Outcome.DELETED;
+      }
+    }
+
+    Result result = new Result(
+        outcome,
+        baseDir,
+        deleteDirCount.get(),
+        exception);
+    if (!result.succeeded() && !args.suppressExceptions) {
+      result.maybeRethrowException();
+    }
+
+    return result;
+  }
+
+  /**
+   * Delete a single TA dir in a parallel task.
+   * Updates the audit context.
+   * Exceptions are swallowed so that attempts are still made
+   * to delete the others, but the first exception
+   * caught is saved in a field which can be retrieved
+   * via {@link #getLastDeleteException()}.
+   *
+   * @param status dir to be deleted.
+   * @throws IOException delete failure.
+   */
+  private void rmTaskAttemptDir(FileStatus status) throws IOException {
+    // stage name in audit context is the one set in the arguments.
+    updateAuditContext(stageName);
+    // update the progress callback in case delete is really slow.
+    progress();
+    deleteOneDir(status.getPath());
+  }
+
+  /**
+   * Delete a directory.
+   * The {@link #deleteFailureCount} counter.
+   * is incremented on every failure.
+   * @param dir directory
+   * @throws IOException if an IOE was raised
+   * @return any IOE raised.
+   */
+  private IOException deleteOneDir(final Path dir)
+      throws IOException {
+
+    deleteDirCount.incrementAndGet();
+    IOException ex = deleteDir(dir, true);
+    if (ex != null) {
+      deleteFailure(ex);
+    }
+    return ex;
+  }
+
+  /**
+   * Note a failure.
+   * @param ex exception
+   */
+  private synchronized void deleteFailure(IOException ex) {
+    // excaption: add the count
+    deleteFailureCount.incrementAndGet();
+    lastDeleteException = ex;
+  }
+
+  /**
+   * Get the last delete exception; synchronized.
+   * @return the last delete exception or null.
+   */
+  public synchronized IOException getLastDeleteException() {
+    return lastDeleteException;
+  }
+
+  /**
+   * Options to pass down to the cleanup stage.
+   */
+  public static final class Arguments {
+
+    /**
+     * Statistic to update.
+     */
+    private final String statisticName;
+
+    /** Delete is enabled? */
+    private final boolean enabled;
+
+    /** Attempt parallel delete of task attempt dirs? */
+    private final boolean deleteTaskAttemptDirsInParallel;
+
+    /** Ignore failures? */
+    private final boolean suppressExceptions;
+
+    /**
+     * Arguments to the stage.
+     * @param statisticName stage name to report
+     * @param enabled is the stage enabled?
+     * @param deleteTaskAttemptDirsInParallel delete task attempt dirs in
+     *        parallel?
+     * @param suppressExceptions suppress exceptions?
+     */
+    public Arguments(
+        final String statisticName,
+        final boolean enabled,
+        final boolean deleteTaskAttemptDirsInParallel,
+        final boolean suppressExceptions) {
+      this.statisticName = statisticName;
+      this.enabled = enabled;
+      this.deleteTaskAttemptDirsInParallel = deleteTaskAttemptDirsInParallel;
+      this.suppressExceptions = suppressExceptions;
+    }
+
+    public String getStatisticName() {
+      return statisticName;
+    }
+
+    public boolean isEnabled() {
+      return enabled;
+    }
+
+    public boolean isDeleteTaskAttemptDirsInParallel() {
+      return deleteTaskAttemptDirsInParallel;
+    }
+
+    public boolean isSuppressExceptions() {
+      return suppressExceptions;
+    }
+
+    @Override
+    public String toString() {
+      return "Arguments{" +
+          "statisticName='" + statisticName + '\''
+          + ", enabled=" + enabled
+          + ", deleteTaskAttemptDirsInParallel="
+          + deleteTaskAttemptDirsInParallel
+          + ", suppressExceptions=" + suppressExceptions
+          + '}';
+    }
+  }
+
+  /**
+   * Static disabled arguments.
+   */
+  public static final Arguments DISABLED = new Arguments(OP_STAGE_JOB_CLEANUP,
+      false,
+      false,
+      false
+  );
+
+  /**
+   * Build an options argument from a configuration, using the
+   * settings from FileOutputCommitter and manifest committer.
+   * @param statisticName statistic name to use in duration tracking.
+   * @param conf configuration to use.
+   * @return the options to process
+   */
+  public static Arguments cleanupStageOptionsFromConfig(
+      String statisticName, Configuration conf) {
+
+    boolean enabled = !conf.getBoolean(FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED,
+        FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT);
+    boolean suppressExceptions = conf.getBoolean(
+        FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED,
+        FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT);
+    boolean deleteTaskAttemptDirsInParallel = conf.getBoolean(
+        OPT_CLEANUP_PARALLEL_DELETE,
+        OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT);
+    return new Arguments(
+        statisticName,
+        enabled,
+        deleteTaskAttemptDirsInParallel,
+        suppressExceptions
+    );
+  }
+
+  /**
+   * Enum of outcomes.
+   */
+  public enum Outcome {
+    DISABLED("Disabled", false),
+    NOTHING_TO_CLEAN_UP("Nothing to clean up", true),
+    PARALLEL_DELETE("Parallel Delete of Task Attempt Directories", true),
+    DELETED("Delete of job directory", true),
+    FAILURE("Delete failed", false);
+
+    private final String description;
+
+    private final boolean success;
+
+    Outcome(String description, boolean success) {
+      this.description = description;
+      this.success = success;
+    }
+
+    @Override
+    public String toString() {
+      return "Outcome{" + name() +
+          " '" + description + '\'' +
+          "}";
+    }
+
+    /**
+     * description.
+     * @return text for logging
+     */
+    public String getDescription() {
+      return description;
+    }
+
+    /**
+     * Was this a success?
+     * @return true if this outcome is good.
+     */
+    public boolean isSuccess() {
+      return success;
+    }
+  }
+
+  /**
+   * Result of the cleanup.
+   * If the outcome == FAILURE but exceptions were suppressed
+   * (which they are implicitly if an instance of this object
+   * is created and returned), then the exception
+   * MUST NOT be null.
+   */
+  public static final class Result {
+
+    /** Outcome. */
+    private final Outcome outcome;
+
+    /** Directory cleaned up. */
+    private final Path directory;
+
+    /**
+     * Number of delete calls made across all threads.
+     */
+    private final int deleteCalls;
+
+    /**
+     * Any IOE raised.
+     */
+    private final IOException exception;
+
+    public Result(
+        final Outcome outcome,
+        final Path directory,
+        final int deleteCalls,
+        IOException exception) {
+      this.outcome = requireNonNull(outcome, "outcome");
+      this.directory = directory;
+      this.deleteCalls = deleteCalls;
+      this.exception = exception;
+      if (outcome == Outcome.FAILURE) {
+        requireNonNull(exception, "No exception in failure result");
+      }
+    }
+
+    public Path getDirectory() {
+      return directory;
+    }
+
+    public boolean wasExecuted() {
+      return outcome != Outcome.DISABLED;
+    }
+
+    /**
+     * Was the outcome a success?
+     * That is: either the dir wasn't there or through
+     * delete/rename it is no longer there.
+     * @return true if the temporary dir no longer exists.
+     */
+    public boolean succeeded() {
+      return outcome.isSuccess();
+    }
+
+    public Outcome getOutcome() {
+      return outcome;
+    }
+
+    public int getDeleteCalls() {
+      return deleteCalls;
+    }
+
+    public IOException getException() {
+      return exception;
+    }
+
+    /**
+     * If there was an IOE caught, throw it.
+     * For ease of use in (meaningful) lambda expressions
+     * in tests, returns the string value if there
+     * was no exception to throw (for use in tests)
+     * @throws IOException exception.
+     */
+    public String maybeRethrowException() throws IOException {
+      if (exception != null) {
+        throw exception;
+      }
+      return toString();
+    }
+
+    @Override
+    public String toString() {
+      return "CleanupResult{" +
+          "outcome=" + outcome +
+          ", directory=" + directory +
+          ", deleteCalls=" + deleteCalls +
+          ", exception=" + exception +
+          '}';
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java
new file mode 100644
index 0000000..a754f2a
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java
@@ -0,0 +1,245 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.io.FileUtils.byteCountToDisplaySize;
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.MANIFESTS;
+
+/**
+ * Commit the Job.
+ * Arguments (save manifest, validate output)
+ * Inputs: saveMarker: boolean, validateOutput: boolean
+ * Outputs: SuccessData
+ */
+public class CommitJobStage extends
+    AbstractJobOrTaskStage<
+            CommitJobStage.Arguments,
+            CommitJobStage.Result> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      CommitJobStage.class);
+
+  public CommitJobStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_COMMIT, true);
+  }
+
+  @Override
+  protected CommitJobStage.Result executeStage(
+      final CommitJobStage.Arguments arguments) throws IOException {
+
+    LOG.info("{}: Committing job \"{}\". resilient commit supported = {}",
+        getName(),
+        getJobId(),
+        storeSupportsResilientCommit());
+
+    boolean createMarker = arguments.isCreateMarker();
+
+    // load the manifests
+    final StageConfig stageConfig = getStageConfig();
+    LoadManifestsStage.Result result
+        = new LoadManifestsStage(stageConfig).apply(true);
+    List<TaskManifest> manifests = result.getManifests();
+    LoadManifestsStage.SummaryInfo summary = result.getSummary();
+
+    LOG.debug("{}: Job Summary {}", getName(), summary);
+    LOG.info("{}: Committing job with file count: {}; total size {} bytes",
+        getName(),
+        summary.getFileCount(),
+        byteCountToDisplaySize(summary.getTotalFileSize()));
+
+
+    // add in the manifest statistics to our local IOStatistics for
+    // reporting.
+    IOStatisticsStore iostats = getIOStatistics();
+    iostats.aggregate(summary.getIOStatistics());
+
+    // prepare destination directories.
+    final CreateOutputDirectoriesStage.Result dirStageResults =
+        new CreateOutputDirectoriesStage(stageConfig)
+            .apply(manifests);
+
+    // commit all the tasks.
+    // The success data includes a snapshot of the IO Statistics
+    // and hence all aggregate stats from the tasks.
+    ManifestSuccessData successData;
+    successData = new RenameFilesStage(stageConfig).apply(
+        Pair.of(manifests, dirStageResults.getCreatedDirectories()));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson());
+    }
+    // update the counter of bytes committed and files.
+    // use setCounter so as to ignore any values accumulated when
+    // aggregating tasks.
+    iostats.setCounter(
+        COMMITTER_FILES_COMMITTED_COUNT,
+        summary.getFileCount());
+    iostats.setCounter(
+        COMMITTER_BYTES_COMMITTED_COUNT,
+        summary.getTotalFileSize());
+    successData.snapshotIOStatistics(iostats);
+
+
+    // rename manifests. Only warn on failure here.
+    final String manifestRenameDir = arguments.getManifestRenameDir();
+    if (isNotBlank(manifestRenameDir)) {
+      Path manifestRenamePath = new Path(
+          new Path(manifestRenameDir),
+          getJobId());
+      LOG.info("{}: Renaming manifests to {}", getName(), manifestRenamePath);
+      try {
+        renameDir(getTaskManifestDir(), manifestRenamePath);
+
+        // save this path in the summary diagnostics
+        successData.getDiagnostics().put(MANIFESTS, manifestRenamePath.toUri().toString());
+      } catch (IOException | IllegalArgumentException e) {
+        // rename failure, including path for wrong filesystem
+        LOG.warn("{}: Failed to rename manifests to {}", getName(), manifestRenamePath, e);
+      }
+    }
+
+    // save the _SUCCESS if the option is enabled.
+    Path successPath = null;
+    if (createMarker) {
+      // save a snapshot of the IO Statistics
+
+      successPath = new SaveSuccessFileStage(stageConfig)
+          .apply(successData);
+      LOG.debug("{}: Saving _SUCCESS file to {}", getName(), successPath);
+    }
+
+    // optional cleanup
+    new CleanupJobStage(stageConfig).apply(arguments.getCleanupArguments());
+
+    // and then, after everything else: optionally validate.
+    if (arguments.isValidateOutput()) {
+      // cache and restore the active stage field
+      LOG.info("{}: Validating output.", getName());
+      new ValidateRenamedFilesStage(stageConfig)
+          .apply(result.getManifests());
+    }
+
+    // restore the active stage so that when the report is saved
+    // it is declared as job commit, not cleanup or validate.
+    stageConfig.enterStage(getStageName(arguments));
+
+    // the result
+    return new CommitJobStage.Result(successPath, successData);
+  }
+
+  /**
+   * Arguments for job commit.
+   */
+  public static final class Arguments {
+
+    /** create the _SUCCESS marker? */
+    private final boolean createMarker;
+
+    /** perform validation checks on the files? */
+    private final boolean validateOutput;
+
+    /** optional directory to rename the task manifests to. */
+    private final String manifestRenameDir;
+
+    /** cleanup arguments.. */
+    private final CleanupJobStage.Arguments cleanupArguments;
+
+    /**
+     *
+     * @param createMarker create the _SUCCESS marker?
+     * @param validateOutput perform validation checks on the files?
+     * @param manifestRenameDir optional directory to rename the task manifests to
+     * @param cleanupArguments cleanup arguments.
+     */
+    public Arguments(
+        boolean createMarker,
+        boolean validateOutput,
+        @Nullable String manifestRenameDir,
+        CleanupJobStage.Arguments cleanupArguments) {
+
+      this.createMarker = createMarker;
+      this.validateOutput = validateOutput;
+      this.manifestRenameDir = manifestRenameDir;
+      this.cleanupArguments = requireNonNull(cleanupArguments);
+    }
+
+    public boolean isCreateMarker() {
+      return createMarker;
+    }
+
+    public boolean isValidateOutput() {
+      return validateOutput;
+    }
+
+    public String getManifestRenameDir() {
+      return manifestRenameDir;
+    }
+
+    public CleanupJobStage.Arguments getCleanupArguments() {
+      return cleanupArguments;
+    }
+  }
+
+  /**
+   * Result of the stage.
+   */
+  public static final class Result {
+    /**
+     * Manifest success data.
+     */
+    private final ManifestSuccessData jobSuccessData;
+
+    /**
+     * Success file path. null if not saved.
+     */
+    private final Path successPath;
+
+    public Result(final Path successPath,
+        ManifestSuccessData jobSuccessData) {
+      this.successPath = successPath;
+      this.jobSuccessData = jobSuccessData;
+    }
+
+    public ManifestSuccessData getJobSuccessData() {
+      return jobSuccessData;
+    }
+
+    public Path getSuccessPath() {
+      return successPath;
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java
new file mode 100644
index 0000000..bf5ba27
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_COMMIT;
+
+/**
+ * Commit a task attempt.
+ * Scan the task attempt directories through
+ * {@link TaskAttemptScanDirectoryStage}
+ * and then save to the task manifest path at
+ * {@link SaveTaskManifestStage}.
+ */
+
+public class CommitTaskStage extends
+    AbstractJobOrTaskStage<Void, CommitTaskStage.Result> {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      CommitTaskStage.class);
+
+  public CommitTaskStage(final StageConfig stageConfig) {
+    super(true, stageConfig, OP_STAGE_TASK_COMMIT, false);
+  }
+
+  /**
+   * Scan the task attempt dir then save the manifest.
+   * A snapshot of the IOStats will be included in the manifest;
+   * this includes the scan time.
+   * @param arguments arguments to the function.
+   * @return the path the manifest was saved to, and the manifest.
+   * @throws IOException IO failure.
+   */
+  @Override
+  protected CommitTaskStage.Result executeStage(final Void arguments)
+      throws IOException {
+    LOG.info("{}: Committing task \"{}\"", getName(), getTaskAttemptId());
+
+    // execute the scan
+    final TaskAttemptScanDirectoryStage scanStage =
+        new TaskAttemptScanDirectoryStage(getStageConfig());
+    TaskManifest manifest = scanStage.apply(arguments);
+
+    // add the scan as task commit. It's not quite, as it doesn't include
+    // the saving, but ...
+    scanStage.addExecutionDurationToStatistics(getIOStatistics(), OP_STAGE_TASK_COMMIT);
+
+    // save a snapshot of the IO Statistics
+    final IOStatisticsSnapshot manifestStats = snapshotIOStatistics();
+    manifestStats.aggregate(getIOStatistics());
+    manifest.setIOStatistics(manifestStats);
+
+    // Now save with rename
+    Path manifestPath = new SaveTaskManifestStage(getStageConfig())
+        .apply(manifest);
+    return new CommitTaskStage.Result(manifestPath, manifest);
+  }
+
+  /**
+   * Result of the stage.
+   */
+  public static final class Result {
+    /** The path the manifest was saved to. */
+    private final Path path;
+    /** The manifest. */
+    private final TaskManifest taskManifest;
+
+    public Result(Path path,
+        TaskManifest taskManifest) {
+      this.path = path;
+      this.taskManifest = taskManifest;
+    }
+
+    /**
+     * Get the manifest path.
+     * @return The path the manifest was saved to.
+     */
+    public Path getPath() {
+      return path;
+    }
+
+    /**
+     * Get the manifest.
+     * @return The manifest.
+     */
+    public TaskManifest getTaskManifest() {
+      return taskManifest;
+    }
+
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java
new file mode 100644
index 0000000..927004e
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java
@@ -0,0 +1,423 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_CREATE_DIRECTORIES;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_DELETE_FILE_UNDER_DESTINATION;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MKDIRS_RETURNED_FALSE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_PREPARE_DIR_ANCESTORS;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CREATE_TARGET_DIRS;
+import static org.apache.hadoop.util.OperationDuration.humanTime;
+
+/**
+ * Prepare the destination directory tree, as efficiently as possible.
+ * possible -and doing those IO operations in the thread pool.
+ *
+ * The classic FileOutputCommitter does a recursive treewalk and
+ * deletes any files found at paths where directories are to be created.
+ *
+ * Each task manifest's directories are combined with those of the other tasks
+ * to build a set of all directories which are needed, without duplicates.
+ *
+ * This stage requires the aggregate set of manifests to contain
+ * all directories to create, including level,
+ * and expects them to have been probed for existence/state.
+ *
+ * For each level, all dirs are processed in parallel to
+ * be created or, if files, deleted.
+ *
+ * The stage returns the list of directories created, and for testing,
+ * the map of paths to outcomes.
+ *
+ * Directory creation can be surprisingly slow against object stores,
+ * do use benchmarks from real test runs when tuning this algorithm.
+ */
+public class CreateOutputDirectoriesStage extends
+    AbstractJobOrTaskStage<
+        List<TaskManifest>,
+        CreateOutputDirectoriesStage.Result> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      CreateOutputDirectoriesStage.class);
+
+  /**
+   * Directories as a map of (path, path).
+   * Using a map rather than any set for efficient concurrency; the
+   * concurrent sets don't do lookups so fast.
+   */
+  private final Map<Path, DirMapState> dirMap = new ConcurrentHashMap<>();
+
+  /**
+   * A list of created paths for the results.
+   */
+  private final List<Path> createdDirectories = new ArrayList<>();
+
+  public CreateOutputDirectoriesStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_CREATE_TARGET_DIRS, true);
+    // add the dest dir to the dir map as we expect the job setup to create it.
+    dirMap.put(getDestinationDir(), DirMapState.dirWasCreated);
+  }
+
+  @Override
+  protected Result executeStage(
+      final List<TaskManifest> taskManifests)
+      throws IOException {
+
+    final List<Path> directories = createAllDirectories(taskManifests);
+    LOG.debug("{}: Created {} directories", getName(), directories.size());
+    return new Result(new HashSet<>(directories), dirMap);
+  }
+
+  /**
+   * For each task, build the list of directories it wants.
+   * @param taskManifests task manifests
+   * @return the list of paths which have been created.
+   */
+  private List<Path> createAllDirectories(final List<TaskManifest> taskManifests)
+      throws IOException {
+
+    // all directories which need to exist across all
+    // tasks.
+    // leaf directories
+    final Map<Path, DirEntry> leaves = new HashMap<>();
+    // parent directories. these do not need to be
+    // explicitly created.
+    final Map<Path, DirEntry> parents = new HashMap<>();
+    // the files which must be deleted as a directory
+    // will be created at that path.
+    final Set<Path> filesToDelete = new HashSet<>();
+
+    // iterate through the task manifests
+    // and all output dirs into the set of dirs to
+    // create.
+    // hopefully there is a lot of overlap, so the
+    // final number of dirs to create is small.
+    for (TaskManifest task: taskManifests) {
+      final List<DirEntry> destDirectories = task.getDestDirectories();
+      Collections.sort(destDirectories, (o1, o2) ->
+          o1.getLevel() - o2.getLevel());
+      for (DirEntry entry: destDirectories) {
+        // add the dest entry
+        final Path path = entry.getDestPath();
+        if (!leaves.containsKey(path)) {
+          leaves.put(path, entry);
+
+          // if it is a file to delete, record this.
+          if (entry.getStatus() == EntryStatus.file) {
+            filesToDelete.add(path);
+          }
+          final Path parent = path.getParent();
+          if (parent != null && leaves.containsKey(parent)) {
+            // there's a parent dir, move it from the leaf list
+            // to parent list
+            parents.put(parent,
+                leaves.remove(parent));
+          }
+        }
+      }
+    }
+
+    // at this point then there is a map of all directories which
+    // are leaf entries and so need to be created if not present,
+    // and the maximum level is known.
+    // we can iterate through all levels deleting any files if there are any.
+
+    // Prepare parent directories.
+    deleteFiles(filesToDelete);
+
+    // Now the real work.
+    final int createCount = leaves.size();
+    LOG.info("Preparing {} directory/directories", createCount);
+    // now probe for and create the leaf dirs, which are those at the
+    // bottom level
+    Duration d = measureDurationOfInvocation(getIOStatistics(), OP_CREATE_DIRECTORIES, () ->
+        TaskPool.foreach(leaves.values())
+            .executeWith(getIOProcessors(createCount))
+            .onFailure(this::reportMkDirFailure)
+            .stopOnFailure()
+            .run(this::createOneDirectory));
+    LOG.info("Time to prepare directories {}", humanTime(d.toMillis()));
+    return createdDirectories;
+  }
+
+  /**
+   * How many failures have been reported.
+   */
+  private final AtomicInteger failureCount = new AtomicInteger();
+
+  /**
+   * report a single directory failure.
+   * @param path path which could not be deleted
+   * @param e exception raised.
+   */
+  private void reportMkDirFailure(DirEntry dirEntry, Exception e) {
+    Path path = dirEntry.getDestPath();
+    final int count = failureCount.incrementAndGet();
+    LOG.warn("{}: mkdir failure #{} Failed to create directory \"{}\": {}",
+        getName(), count, path, e.toString());
+    LOG.debug("{}: Full exception details",
+        getName(), e);
+  }
+
+  /**
+   * Delete all directories where there is a file.
+   * @param filesToDelete set of dirs to where there is a file.
+   * @throws IOException IO problem
+   */
+  private void deleteFiles(final Set<Path> filesToDelete)
+      throws IOException {
+
+    final int size = filesToDelete.size();
+    if (size == 0) {
+      // nothing to delete.
+      return;
+    }
+    LOG.info("{}: Directory entries containing files to delete: {}", getName(), size);
+    Duration d = measureDurationOfInvocation(getIOStatistics(),
+        OP_PREPARE_DIR_ANCESTORS, () ->
+            TaskPool.foreach(filesToDelete)
+                .executeWith(getIOProcessors(size))
+                .stopOnFailure()
+                .run(dir -> {
+                  updateAuditContext(OP_PREPARE_DIR_ANCESTORS);
+                  deleteDirWithFile(dir);
+                }));
+    LOG.info("Time to delete files {}", humanTime(d.toMillis()));
+  }
+
+  /**
+   * Prepare a parent directory.
+   * @param dir directory to probe
+   * @throws IOException failure in probe other than FNFE
+   */
+  private void deleteDirWithFile(Path dir) throws IOException {
+    // report progress back
+    progress();
+    LOG.info("{}: Deleting file {}", getName(), dir);
+    delete(dir, false, OP_DELETE);
+    // note its final state
+    addToDirectoryMap(dir, DirMapState.fileNowDeleted);
+  }
+
+
+  /**
+   * Create a directory is required, updating the directory map
+   * and, if the operation took place, the list of created dirs.
+   * Reports progress on invocation.
+   * @param dirEntry entry
+   * @throws IOException failure.
+   */
+  private void createOneDirectory(final DirEntry dirEntry) throws IOException {
+    // report progress back
+    progress();
+    final Path dir = dirEntry.getDestPath();
+    updateAuditContext(OP_STAGE_JOB_CREATE_TARGET_DIRS);
+    final DirMapState state = maybeCreateOneDirectory(dirEntry);
+    switch (state) {
+    case dirFoundInStore:
+      addToDirectoryMap(dir, state);
+      break;
+    case dirWasCreated:
+    case dirCreatedOnSecondAttempt:
+      addCreatedDirectory(dir);
+      addToDirectoryMap(dir, state);
+      break;
+    default:
+      break;
+    }
+
+  }
+
+
+  /**
+   * Try to efficiently and robustly create a directory in a method which is
+   * expected to be executed in parallel with operations creating
+   * peer directories.
+   * @param path path to create
+   * @return true if dir created/found
+   * @throws IOException IO Failure.
+   */
+  private DirMapState maybeCreateOneDirectory(DirEntry dirEntry) throws IOException {
+    final EntryStatus status = dirEntry.getStatus();
+    if (status == EntryStatus.dir) {
+      return DirMapState.dirFoundInStore;
+    }
+    // present in case directories are ever created in task commits
+    if (status == EntryStatus.created_dir) {
+      return DirMapState.dirWasCreated;
+    }
+
+    // here the dir doesn't exist because
+    // it was a file and has been deleted, or
+    // checks failed. create it.
+    final Path path = dirEntry.getDestPath();
+
+    LOG.info("Creating directory {}", path);
+
+    try {
+      if (mkdirs(path, false)) {
+        // success -return immediately.
+        return DirMapState.dirWasCreated;
+      }
+      getIOStatistics().incrementCounter(OP_MKDIRS_RETURNED_FALSE);
+
+      LOG.info("{}: mkdirs({}) returned false, attempting to recover",
+          getName(), path);
+    } catch (IOException e) {
+      // can be caused by file existing, etc.
+      LOG.info("{}: mkdir({}) raised exception {}", getName(), path, e.toString());
+      LOG.debug("{}: Mkdir stack", getName(), e);
+    }
+
+    // fallback to checking the FS, in case a different process did it.
+    FileStatus st = getFileStatusOrNull(path);
+    if (st != null) {
+      if (!st.isDirectory()) {
+        // is bad: delete a file
+        LOG.info("{}: Deleting file where a directory should go: {}",
+            getName(), st);
+        delete(path, false, OP_DELETE_FILE_UNDER_DESTINATION);
+      } else {
+        // is good.
+        LOG.warn("{}: Even though mkdirs({}) failed, there is now a directory there",
+            getName(), path);
+        return DirMapState.dirFoundInStore;
+      }
+    } else {
+      // nothing found. This should never happen.
+      LOG.warn("{}: Although mkdirs({}) returned false, there's nothing at that path to prevent it",
+          getName(), path);
+
+    }
+
+    // try to create the directory again
+    // if this fails, and IOE is still raised, that
+    // propagate to the caller.
+    if (!mkdirs(path, false)) {
+
+      // mkdirs failed again
+      getIOStatistics().incrementCounter(OP_MKDIRS_RETURNED_FALSE);
+
+      // require the dir to exist, raising an exception if it does not.
+      directoryMustExist("Creating directory ", path);
+    }
+
+    // we only get here if the second attempt recovered
+    return DirMapState.dirCreatedOnSecondAttempt;
+
+  }
+
+  /**
+   * Add a created dir to the list of created dirs.
+   * @param dir new dir.
+   */
+  private void addCreatedDirectory(final Path dir) {
+    synchronized (createdDirectories) {
+      createdDirectories.add(dir);
+    }
+  }
+
+  /**
+   * Add a dir  to the directory map if there is not already an entry there.
+   * @param dir directory.
+   * @param state state of entry
+   */
+  private void addToDirectoryMap(final Path dir,
+      DirMapState state) {
+    if (!dirMap.containsKey(dir)) {
+      dirMap.put(dir, state);
+    }
+  }
+
+
+  /**
+   * Result of the operation.
+   */
+  public static final class Result {
+
+    /** directories created. */
+    private final Set<Path> createdDirectories;
+
+    /**
+     * Map of dirs built up during preparation.
+     */
+    private final Map<Path, DirMapState> dirMap;
+
+    public Result(Set<Path> createdDirectories,
+        Map<Path, DirMapState> dirMap) {
+      this.createdDirectories = requireNonNull(createdDirectories);
+      this.dirMap = requireNonNull(dirMap);
+    }
+
+    public Set<Path> getCreatedDirectories() {
+      return createdDirectories;
+    }
+
+    public Map<Path, DirMapState> getDirMap() {
+      return dirMap;
+    }
+
+    @Override
+    public String toString() {
+      return "Result{" +
+          "directory count=" + createdDirectories.size() +
+          '}';
+    }
+  }
+
+  /**
+   * Enumeration of dir states in the dir map.
+   */
+  public enum DirMapState {
+    dirFoundInStore,
+    dirFoundInMap,
+    dirWasCreated,
+    dirCreatedOnSecondAttempt,
+    fileNowDeleted,
+    ancestorWasDirOrMissing,
+    parentWasNotFile,
+    parentOfCreatedDir
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java
new file mode 100644
index 0000000..98975bf
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java
@@ -0,0 +1,33 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.util.functional.FunctionRaisingIOE;
+
+/**
+ * Job Stage is a function raising an IOE. and which also
+ * provides IOStats.
+ * @param <IN> Type of arguments to the stage.
+ * @param <OUT> Type of result.
+ */
+public interface JobOrTaskStage<IN, OUT> extends FunctionRaisingIOE<IN, OUT>,
+    IOStatisticsSource {
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java
new file mode 100644
index 0000000..3720f54
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java
@@ -0,0 +1,293 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static org.apache.commons.io.FileUtils.byteCountToDisplaySize;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_MANIFEST_FILE_SIZE;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_ALL_MANIFESTS;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_LOAD_MANIFESTS;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics;
+
+/**
+ * Stage to load all the task manifests in the job attempt directory.
+ * Invoked in Job Commit.
+ * Manifests are loaded in parallel.
+ * The IOStatistics snapshot passed in is built up with the statistics
+ * and the statistics stripped from the manifest if prune == true.
+ * This keeps the memory footprint of each manifest down.
+ */
+public class LoadManifestsStage extends
+    AbstractJobOrTaskStage<
+        Boolean,
+        LoadManifestsStage.Result> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      LoadManifestsStage.class);
+
+  /**
+   * Summary of manifest loading.
+   */
+  private final SummaryInfo summaryInfo = new SummaryInfo();
+
+  /**
+   * Should manifests be pruned of IOStatistics?
+   */
+  private boolean pruneManifests;
+
+  /**
+   * List of loaded manifests.
+   */
+  private final List<TaskManifest> manifests = new ArrayList<>();
+
+  public LoadManifestsStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true);
+  }
+
+  /**
+   * Load the manifests.
+   * @param prune should manifests be pruned of IOStatistics?
+   * @return the summary and a list of manifests.
+   * @throws IOException IO failure.
+   */
+  @Override
+  protected LoadManifestsStage.Result executeStage(
+      final Boolean prune) throws IOException {
+
+    final Path manifestDir = getTaskManifestDir();
+    LOG.info("{}: Executing Manifest Job Commit with manifests in {}",
+        getName(),
+        manifestDir);
+    pruneManifests = prune;
+    // build a list of all task manifests successfully committed
+    //
+    msync(manifestDir);
+    final RemoteIterator<FileStatus> manifestFiles = listManifests();
+
+    final List<TaskManifest> manifestList = loadAllManifests(manifestFiles);
+    LOG.info("{}: Summary of {} manifests loaded in {}: {}",
+        getName(),
+        manifestList.size(),
+        manifestDir,
+        summaryInfo);
+
+    // collect any stats
+    maybeAddIOStatistics(getIOStatistics(), manifestFiles);
+    return new LoadManifestsStage.Result(summaryInfo, manifestList);
+  }
+
+  /**
+   * Load all the manifests.
+   * @param manifestFiles list of manifest files.
+   * @return the loaded manifests.
+   * @throws IOException IO Failure.
+   */
+  private List<TaskManifest> loadAllManifests(
+      final RemoteIterator<FileStatus> manifestFiles) throws IOException {
+
+    trackDurationOfInvocation(getIOStatistics(), OP_LOAD_ALL_MANIFESTS, () ->
+        TaskPool.foreach(manifestFiles)
+            .executeWith(getIOProcessors())
+            .stopOnFailure()
+            .run(this::processOneManifest));
+    return manifests;
+  }
+
+  /**
+   * Method invoked to process one manifest.
+   * @param status file to process.
+   * @throws IOException failure to load/parse
+   */
+  private void processOneManifest(FileStatus status)
+      throws IOException {
+    updateAuditContext(OP_LOAD_ALL_MANIFESTS);
+
+    TaskManifest m = fetchTaskManifest(status);
+    progress();
+
+    // update the manifest list in a synchronized block.
+
+    synchronized (manifests) {
+      manifests.add(m);
+      // and the summary info in the same block, to
+      // eliminate the need to acquire a second lock.
+      summaryInfo.add(m);
+    }
+    if (pruneManifests) {
+      m.setIOStatistics(null);
+      m.getExtraData().clear();
+    }
+  }
+
+  /**
+   * Precommit preparation of a single manifest file.
+   * To reduce the memory foot print, the IOStatistics and
+   * extra data of each manifest is cleared.
+   * @param status status of file.
+   * @return number of files.
+   * @throws IOException IO Failure.
+   */
+  private TaskManifest fetchTaskManifest(FileStatus status)
+      throws IOException {
+    if (status.getLen() == 0 || !status.isFile()) {
+      throw new PathIOException(status.getPath().toString(),
+          "Not a valid manifest file; file status = " + status);
+    }
+    // load the manifest, which includes validation.
+    final TaskManifest manifest = loadManifest(status);
+    final String id = manifest.getTaskAttemptID();
+    final int filecount = manifest.getFilesToCommit().size();
+    final long size = manifest.getTotalFileSize();
+    LOG.info("{}: Task Attempt {} file {}: File count: {}; data size={}",
+        getName(), id, status.getPath(), filecount, size);
+    // record file size for tracking of memory consumption.
+    getIOStatistics().addMeanStatisticSample(COMMITTER_TASK_MANIFEST_FILE_SIZE,
+        status.getLen());
+    return manifest;
+  }
+
+  /**
+   * Result of the stage.
+   */
+  public static final class Result {
+    private final SummaryInfo summary;
+
+    private final List<TaskManifest> manifests;
+
+    public Result(SummaryInfo summary,
+        List<TaskManifest> manifests) {
+      this.summary = summary;
+      this.manifests = manifests;
+    }
+
+    public SummaryInfo getSummary() {
+      return summary;
+    }
+
+    public List<TaskManifest> getManifests() {
+      return manifests;
+    }
+
+  }
+
+  /**
+   * Summary information.
+   */
+  public static final class SummaryInfo implements IOStatisticsSource {
+
+    /**
+     * Aggregate IOStatistics.
+     */
+    private IOStatisticsSnapshot iostatistics = snapshotIOStatistics();
+
+    /**
+     * How many manifests were loaded.
+     */
+    private long manifestCount;
+
+    /**
+     * Total number of files to rename.
+     */
+    private long fileCount;
+
+    /**
+     * Total number of directories which may need
+     * to be created.
+     * As there is no dedup, this is likely to be
+     * a (major) overestimate.
+     */
+    private long directoryCount;
+
+    /**
+     * Total amount of data to be committed.
+     */
+    private long totalFileSize;
+
+    /**
+     * Get the IOStatistics.
+     * @return aggregate IOStatistics
+     */
+    @Override
+    public IOStatisticsSnapshot getIOStatistics() {
+      return iostatistics;
+    }
+
+    public long getFileCount() {
+      return fileCount;
+    }
+
+    public long getDirectoryCount() {
+      return directoryCount;
+    }
+
+    public long getTotalFileSize() {
+      return totalFileSize;
+    }
+
+    public long getManifestCount() {
+      return manifestCount;
+    }
+
+    /**
+     * Add all statistics.
+     * @param manifest manifest to add.
+     */
+    public void add(TaskManifest manifest) {
+      manifestCount++;
+      iostatistics.aggregate(manifest.getIOStatistics());
+      fileCount += manifest.getFilesToCommit().size();
+      directoryCount += manifest.getDestDirectories().size();
+      totalFileSize += manifest.getTotalFileSize();
+    }
+
+    /**
+     * To String includes all summary info except statistics.
+     * @return string value
+     */
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "SummaryInfo{");
+      sb.append("manifestCount=").append(manifestCount);
+      sb.append(", fileCount=").append(fileCount);
+      sb.append(", directoryCount=").append(directoryCount);
+      sb.append(", totalFileSize=").append(
+          byteCountToDisplaySize(totalFileSize));
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java
new file mode 100644
index 0000000..e8124c1
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java
@@ -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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_RENAME_FILES;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome;
+import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat;
+
+/**
+ * This stage renames all the files.
+ * Input: the manifests and the set of directories created, as returned by
+ * {@link CreateOutputDirectoriesStage}.
+ * If the job is configured to delete target files, if the parent dir
+ * had to be created, the delete() call can be skipped.
+ * It returns a manifest success data file summarizing the
+ * output, but does not add iostatistics to it.
+ */
+public class RenameFilesStage extends
+    AbstractJobOrTaskStage<
+        Pair<List<TaskManifest>, Set<Path>>,
+        ManifestSuccessData> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      RenameFilesStage.class);
+
+  /**
+   * List of all files committed.
+   */
+  private final List<FileEntry> filesCommitted = new ArrayList<>();
+
+  /**
+   * Total file size.
+   */
+  private long totalFileSize = 0;
+
+  private Set<Path> createdDirectories;
+
+  public RenameFilesStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_RENAME_FILES, true);
+  }
+
+  /**
+   * Get the list of files committed.
+   * Access is not synchronized.
+   * @return direct access to the list of files.
+   */
+  public synchronized  List<FileEntry> getFilesCommitted() {
+    return filesCommitted;
+  }
+
+  /**
+   * Get the total file size of the committed task.
+   * @return a number greater than or equal to zero.
+   */
+  public synchronized long getTotalFileSize() {
+    return totalFileSize;
+  }
+
+  /**
+   * Rename files in job commit.
+   * @param taskManifests a list of task manifests containing files.
+   * @return the job report.
+   * @throws IOException failure
+   */
+  @Override
+  protected ManifestSuccessData executeStage(
+      Pair<List<TaskManifest>, Set<Path>> args)
+      throws IOException {
+
+    final List<TaskManifest> taskManifests = args.getLeft();
+    createdDirectories = args.getRight();
+
+    final ManifestSuccessData success = createManifestOutcome(getStageConfig(),
+        OP_STAGE_JOB_COMMIT);
+    final int manifestCount = taskManifests.size();
+
+    LOG.info("{}: Executing Manifest Job Commit with {} manifests in {}",
+        getName(), manifestCount, getTaskManifestDir());
+
+    // first step is to aggregate the output of all manifests into a single
+    // list of files to commit.
+    // Which Guava can do in a zero-copy concatenated iterator
+
+    final Iterable<FileEntry> filesToCommit = concat(taskManifests.stream()
+        .map(TaskManifest::getFilesToCommit)
+        .collect(Collectors.toList()));
+
+    TaskPool.foreach(filesToCommit)
+        .executeWith(getIOProcessors())
+        .stopOnFailure()
+        .run(this::commitOneFile);
+
+    // synchronized block to keep spotbugs happy.
+    List<FileEntry> committed = getFilesCommitted();
+    LOG.info("{}: Files committed: {}. Total size {}",
+        getName(), committed.size(), getTotalFileSize());
+
+    // Add a subset of the destination files to the success file;
+    // enough for simple testing
+    success.setFilenamePaths(
+        committed
+            .subList(0, Math.min(committed.size(), SUCCESS_MARKER_FILE_LIMIT))
+            .stream().map(FileEntry::getDestPath)
+            .collect(Collectors.toList()));
+
+    success.setSuccess(true);
+
+    return success;
+  }
+
+  /**
+   * Commit one file by rename, then, if that doesn't fail,
+   * add to the files committed list.
+   * @param entry entry to commit.
+   * @throws IOException faiure.
+   */
+  private void commitOneFile(FileEntry entry) throws IOException {
+    updateAuditContext(OP_STAGE_JOB_RENAME_FILES);
+
+    // report progress back
+    progress();
+
+    // if the dest dir is to be deleted,
+    // look to see if the parent dir was created.
+    // if it was. we know that the file doesn't exist.
+    final boolean deleteDest = getStageConfig().getDeleteTargetPaths()
+        && !createdDirectories.contains(entry.getDestPath().getParent());
+    // do the rename
+    commitFile(entry, deleteDest);
+
+    // update the list and IOStats
+    synchronized (this) {
+      filesCommitted.add(entry);
+      totalFileSize += entry.getSize();
+    }
+
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java
new file mode 100644
index 0000000..eb9c82f
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java
@@ -0,0 +1,79 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_SAVE_SUCCESS;
+
+/**
+ * Save the _SUCCESS file to the destination directory
+ * via a temp file in the job attempt dir.
+ * Returns the path of the file
+ */
+public class SaveSuccessFileStage extends
+    AbstractJobOrTaskStage<ManifestSuccessData, Path> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      SaveSuccessFileStage.class);
+
+  public SaveSuccessFileStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_SAVE_SUCCESS, false);
+  }
+
+  /**
+   * Stage name is always job commit.
+   * @param arguments args to the invocation.
+   * @return stage name
+   */
+  @Override
+  protected String getStageName(ManifestSuccessData arguments) {
+    // set it to the job commit stage, always.
+    return OP_STAGE_JOB_COMMIT;
+  }
+
+  /**
+   * Execute.
+   * @param successData success data to save
+   * @return path saved to.
+   * @throws IOException failure
+   */
+  @Override
+  protected Path executeStage(final ManifestSuccessData successData)
+      throws IOException {
+    // Save the marker
+    Path successFile = getStageConfig().getJobSuccessMarkerPath();
+    Path successTempFile = new Path(getJobAttemptDir(), SUCCESS_MARKER + TMP_SUFFIX);
+    LOG.debug("{}: Saving _SUCCESS file to {} via {}", successFile,
+        getName(),
+        successTempFile);
+    save(successData, successTempFile, successFile);
+    return successFile;
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java
new file mode 100644
index 0000000..fdaf018
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java
@@ -0,0 +1,81 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SAVE_MANIFEST;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestTempPathForTaskAttempt;
+
+/**
+ * Save a task manifest to the job attempt dir, using the task
+ * ID for the name of the final file.
+ * For atomic writes, the manifest is saved
+ * by writing to a temp file and then renaming it.
+ * Uses both the task ID and task attempt ID to determine the temp filename;
+ * Before the rename of (temp, final-path), any file at the final path
+ * is deleted.
+ * This is so that when this stage is invoked in a task commit, its output
+ * overwrites any of the first commit.
+ * When it succeeds, therefore, unless there is any subsequent commit of
+ * another task, the task manifest at the final path is from this
+ * operation.
+ *
+ * Returns the path where the manifest was saved.
+ */
+public class SaveTaskManifestStage extends
+    AbstractJobOrTaskStage<TaskManifest, Path> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      SaveTaskManifestStage.class);
+
+  public SaveTaskManifestStage(final StageConfig stageConfig) {
+    super(true, stageConfig, OP_STAGE_TASK_SAVE_MANIFEST, false);
+  }
+
+  /**
+   * Save the manifest to a temp file and rename to the final
+   * manifest destination.
+   * @param manifest manifest
+   * @return the path to the final entry
+   * @throws IOException IO failure.
+   */
+  @Override
+  protected Path executeStage(final TaskManifest manifest)
+      throws IOException {
+
+    final Path manifestDir = getTaskManifestDir();
+    // final manifest file is by task ID
+    Path manifestFile = manifestPathForTask(manifestDir,
+        getRequiredTaskId());
+    Path manifestTempFile = manifestTempPathForTaskAttempt(manifestDir,
+        getRequiredTaskAttemptId());
+    LOG.info("{}: Saving manifest file to {}", getName(), manifestFile);
+    save(manifest, manifestTempFile, manifestFile);
+    return manifestFile;
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java
new file mode 100644
index 0000000..9b87325
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java
@@ -0,0 +1,63 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_SETUP;
+
+/**
+ * Stage to set up a job by creating the job attempt directory.
+ * The job attempt directory must not exist before the call.
+ */
+public class SetupJobStage extends
+    AbstractJobOrTaskStage<Boolean, Path> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      SetupJobStage.class);
+
+  public SetupJobStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_SETUP, false);
+  }
+
+  /**
+   * Execute the job setup stage.
+   * @param deleteMarker: should any success marker be deleted.
+   * @return the job attempted directory.
+   * @throws IOException failure.
+   */
+  @Override
+  protected Path executeStage(final Boolean deleteMarker) throws IOException {
+    final Path path = getJobAttemptDir();
+    LOG.info("{}: Creating Job Attempt directory {}", getName(), path);
+    createNewDirectory("Job setup", path);
+    createNewDirectory("Creating task manifest dir", getTaskManifestDir());
+    // delete any success marker if so instructed.
+    if (deleteMarker) {
+      delete(getStageConfig().getJobSuccessMarkerPath(), false);
+    }
+    return path;
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java
new file mode 100644
index 0000000..e909ee1
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SETUP;
+
+/**
+ * Stage to set up task.
+ * This creates the task attempt directory, after verifying
+ * that the job attempt dir exists (i.e. this is invoked
+ * after the job is started and before any cleanup.
+ * Argument passed in is task name:only for logging.
+ */
+public class SetupTaskStage extends
+    AbstractJobOrTaskStage<String, Path> {
+
+  public SetupTaskStage(final StageConfig stageConfig) {
+    super(true, stageConfig, OP_STAGE_TASK_SETUP, false);
+  }
+
+  /**
+   * Set up a task.
+   * @param name task name (for logging)
+   * @return task attempt directory
+   * @throws IOException IO failure.
+   */
+  @Override
+  protected Path executeStage(final String name) throws IOException {
+    return createNewDirectory("Task setup " + name,
+        requireNonNull(getTaskAttemptDir(), "No task attempt directory"));
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java
new file mode 100644
index 0000000..d5bbba5
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java
@@ -0,0 +1,556 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations;
+import org.apache.hadoop.util.JsonSerialization;
+import org.apache.hadoop.util.Preconditions;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER;
+
+/**
+ * Stage Config.
+ * Everything to configure a stage which is common to all.
+ *
+ * It's isolated from the details of MR datatypes (taskID, taskattempt etc);
+ * at this point it expects parsed values.
+ *
+ * It uses the builder API, but once {@link #build()} is called it goes
+ * read only. This is to ensure that changes cannot
+ * take place when shared across stages.
+ */
+public class StageConfig {
+
+  /**
+   * A flag which freezes the config for
+   * further updates.
+   */
+  private boolean frozen;
+
+  /**
+   * IOStatistics to update.
+   */
+  private IOStatisticsStore iostatistics;
+
+  /**
+   * Job ID; constant over multiple attempts.
+   */
+  private String jobId;
+
+  /**
+   * Where did the job Unique ID come from?
+   */
+  private String jobIdSource = "";
+
+  /**
+   * Number of the job attempt; starts at zero.
+   */
+  private int jobAttemptNumber;
+
+  /**
+   * ID of the task.
+   */
+  private String taskId;
+
+  /**
+   * ID of this specific attempt at a task.
+   */
+  private String taskAttemptId;
+
+  /**
+   * Destination of job.
+   */
+  private Path destinationDir;
+
+  /**
+   * Job attempt dir.
+   */
+  private Path jobAttemptDir;
+
+  /**
+   * temp directory under job dest dir.
+   */
+  private Path outputTempSubDir;
+
+  /**
+   * Task attempt dir.
+   */
+  private Path taskAttemptDir;
+
+  /**
+   * directory where task manifests must go.
+   */
+  private Path taskManifestDir;
+
+  /**
+   * Subdir under the job attempt dir where task
+   * attempts will have subdirectories.
+   */
+  private Path jobAttemptTaskSubDir;
+
+  /**
+   * Callbacks to update store.
+   * This is not made visible to the stages; they must
+   * go through the superclass which
+   * adds statistics and logging.
+   */
+  private ManifestStoreOperations operations;
+
+  /**
+   * Submitter for doing IO against the store other than
+   * manifest processing.
+   */
+  private TaskPool.Submitter ioProcessors;
+
+  /**
+   * Optional progress callback.
+   */
+  private Progressable progressable;
+
+  /**
+   * Callback when a stage is entered.
+   */
+  private StageEventCallbacks enterStageEventHandler;
+
+  /**
+   * Thread local serializer; created on demand
+   * and shareable across a sequence of stages.
+   */
+  private final ThreadLocal<JsonSerialization<TaskManifest>> threadLocalSerializer =
+      ThreadLocal.withInitial(TaskManifest::serializer);
+
+  /**
+   * Delete target paths on commit? Stricter, but
+   * higher IO cost.
+   */
+  private boolean deleteTargetPaths;
+
+  /**
+   * Name for logging.
+   */
+  private String name = "";
+
+  public StageConfig() {
+  }
+
+  /**
+   * Verify that the config is not yet frozen.
+   */
+  private void checkOpen() {
+    Preconditions.checkState(!frozen,
+        "StageConfig is now read-only");
+  }
+
+  /**
+   * The build command makes the config immutable.
+   * Idempotent.
+   * @return the now-frozen config
+   */
+  public StageConfig build() {
+    frozen = true;
+    return this;
+  }
+
+  /**
+   * Set job destination dir.
+   * @param dir new dir
+   * @return this
+   */
+  public StageConfig withDestinationDir(final Path dir) {
+    destinationDir = dir;
+    return this;
+  }
+
+  /**
+   * Set IOStatistics store.
+   * @param store new store
+   * @return this
+   */
+  public StageConfig withIOStatistics(final IOStatisticsStore store) {
+    checkOpen();
+    iostatistics = store;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withIOProcessors(final TaskPool.Submitter value) {
+    checkOpen();
+    ioProcessors = value;
+    return this;
+  }
+
+  /**
+   * Set Job attempt directory.
+   * @param dir new dir
+   * @return this
+   */
+  public StageConfig withJobAttemptDir(final Path dir) {
+    checkOpen();
+    jobAttemptDir = dir;
+    return this;
+  }
+
+  /**
+   * Directory to put task manifests into.
+   * @return a path under the job attempt dir.
+   */
+  public Path getTaskManifestDir() {
+    return taskManifestDir;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public StageConfig withTaskManifestDir(Path value) {
+    checkOpen();
+    taskManifestDir = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public StageConfig withJobAttemptTaskSubDir(Path value) {
+    jobAttemptTaskSubDir = value;
+    return this;
+  }
+
+  /**
+   * Get the path to the subdirectory under $jobID where task
+   * attempts are. List this dir to find all task attempt dirs.
+   * @return a path under the job attempt dir.
+   */
+  public Path getJobAttemptTaskSubDir() {
+    return jobAttemptTaskSubDir;
+  }
+
+  /**
+   * Set the job directories from the attempt directories
+   * information. Does not set task attempt fields.
+   * @param dirs source of directories.
+   * @return this
+   */
+  public StageConfig withJobDirectories(
+      final ManifestCommitterSupport.AttemptDirectories dirs) {
+
+    checkOpen();
+    withJobAttemptDir(dirs.getJobAttemptDir())
+        .withJobAttemptTaskSubDir(dirs.getJobAttemptTaskSubDir())
+        .withDestinationDir(dirs.getOutputPath())
+        .withOutputTempSubDir(dirs.getOutputTempSubDir())
+        .withTaskManifestDir(dirs.getTaskManifestDir());
+
+    return this;
+  }
+
+  /**
+   * Set job ID with no attempt included.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withJobId(final String value) {
+    checkOpen();
+    jobId = value;
+    return this;
+  }
+
+  public Path getOutputTempSubDir() {
+    return outputTempSubDir;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withOutputTempSubDir(final Path value) {
+    checkOpen();
+    outputTempSubDir = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withOperations(final ManifestStoreOperations value) {
+    checkOpen();
+    operations = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withTaskAttemptId(final String value) {
+    checkOpen();
+    taskAttemptId = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withTaskId(final String value) {
+    checkOpen();
+    taskId = value;
+    return this;
+  }
+
+  /**
+   * Set handler for stage entry events..
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withStageEventCallbacks(StageEventCallbacks value) {
+    checkOpen();
+    enterStageEventHandler = value;
+    return this;
+  }
+
+  /**
+   * Optional progress callback.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withProgressable(final Progressable value) {
+    checkOpen();
+    progressable = value;
+    return this;
+  }
+
+  /**
+   * Set the Task attempt directory.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withTaskAttemptDir(final Path value) {
+    checkOpen();
+    taskAttemptDir = value;
+    return this;
+  }
+
+  /**
+   * Set the job attempt number.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withJobAttemptNumber(final int value) {
+    checkOpen();
+    jobAttemptNumber = value;
+    return this;
+  }
+
+  /**
+   * Set the Job ID source.
+   * @param value new value
+   * @return this
+   */
+  public StageConfig withJobIdSource(final String value) {
+    checkOpen();
+    jobIdSource = value;
+    return this;
+  }
+
+  /**
+   * Set name of task/job.
+   * @param value new value
+   * @return the builder
+   */
+  public StageConfig withName(String value) {
+    name = value;
+    return this;
+  }
+
+  /**
+   * Get name of task/job.
+   * @return name for logging.
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Handler for stage entry events.
+   * @return the handler.
+   */
+  public StageEventCallbacks getEnterStageEventHandler() {
+    return enterStageEventHandler;
+  }
+
+  /**
+   * IOStatistics to update.
+   */
+  public IOStatisticsStore getIOStatistics() {
+    return iostatistics;
+  }
+
+  /**
+   * Job ID.
+   */
+  public String getJobId() {
+    return jobId;
+  }
+
+  /**
+   * ID of the task.
+   */
+  public String getTaskId() {
+    return taskId;
+  }
+
+  /**
+   * ID of this specific attempt at a task.
+   */
+  public String getTaskAttemptId() {
+    return taskAttemptId;
+  }
+
+  /**
+   * Job attempt dir.
+   */
+  public Path getJobAttemptDir() {
+    return jobAttemptDir;
+  }
+
+  /**
+   * Destination of job.
+   */
+  public Path getDestinationDir() {
+    return destinationDir;
+  }
+
+  /**
+   * Get the location of the success marker.
+   * @return a path under the destination directory.
+   */
+  public Path getJobSuccessMarkerPath() {
+    return new Path(destinationDir, SUCCESS_MARKER);
+  }
+
+  /**
+   * Callbacks to update store.
+   * This is not made visible to the stages; they must
+   * go through the wrapper classes in this class, which
+   * add statistics and logging.
+   */
+  public ManifestStoreOperations getOperations() {
+    return operations;
+  }
+
+  /**
+   * Submitter for doing IO against the store other than
+   * manifest processing.
+   */
+  public TaskPool.Submitter getIoProcessors() {
+    return ioProcessors;
+  }
+
+  /**
+   * Get optional progress callback.
+   * @return callback or null
+   */
+  public Progressable getProgressable() {
+    return progressable;
+  }
+
+  /**
+   * Task attempt directory.
+   * @return the task attempt dir.
+   */
+  public Path getTaskAttemptDir() {
+    return taskAttemptDir;
+  }
+
+  /**
+   * Get the job attempt number.
+   * @return the value
+   */
+  public int getJobAttemptNumber() {
+    return jobAttemptNumber;
+  }
+
+  public String getJobIdSource() {
+    return jobIdSource;
+  }
+
+  /**
+   * Get a thread local task manifest serializer.
+   * @return a serializer.
+   */
+  public JsonSerialization<TaskManifest> currentManifestSerializer() {
+    return threadLocalSerializer.get();
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public StageConfig withDeleteTargetPaths(boolean value) {
+    checkOpen();
+    deleteTargetPaths = value;
+    return this;
+  }
+
+  public boolean getDeleteTargetPaths() {
+    return deleteTargetPaths;
+  }
+
+  /**
+   * Enter the stage; calls back to
+   * {@link #enterStageEventHandler} if non-null.
+   * @param stage stage entered
+   */
+  public void enterStage(String stage) {
+    if (enterStageEventHandler != null) {
+      enterStageEventHandler.enterStage(stage);
+    }
+  }
+
+  /**
+   * Exit the stage; calls back to
+   * {@link #enterStageEventHandler} if non-null.
+   * @param stage stage entered
+   */
+  public void exitStage(String stage) {
+    if (enterStageEventHandler != null) {
+      enterStageEventHandler.exitStage(stage);
+    }
+  }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java
new file mode 100644
index 0000000..e23f720
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+/**
+ * Callbacks on stage entry/exit.
+ */
+public interface StageEventCallbacks {
+
+  /**
+   * Callback on stage entry.
+   * @param stage new stage
+   */
+  void enterStage(String stage);
+
+  /**
+   * Callback on stage exit.
+   * @param stage stage exited.
+   */
+  void exitStage(String stage);
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java
new file mode 100644
index 0000000..4765bcc
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java
@@ -0,0 +1,214 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.LongSummaryStatistics;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.util.DurationInfo;
+
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_DIRECTORY_COUNT_MEAN;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_DIRECTORY_DEPTH_MEAN;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_FILE_COUNT_MEAN;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_FILE_SIZE_MEAN;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SCAN_DIRECTORY;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createTaskManifest;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics;
+
+/**
+ * Stage to scan a directory tree and build a task manifest.
+ * This is executed by the task committer.
+ */
+public final class TaskAttemptScanDirectoryStage
+    extends AbstractJobOrTaskStage<Void, TaskManifest> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TaskAttemptScanDirectoryStage.class);
+
+  public TaskAttemptScanDirectoryStage(
+      final StageConfig stageConfig) {
+    super(true, stageConfig, OP_STAGE_TASK_SCAN_DIRECTORY, false);
+  }
+
+  /**
+   * Build the Manifest.
+   * @return the manifest
+   * @throws IOException failure.
+   */
+  @Override
+  protected TaskManifest executeStage(final Void arguments)
+      throws IOException {
+
+    final Path taskAttemptDir = getRequiredTaskAttemptDir();
+    final TaskManifest manifest = createTaskManifest(getStageConfig());
+
+    LOG.info("{}: scanning directory {}",
+        getName(), taskAttemptDir);
+
+    final int depth = scanDirectoryTree(manifest,
+        taskAttemptDir,
+        getDestinationDir(),
+        0, true);
+    List<FileEntry> filesToCommit = manifest.getFilesToCommit();
+    LongSummaryStatistics fileSummary = filesToCommit.stream()
+        .mapToLong(FileEntry::getSize)
+        .summaryStatistics();
+    long fileDataSize = fileSummary.getSum();
+    long fileCount = fileSummary.getCount();
+    int dirCount = manifest.getDestDirectories().size();
+    LOG.info("{}: directory {} contained {} file(s); data size {}",
+        getName(),
+        taskAttemptDir,
+        fileCount,
+        fileDataSize);
+    LOG.info("{}: Directory count = {}; maximum depth {}",
+        getName(),
+        dirCount,
+        depth);
+    // add statistics about the task output which, when aggregated, provides
+    // insight into structure of job, task skew, etc.
+    IOStatisticsStore iostats = getIOStatistics();
+    iostats.addSample(COMMITTER_TASK_DIRECTORY_COUNT_MEAN, dirCount);
+    iostats.addSample(COMMITTER_TASK_DIRECTORY_DEPTH_MEAN, depth);
+    iostats.addSample(COMMITTER_TASK_FILE_COUNT_MEAN, fileCount);
+    iostats.addSample(COMMITTER_TASK_FILE_SIZE_MEAN, fileDataSize);
+
+    return manifest;
+  }
+
+  /**
+   * Recursively scan a directory tree.
+   * The manifest will contain all files to rename
+   * (source and dest) and directories to create.
+   * All files are processed before any of the subdirs are.
+   * This helps in statistics gathering.
+   * There's some optimizations which could be done with async
+   * fetching of the iterators of those subdirs, but as this
+   * is generally off-critical path then that "enhancement"
+   * can be postponed until data suggests this needs improvement.
+   * @param manifest manifest to update
+   * @param srcDir dir to scan
+   * @param destDir destination directory
+   * @param depth depth from the task attempt dir.
+   * @param parentDirExists does the parent dir exist?
+   * @return the maximum depth of child directories
+   * @throws IOException IO failure.
+   */
+  private int scanDirectoryTree(
+      TaskManifest manifest,
+      Path srcDir,
+      Path destDir,
+      int depth,
+      boolean parentDirExists) throws IOException {
+
+    // generate some task progress in case directory scanning is very slow.
+    progress();
+
+    int maxDepth = 0;
+    int files = 0;
+    boolean dirExists = parentDirExists;
+    List<FileStatus> subdirs = new ArrayList<>();
+    try (DurationInfo ignored = new DurationInfo(LOG, false,
+        "Task Attempt %s source dir %s, dest dir %s",
+        getTaskAttemptId(), srcDir, destDir)) {
+
+      // list the directory. This may block until the listing is complete,
+      // or, if the FS does incremental or asynchronous fetching,
+      // then the next()/hasNext() call will block for the results
+      // unless turned off, ABFS does to this async
+      final RemoteIterator<FileStatus> listing = listStatusIterator(srcDir);
+
+      // when the FS (especially ABFS) does an asyn fetch of the listing,
+      // we can probe for the status of the destination dir while that
+      // page is being fetched.
+      // probe for and add the dest dir entry for all but
+      // the base dir
+
+      if (depth > 0) {
+        final EntryStatus status;
+        if (parentDirExists) {
+          final FileStatus destDirStatus = getFileStatusOrNull(destDir);
+          status = EntryStatus.toEntryStatus(destDirStatus);
+          dirExists = destDirStatus != null;
+        } else {
+          // if there is no parent dir, then there is no need to look
+          // for this directory -report it as missing automatically.
+          status = EntryStatus.not_found;
+        }
+        manifest.addDirectory(DirEntry.dirEntry(
+            destDir,
+            status,
+            depth));
+      }
+
+      // process the listing; this is where abfs will block
+      // to wait the result of the list call.
+      while (listing.hasNext()) {
+        final FileStatus st = listing.next();
+        if (st.isFile()) {
+          // this is a file, so add to the list of files to commit.
+          files++;
+          final FileEntry entry = fileEntry(st, destDir);
+          manifest.addFileToCommit(entry);
+          LOG.debug("To rename: {}", entry);
+        } else {
+          if (st.isDirectory()) {
+            // will need to scan this directory too.
+            subdirs.add(st);
+          } else {
+            // some other object. ignoring
+            LOG.info("Ignoring FS object {}", st);
+          }
+        }
+      }
+      // add any statistics provided by the listing.
+      maybeAddIOStatistics(getIOStatistics(), listing);
+    }
+
+    // now scan the subdirectories
+    LOG.debug("{}: Number of subdirectories under {} found: {}; file count {}",
+        getName(), srcDir, subdirs.size(), files);
+
+    for (FileStatus st : subdirs) {
+      Path destSubDir = new Path(destDir, st.getPath().getName());
+      final int d = scanDirectoryTree(manifest,
+          st.getPath(),
+          destSubDir,
+          depth + 1,
+          dirExists);
+      maxDepth = Math.max(maxDepth, d);
+    }
+
+    return 1 + maxDepth;
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java
new file mode 100644
index 0000000..8b72112
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java
@@ -0,0 +1,200 @@
+/*
+ * 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.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport;
+import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException;
+import org.apache.hadoop.util.functional.TaskPool;
+
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_VALIDATE_OUTPUT;
+import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat;
+
+/**
+ * This stage validates all files by scanning the manifests
+ * and verifying every file in every manifest is of the given size.
+ * Returns a list of all files committed.
+ *
+ * Its cost is one getFileStatus() call (parallelized) per file.
+ * Raises a {@link OutputValidationException} on a validation failure.
+ */
+public class ValidateRenamedFilesStage extends
+    AbstractJobOrTaskStage<
+        List<TaskManifest>,
+        List<FileEntry>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ValidateRenamedFilesStage.class);
+
+  /**
+   * Set this to halt all workers.
+   */
+  private final AtomicBoolean halt = new AtomicBoolean();
+
+  /**
+   * List of all files committed.
+   */
+  private List<FileEntry> filesCommitted = new ArrayList<>();
+
+  public ValidateRenamedFilesStage(final StageConfig stageConfig) {
+    super(false, stageConfig, OP_STAGE_JOB_VALIDATE_OUTPUT, true);
+  }
+
+  /**
+   * Get the list of files committed.
+   * @return a possibly empty list.
+   */
+  private synchronized List<FileEntry> getFilesCommitted() {
+    return filesCommitted;
+  }
+
+  /**
+   * Add a file entry to the list of committed files.
+   * @param entry entry
+   */
+  private synchronized void addFileCommitted(FileEntry entry) {
+    filesCommitted.add(entry);
+  }
+
+  /**
+   * Validate the task manifests.
+   * This is done by listing all the directories
+   * and verifying that every file in the source list
+   * has a file in the destination of the same size.
+   * If two tasks have both written the same file or
+   * a source file was changed after the task was committed,
+   * then a mistmatch will be detected -provided the file
+   * length is now different.
+   * @param taskManifests list of manifests.
+   * @return list of files committed.
+   */
+  @Override
+  protected List<FileEntry> executeStage(
+      final List<TaskManifest> taskManifests)
+      throws IOException {
+
+    // set the list of files to be as big as the number of tasks.
+    // synchronized to stop complaints.
+    synchronized (this) {
+      filesCommitted = new ArrayList<>(taskManifests.size());
+    }
+
+    // validate all the files.
+
+    final Iterable<FileEntry> filesToCommit = concat(taskManifests.stream()
+        .map(TaskManifest::getFilesToCommit)
+        .collect(Collectors.toList()));
+
+    TaskPool.foreach(filesToCommit)
+        .executeWith(getIOProcessors())
+        .stopOnFailure()
+        .run(this::validateOneFile);
+
+    return getFilesCommitted();
+  }
+
+  /**
+   * Validate a file.
+   * @param entry entry to probe for
+   * @throws IOException IO problem.
+   * @throws OutputValidationException if the entry is not valid
+   */
+  private void validateOneFile(FileEntry entry) throws IOException {
+    updateAuditContext(OP_STAGE_JOB_VALIDATE_OUTPUT);
+
+    if (halt.get()) {
+      // told to stop
+      return;
+    }
+    // report progress back
+    progress();
+    // look validate the file.
+    // raising an FNFE if the file isn't there.
+    FileStatus destStatus;
+    final Path sourcePath = entry.getSourcePath();
+    Path destPath = entry.getDestPath();
+    try {
+      destStatus = getFileStatus(destPath);
+
+      // it must be a file
+      if (!destStatus.isFile()) {
+        throw new OutputValidationException(destPath,
+            "Expected a file renamed from " + sourcePath
+                + "; found " + destStatus);
+      }
+      final long sourceSize = entry.getSize();
+      final long destSize = destStatus.getLen();
+
+      // etags, if the source had one.
+      final String sourceEtag = entry.getEtag();
+      if (isNotBlank(sourceEtag)) {
+        final String destEtag = ManifestCommitterSupport.getEtag(destStatus);
+        if (!sourceEtag.equals(destEtag)) {
+          LOG.warn("Etag of dest file {}: {} does not match that of manifest entry {}",
+              destPath, destStatus, entry);
+          throw new OutputValidationException(destPath,
+              String.format("Expected the file"
+                      + " renamed from %s"
+                      + " with etag %s and length %s"
+                      + " but found a file with etag %s and length %d",
+                  sourcePath,
+                  sourceEtag,
+                  sourceSize,
+                  destEtag,
+                  destSize));
+
+        }
+      }
+      // check the expected length after any etag validation
+      if (destSize != sourceSize) {
+        LOG.warn("Length of dest file {}: {} does not match that of manifest entry {}",
+            destPath, destStatus, entry);
+        throw new OutputValidationException(destPath,
+            String.format("Expected the file"
+                    + " renamed from %s"
+                    + " with length %d"
+                    + " but found a file of length %d",
+                sourcePath,
+                sourceSize,
+                destSize));
+      }
+
+    } catch (FileNotFoundException e) {
+      // file didn't exist
+      throw new OutputValidationException(destPath,
+          "Expected a file, but it was not found", e);
+    }
+    addFileCommitted(entry);
+  }
+
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java
new file mode 100644
index 0000000..8b86046
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/**
+ * Intermediate manifest committer.
+ *
+ * The individual and aggregate stages of the task and job commit process.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 844c91c..d315a00 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -2231,4 +2231,23 @@
   </description>
 </property>
 
+<!-- not yet enabled by default.
+
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.abfs</name>
+  <value>org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory</value>
+  <description>
+    The default committer factory for ABFS is for the manifest committer with
+    abfs-specific tuning.
+  </description>
+</property>
+
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.gs</name>
+  <value>org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory</value>
+  <description>
+    The default committer factory for google cloud storage is for the manifest committer.
+  </description>
+</property>
+-->
 </configuration>
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md
new file mode 100644
index 0000000..b446be2
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md
@@ -0,0 +1,605 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+
+# The Manifest Committer for Azure and Google Cloud Storage
+
+This document how to use the _Manifest Committer_.
+
+The _Manifest_ committer is a committer for work which provides
+performance on ABFS for "real world" queries,
+and performance and correctness on GCS.
+It also works with other filesystems, including HDFS.
+However, the design is optimized for object stores where
+listing operatons are slow and expensive.
+
+The architecture and implementation of the committer is covered in
+[Manifest Committer Architecture](manifest_committer_architecture.html).
+
+
+The protocol and its correctness are covered in
+[Manifest Committer Protocol](manifest_committer_protocol.html).
+
+It was added in March 2022, and should be considered unstable
+in early releases.
+
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+## Problem:
+
+The only committer of work from Spark to Azure ADLS Gen 2 "abfs://" storage
+which is safe to use is the "v1 file committer".
+
+This is "correct" in that if a task attempt fails, its output is guaranteed not
+to be included in the final out. The "v2" commit algorithm cannot meet that
+guarantee, which is why it is no longer the default.
+
+But: it is slow, especially on jobs where deep directory trees of output are used.
+Why is it slow? It's hard to point at a particular cause, primarily because of
+the lack of any instrumentation in the `FileOutputCommitter`.
+Stack traces of running jobs generally show `rename()`, though list operations
+do surface too.
+
+On Google GCS, neither the v1 nor v2 algorithm are _safe_ because the google
+filesystem doesn't have the atomic directory rename which the v1 algorithm
+requires.
+
+A further issue is that both Azure and GCS storage may encounter scale issues
+with deleting directories with many descendants.
+This can trigger timeouts because the FileOutputCommitter assumes that
+cleaning up after the job is a fast call to `delete("_temporary", true)`.
+
+## Solution.
+
+The _Intermediate Manifest_ committer is a new committer for
+work which should deliver performance on ABFS
+for "real world" queries, and performance and correctness on GCS.
+
+This committer uses the extension point which came in for the S3A committers.
+Users can declare a new committer factory for abfs:// and gcs:// URLs.
+A suitably configured spark deployment will pick up the new committer.
+
+Directory performance issues in job cleanup can be addressed by two options
+1. The committer will parallelize deletion of task attempt directories before
+   deleting the `_temporary` directory.
+1. Cleanup can be disabled. .
+
+The committer can be used with any filesystem client which has a "real" file rename()
+operation.
+It has been optimised for remote object stores where listing and file probes
+are expensive -the design is less likely to offer such signifcant speedup
+on HDFS -though the parallel renaming operations will speed up jobs
+there compared to the classic v1 algorithm.
+
+# <a name="how"></a> How it works
+
+The full details are covered in [Manifest Committer Architecture](manifest_committer_architecture.html).
+
+# <a name="use"></a> Using the committer
+
+The hooks put in to support the S3A committers were designed to allow every
+filesystem schema to provide their own committer.
+See [Switching To an S3A Committer](../../hadoop-aws/tools/hadoop-aws/committers.html#Switching_to_an_S3A_Committer)
+
+A factory for the abfs schema would be defined in
+`mapreduce.outputcommitter.factory.scheme.abfs` ; and a similar one for `gcs`.
+
+Some matching spark configuration changes, especially for parquet binding, will be required.
+These can be done in `core-site.xml`, if it is not defined in the `mapred-default.xml` JAR.
+
+
+```xml
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.abfs</name>
+  <value>org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory</value>
+</property>
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.gs</name>
+  <value>org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory</value>
+</property>
+```
+
+## Binding to the manifest committer in Spark.
+
+In Apache Spark, the configuration can be done either with command line options (after the '--conf') or by using the `spark-defaults.conf` file. The following is an example of using `spark-defaults.conf` also including the configuration for Parquet with a subclass of the parquet
+committer which uses the factory mechansim internally.
+
+```
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.abfs org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.gs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory
+spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
+spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
+```
+
+
+### <a name="committerinfo"></a> Using the Cloudstore `committerinfo` command to probe committer bindings.
+
+The hadoop committer settings can be validated in a recent build of [cloudstore](https://github.com/steveloughran/cloudstore)
+and its `committerinfo` command.
+This command instantiates a committer for that path through the same factory mechanism as MR and spark jobs use,
+then prints its `toString` value.
+
+```
+hadoop jar cloudstore-1.0.jar committerinfo abfs://testing@ukwest.dfs.core.windows.net/
+
+2021-09-16 19:42:59,731 [main] INFO  commands.CommitterInfo (StoreDurationInfo.java:<init>(53)) - Starting: Create committer
+Committer factory for path abfs://testing@ukwest.dfs.core.windows.net/ is
+ org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory@3315d2d7
+  (classname org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory)
+2021-09-16 19:43:00,897 [main] INFO  manifest.ManifestCommitter (ManifestCommitter.java:<init>(144)) - Created ManifestCommitter with
+   JobID job__0000, Task Attempt attempt__0000_r_000000_1 and destination abfs://testing@ukwest.dfs.core.windows.net/
+Created committer of class org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitter:
+ ManifestCommitter{ManifestCommitterConfig{destinationDir=abfs://testing@ukwest.dfs.core.windows.net/,
+   role='task committer',
+   taskAttemptDir=abfs://testing@ukwest.dfs.core.windows.net/_temporary/manifest_job__0000/0/_temporary/attempt__0000_r_000000_1,
+   createJobMarker=true,
+   jobUniqueId='job__0000',
+   jobUniqueIdSource='JobID',
+   jobAttemptNumber=0,
+   jobAttemptId='job__0000_0',
+   taskId='task__0000_r_000000',
+   taskAttemptId='attempt__0000_r_000000_1'},
+   iostatistics=counters=();
+
+gauges=();
+
+minimums=();
+
+maximums=();
+
+means=();
+}
+
+```
+
+
+## Verifying that the committer was used
+
+The new committer will write a JSON summary of the operation, including statistics, in the `_SUCCESS` file.
+
+If this file exists and is zero bytes long: the classic `FileOutputCommitter` was used.
+
+If this file exists and is greater than zero bytes long, either the manifest committer was used,
+or in the case of S3A filesystems, one of the S3A committers. They all use the same JSON format.
+
+# <a name="configuration"></a> Configuration options
+
+Here are the main configuration options of the committer.
+
+
+| Option | Meaning | Default Value |
+|--------|---------|---------------|
+| `mapreduce.manifest.committer.delete.target.files` | Delete target files? | `false` |
+| `mapreduce.manifest.committer.io.threads` | Thread count for parallel operations | `64` |
+| `mapreduce.manifest.committer.summary.report.directory` | directory to save reports. | `""` |
+| `mapreduce.manifest.committer.cleanup.parallel.delete` | Delete temporary directories in parallel | `true` |
+| `mapreduce.fileoutputcommitter.cleanup.skipped` | Skip cleanup of `_temporary` directory| `false` |
+| `mapreduce.fileoutputcommitter.cleanup-failures.ignored` | Ignore errors during cleanup | `false` |
+| `mapreduce.fileoutputcommitter.marksuccessfuljobs` | Create a `_SUCCESS` marker file on successful completion. (and delete any existing one in job setup) | `true` |
+
+There are some more, as covered in the (Advanced)[#advanced] section.
+
+
+## <a name="scaling"></a> Scaling jobs `mapreduce.manifest.committer.io.threads`
+
+The core reason that this committer is faster than the classic `FileOutputCommitter`
+is that it tries to parallelize as much file IO as it can during job commit, specifically:
+
+* task manifest loading
+* deletion of files where directories will be created
+* directory creation
+* file-by-file renaming
+* deletion of task attempt directories in job cleanup
+
+These operations are all performed in the same thread pool, whose size is set
+in the option `mapreduce.manifest.committer.io.threads`.
+
+Larger values may be used.
+
+XML
+```xml
+<property>
+  <name>mapreduce.manifest.committer.io.threads</name>
+  <value>200</value>
+</property>
+```
+
+spark-defaults.conf
+```
+spark.hadoop.mapreduce.manifest.committer.io.threads 200
+```
+
+A larger value than that of the number of cores allocated to
+the MapReduce AM or Spark Driver does not directly overload
+the CPUs, as the threads are normally waiting for (slow) IO
+against the object store/filesystem to complete.
+
+Caveats
+* In Spark, multiple jobs may be committed in the same process,
+  each of which will create their own thread pool during job
+  commit or cleanup.
+* Azure rate throttling may be triggered if too many IO requests
+  are made against the store. The rate throttling option
+  `mapreduce.manifest.committer.io.rate` can help avoid this.
+
+
+## <a name="deleting"></a> Optional: deleting target files in Job Commit
+
+The classic `FileOutputCommitter` deletes files at the destination paths
+before renaming the job's files into place.
+
+This is optional in the manifest committers, set in the option
+`mapreduce.manifest.committer.delete.target.files` with a default value of `false`.
+
+This increases performance and is safe to use when all files created by a job
+have unique filenames.
+
+Apache Spark does generate unique filenames for ORC and Parquet since
+[SPARK-8406](https://issues.apache.org/jira/browse/SPARK-8406)
+_Adding UUID to output file name to avoid accidental overwriting_
+
+Avoiding checks for/deleting target files saves one delete call per file being committed, so can
+save a significant amount of store IO.
+
+When appending to existing tables, using formats other than ORC and parquet,
+unless confident that unique identifiers
+are added to each filename, enable deletion of the target files.
+
+```
+spark.hadoop.mapreduce.manifest.committer.delete.target.files true
+```
+
+*Note 1:* the committer will skip deletion operations when it
+created the directory into which a file is to be renamed.
+This makes it slightly more efficient, at least if jobs
+appending data are creating and writing into new partitions.
+
+*Note 2:* the committer still requires tasks within a single
+job to create unique files. This is foundational for
+any job to generate correct data.
+
+# <a name="SUCCESS"></a> Job Summaries in `_SUCCESS` files
+
+The original hadoop committer creates a zero byte `_SUCCESS` file in the root of the output directory
+unless disabled.
+
+This committer writes a JSON summary which includes
+* The name of the committer.
+* Diagnostics information.
+* A list of some of the files created (for testing; a full list is excluded as it can get big).
+* IO Statistics.
+
+If, after running a query, this `_SUCCESS` file is zero bytes long,
+*the new committer has not been used*
+
+If it is not empty, then it can be examined.
+
+## <a name="printer"></a> Viewing `_SUCCESS` file files through the `ManifestPrinter` tool.
+
+The summary files are JSON, and can be viewed in any text editor.
+
+For a more succinct summary, including better display of statistics, use the `ManifestPrinter` tool.
+
+```bash
+hadoop org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter <path>
+```
+
+This works for the files saved at the base of an output directory, and
+any reports saved to a report directory.
+
+## <a name="summaries"></a> Collecting Job Summaries `mapreduce.manifest.committer.summary.report.directory`
+
+The committer can be configured to save the `_SUCCESS` summary files to a report directory,
+irrespective of whether the job succeed or failed, by setting a fileystem path in
+the option `mapreduce.manifest.committer.summary.report.directory`.
+
+The path does not have to be on the same
+store/filesystem as the destination of work. For example, a local fileystem could be used.
+
+XML
+
+```xml
+<property>
+  <name>mapreduce.manifest.committer.summary.report.directory</name>
+  <value>file:///tmp/reports</value>
+</property>
+```
+
+spark-defaults.conf
+
+```
+spark.hadoop.mapreduce.manifest.committer.summary.report.directory file:///tmp/reports
+```
+
+This allows for the statistics of jobs to be collected irrespective of their outcome, Whether or not
+saving the `_SUCCESS` marker is enabled, and without problems caused by a chain of queries
+overwriting the markers.
+
+
+# <a name="cleanup"></a> Cleanup
+
+Job cleanup is convoluted as it is designed to address a number of issues which
+may surface in cloud storage.
+
+* Slow performance for deletion of directories.
+* Timeout when deleting very deep and wide directory trees.
+* General resilience to cleanup issues escalating to job failures.
+
+
+| Option | Meaning | Default Value |
+|--------|---------|---------------|
+| `mapreduce.fileoutputcommitter.cleanup.skipped` | Skip cleanup of `_temporary` directory| `false` |
+| `mapreduce.fileoutputcommitter.cleanup-failures.ignored` | Ignore errors during cleanup | `false` |
+| `mapreduce.manifest.committer.cleanup.parallel.delete` | Delete task attempt directories in parallel | `true` |
+
+The algorithm is:
+
+```
+if `mapreduce.fileoutputcommitter.cleanup.skipped`:
+  return
+if `mapreduce.manifest.committer.cleanup.parallel.delete`:
+  attempt parallel delete of task directories; catch any exception
+if not `mapreduce.fileoutputcommitter.cleanup.skipped`:
+  delete(`_temporary`); catch any exception
+if caught-exception and not `mapreduce.fileoutputcommitter.cleanup-failures.ignored`:
+  throw caught-exception
+```
+
+It's a bit complicated, but the goal is to perform a fast/scalable delete and
+throw a meaningful exception if that didn't work.
+
+When working with ABFS and GCS, these settings should normally be left alone.
+If somehow errors surface during cleanup, enabling the option to
+ignore failures will ensure the job still completes.
+Disabling cleanup even avoids the overhead of cleanup, but
+requires a workflow or manual operation to clean up all
+`_temporary` directories on a regular basis.
+
+
+# <a name="abfs"></a> Working with Azure ADLS Gen2 Storage
+
+To switch to the manifest committer, the factory for committers for destinations with `abfs://` URLs must
+be switched to the manifest committer factory, either for the application or
+the entire cluster.
+
+```xml
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.abfs</name>
+  <value>org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory</value>
+</property>
+```
+
+This allows for ADLS Gen2 -specific performance and consistency logic to be used from within the committer.
+In particular:
+* the [Etag](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/ETag) header
+can be collected in listings and used in the job commit phase.
+* IO rename operations are rate limited
+* recovery is attempted when throttling triggers rename failures.
+
+*Warning* This committer is not compatible with older Azure storage services
+(WASB or ADLS Gen 1).
+
+The core set of Azure-optimized options becomes
+
+```xml
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.abfs</name>
+  <value>org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory</value>
+</property>
+
+<property>
+  <name>spark.hadoop.fs.azure.io.rate.limit</name>
+  <value>10000</value>
+</property>
+```
+
+And optional settings for debugging/performance analysis
+
+```xml
+<property>
+  <name>mapreduce.manifest.committer.summary.report.directory</name>
+  <value>abfs:// Path within same store/separate store</value>
+  <description>Optional: path to where job summaries are saved</description>
+</property>
+```
+
+## <a name="abfs-options"></a> Full set of ABFS options for spark
+
+```
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.abfs org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory
+spark.hadoop.fs.azure.io.rate.limit 10000
+spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
+spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
+
+spark.hadoop.mapreduce.manifest.committer.summary.report.directory  (optional: URI of a directory for job summaries)
+```
+
+## Experimental: ABFS Rename Rate Limiting `fs.azure.io.rate.limit`
+
+To avoid triggering store throttling and backoff delays, as well as other
+throttling-related failure conditions file renames during job commit
+are throttled through a "rate limiter" which limits the number of
+rename operations per second a single instance of the ABFS FileSystem client
+may issue.
+
+| Option | Meaning |
+|--------|---------|
+| `fs.azure.io.rate.limit` | Rate limit in operations/second for IO operations. |
+
+Set the option to `0` remove all rate limiting.
+
+The default value of this is set to 10000, which is the default IO capacity for
+an ADLS storage account.
+
+```xml
+<property>
+  <name>fs.azure.io.rate.limit</name>
+  <value>10000</value>
+  <description>maximum number of renames attempted per second</description>
+</property>
+```
+
+This capacity is set at the level of the filesystem client, and so not
+shared across all processes within a single application, let
+alone other applications sharing the same storage account.
+
+It will be shared with all jobs being committed by the same
+Spark driver, as these do share that filesystem connector.
+
+If rate limiting is imposed, the statistic `store_io_rate_limited` will
+report the time to acquire permits for committing files.
+
+If server-side throttling took place, signs of this can be seen in
+* The store service's logs and their throttling status codes (usually 503 or 500).
+* The job statistic `commit_file_rename_recovered`. This statistic indicates that
+  ADLS throttling manifested as failures in renames, failures which were recovered
+  from in the comitter.
+
+If these are seen -or other applications running at the same time experience
+throttling/throttling-triggered problems, consider reducing the value of
+`fs.azure.io.rate.limit`, and/or requesting a higher IO capacity from Microsoft.
+
+*Important* if you do get extra capacity from Microsoft and you want to use
+it to speed up job commits, increase the value of `fs.azure.io.rate.limit`
+either across the cluster, or specifically for those jobs which you wish
+to allocate extra priority to.
+
+This is still a work in progress; it may be expanded to support
+all IO operations performed by a single filesystem instance.
+
+# <a name="gcs"></a> Working with Google Cloud Storage
+
+The manifest committer is compatible with and tested against Google cloud storage through
+the gcs-connector library from google, which provides a Hadoop filesystem client for the
+schema `gs`.
+
+Google cloud storage has the semantics needed for the commit protocol
+to work safely.
+
+The Spark settings to switch to this committer are
+
+```
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.gs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory
+spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
+spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
+
+spark.hadoop.mapreduce.manifest.committer.summary.report.directory  (optional: URI of a directory for job summaries)
+```
+
+The store's directory delete operations are `O(files)` so the value
+of `mapreduce.manifest.committer.cleanup.parallel.delete`
+SHOULD be left at the default of `true`.
+
+For mapreduce, declare the binding in `core-site.xml`or `mapred-site.xml`
+```xml
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.gcs</name>
+  <value>org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory</value>
+</property>
+```
+
+
+# <a name="hdfs"></a> Working with HDFS
+
+This committer _does_ work with HDFS, it has just been targeted at object stores with
+reduced performance on some operations, especially listing and renaming,
+and semantics too reduced for the classic `FileOutputCommitter` to rely on
+(specifically GCS).
+
+To use on HDFS, set the `ManifestCommitterFactory` as the committer factory for `hdfs://` URLs.
+
+Because HDFS does fast directory deletion, there is no need to parallelize deletion
+of task attempt directories during cleanup, so set
+`mapreduce.manifest.committer.cleanup.parallel.delete` to `false`
+
+The final spark bindings becomes
+
+```
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.hdfs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory
+spark.hadoop.mapreduce.manifest.committer.cleanup.parallel.delete false
+spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
+spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
+
+spark.hadoop.mapreduce.manifest.committer.summary.report.directory  (optional: URI of a directory for job summaries)
+```
+
+# <a name="advanced"></a> Advanced Topics
+
+## Advanced Configuration options
+
+There are some advanced options which are intended for development and testing,
+rather than production use.
+
+| Option | Meaning | Default Value |
+|--------|---------|---------------|
+| `mapreduce.manifest.committer.store.operations.classname` | Classname for Manifest Store Operations | `""` |
+| `mapreduce.manifest.committer.validate.output` | Perform output validation? | `false` |
+
+
+## Validating output  `mapreduce.manifest.committer.validate.output`
+
+The option `mapreduce.manifest.committer.validate.output` triggers a check of every renamed file to
+verify it has the expected length.
+
+This adds the overhead of a `HEAD` request per file, and so is recommended for testing only.
+
+There is no verification of the actual contents.
+
+## Controlling storage integration `mapreduce.manifest.committer.store.operations.classname`
+
+The manifest committer interacts with filesystems through implementations of the interface
+`ManifestStoreOperations`.
+It is possible to provide custom implementations for store-specific features.
+There is one of these for ABFS; when the abfs-specific committer factory is used this
+is automatically set.
+
+It can be explicitly set.
+```xml
+<property>
+  <name>mapreduce.manifest.committer.store.operations.classname</name>
+  <value>org.apache.hadoop.fs.azurebfs.commit.AbfsManifestStoreOperations</value>
+</property>
+```
+
+The default implementation may also be configured.
+
+```xml
+<property>
+  <name>mapreduce.manifest.committer.store.operations.classname</name>
+  <value>org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem</value>
+</property>
+```
+
+There is no need to alter these values, except when writing new implementations for other stores,
+something which is only needed if the store provides extra integration support for the
+committer.
+
+## <a name="concurrent"></a> Support for concurrent test runs.
+
+It *may* be possible to run multiple jobs targeting the same directory tree.
+
+For this to work, a number of conditions must be met:
+
+* When using spark, unique job IDs must be set. This meangs the Spark distribution
+  MUST contain the patches for
+  [SPARK-33402](https://issues.apache.org/jira/browse/SPARK-33402)
+  and
+  [SPARK-33230](https://issues.apache.org/jira/browse/SPARK-33230).
+* Cleanup of the `_temporary` directory must be disabled by setting
+  `mapreduce.fileoutputcommitter.cleanup.skipped` to `true`.
+* All jobs/tasks must create files with unique filenames.
+* All jobs must create output with the same directory partition structure.
+* Remember to delete the `_temporary` directory later!
+
+This has *NOT BEEN TESTED*
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md
new file mode 100644
index 0000000..d2b4f1e
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md
@@ -0,0 +1,335 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+
+# Manifest Committer Architecture
+
+This document describes the architecture and other implementation/correctness
+aspects of the [Manifest Committer](manifest_committer.html)
+
+The protocol and its correctness are covered in [Manifest Committer Protocol](manifest_committer_protocol.html).
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+The _Manifest_ committer is a committer for work which provides performance on ABFS for "real world"
+queries, and performance and correctness on GCS.
+
+This committer uses the extension point which came in for the S3A committers.
+Users can declare a new committer factory for `abfs://` and `gcs://` URLs.
+It can be used through Hadoop MapReduce and Apache Spark.
+
+## Background
+
+### Terminology
+
+| Term | Meaning|
+|------|--------|
+| Committer |  A class which can be invoked by MR/Spark to perform the task and job commit operations. |
+| Spark Driver | The spark process scheduling the work and choreographing the commit operation.|
+| Job  | In MapReduce. the entire application. In spark, this is a single stage in a chain of work |
+| Job Attempt | A single attempt at a job. MR supports multiple Job attempts with recovery on partial job failure. Spark says "start again from scratch" |
+| Task | a subsection of a job, such as processing one file, or one part of a file |
+| Task ID |  ID of the task, unique within this job. Usually starts at 0 and is used in filenames (part-0000, part-001, etc.) |
+| Task attempt (TA) | An attempt to perform a task. It may fail, in which case MR/spark will schedule another. |
+| Task Attempt ID | A unique ID for the task attempt. The Task ID + an attempt counter.|
+|  Destination directory | The final destination of work.|
+| Job Attempt Directory | A temporary directory used by the job attempt. This is always _underneath_ the destination directory, so as to ensure it is in the same encryption zone as HDFS, storage volume in other filesystems, etc.|
+| Task Attempt directory | (also known as "Task Attempt Working Directory"). Directory exclusive for each task attempt under which files are written |
+| Task Commit | Taking the output of a Task Attempt and making it the final/exclusive result of that "successful" Task.|
+| Job Commit | aggregating all the outputs of all committed tasks and producing the final results of the job. |
+
+
+
+The purpose of a committer is to ensure that the complete output of
+a job ends up in the destination, even in the presence of failures of tasks.
+
+- _Complete:_ the output includes the work of all successful tasks.
+- _Exclusive:_ the output of unsuccessful tasks is not present.
+- _Concurrent:_ When multiple tasks are committed in parallel the output is the same as when
+  the task commits are serialized. This is not a requirement of Job Commit.
+- _Abortable:_  jobs and tasks may be aborted prior to job commit, after which their output is not visible.
+- _Continuity of correctness:_ once a job is committed, the output of any failed,
+  aborted, or unsuccessful task MUST NO appear at some point in the future.
+
+For Hive's classic hierarchical-directory-structured tables, job committing
+requires the output of all committed tasks to be put into the correct location
+in the directory tree.
+
+The committer built into `hadoop-mapreduce-client-core` module is the `FileOutputCommitter`.
+
+
+
+## The Manifest Committer: A high performance committer for Spark on Azure and Google storage.
+
+The Manifest Committer is a higher performance committer for ABFS and GCS storage
+for jobs which create file across deep directory trees through many tasks.
+
+It will also work on `hdfs://` and indeed, `file://` URLs, but
+it is optimized to address listing and renaming performance and throttling
+issues in cloud storage.
+
+It *will not* work correctly with S3, because it relies on an atomic rename-no-overwrite
+operation to commit the manifest file. It will also have the performance
+problems of copying rather than moving all the generated data.
+
+Although it will work with MapReduce
+there is no handling of multiple job attempts with recovery from previous failed
+attempts.
+
+### The Manifest
+
+A Manifest file is designed which contains (along with IOStatistics and some
+other things)
+
+1. A list of destination directories which must be created if they do not exist.
+1. A list of files to rename, recorded as (absolute source, absolute destination,
+   file-size) entries.
+
+### Task Commit
+
+Task attempts are committed by:
+
+1. Recursively listing the task attempt working dir to build
+   1. A list of directories under which files are renamed.
+   2. A list of files to rename: source, destination, size and optionally, etag.
+2. Saving this information in a manifest file in the job attempt directory with
+   a filename derived from the Task ID.
+   Note: writing to a temp file and then renaming to the final path will be used
+   to ensure the manifest creation is atomic.
+
+
+No renaming takes place —the files are left in their original location.
+
+The directory treewalk is single-threaded, then it is `O(directories)`,
+with each directory listing using one or more paged LIST calls.
+
+This is simple, and for most tasks, the scan is off the critical path of of the job.
+
+Statistics analysis may justify moving to parallel scans in future.
+
+
+### Job Commit
+
+Job Commit consists of:
+
+1. List all manifest files in the job attempt directory.
+1. Load each manifest file, create directories which do not yet exist, then
+   rename each file in the rename list.
+1. Save a JSON `_SUCCESS` file with the same format as the S3A committer (for
+   testing; use write and rename for atomic save)
+
+The job commit phase supports parallelization for many tasks and many files
+per task, specifically:
+
+1. Manifest tasks are loaded and processed in a pool of "manifest processor"
+   threads.
+2. Directory creation and file rename operations are each processed in a pool of "
+   executor" threads: many renames can execute in parallel as they use minimal
+   network IO.
+3. job cleanup can parallelize deletion of task attempt directories. This
+   is relevant as directory deletion is `O(files)` on Google cloud storage,
+   and also on ABFS when OAuth authentication is used.
+
+
+### Ancestor directory preparation
+
+Optional scan of all ancestors ...if any are files, delete.
+
+
+### Parent directory creation
+
+1. Probe shared directory map for directory existing. If found: operation is
+   complete.
+1. if the map is empty, call `getFileStatus()` on the path. Not found: create
+   directory, add entry and those of all parent paths Found and is directory:
+   add entry and those of all parent paths Found and is file: delete. then
+   create as before.
+
+Efficiently handling concurrent creation of directories (or delete+create) is going to be a
+troublespot; some effort is invested there to build the set of directories to
+create.
+
+### File Rename
+
+Files are renamed in parallel.
+
+A pre-rename check for anything being at that path (and deleting it) will be optional.
+With spark creating new UUIDs for each file, this isn't going to happen, and
+saves HTTP requests.
+
+
+### Validation
+
+Optional scan of all committed files and verify length and, if known,
+etag. For testing and diagnostics.
+
+## Benefits
+
+* Pushes the source tree list operations into the task commit phase, which is
+  generally off the critical path of execution
+* Provides an atomic task commit to GCS, as there is no expectation that
+  directory rename is atomic
+* It is possible to pass IOStatistics from workers in manifest.
+* Allows for some pre-rename operations similar to the S3A "Partitioned Staging
+  committer". This can be configured to delete all existing entries in
+  directories scheduled to be created -or fail if those partitions are
+  non-empty.
+  See [Partitioned Staging Committer](../../hadoop-aws/tools/hadoop-aws/committers.html#The_.E2.80.9CPartitioned.E2.80.9D_Staging_Committer)
+* Allows for an optional preflight validation check (verify no duplicate files created by different tasks)
+* Manifests can be viewed, size of output determined, etc, during
+  development/debugging.
+
+### Disadvantages
+
+* Needs a new manifest file format.
+* May makes task commit more complex.
+
+This solution is necessary for GCS and should be beneficial on ABFS as listing
+overheads are paid for in the task committers.
+
+# Implementation Details
+
+### Constraints
+
+A key goal is to keep the manifest committer isolated and neither
+touch the existing committer code nor other parts of the hadoop codebase.
+
+It must plug directly into MR and Spark without needing any changes
+other than already implemented for the S3A Committers
+
+* Self-contained: MUST NOT require changes to hadoop-common, etc.
+* Isolated: MUST NOT make changes to existing committers
+* Integrated: MUST bind via `PathOutputCommitterFactory`.
+
+As a result of this there's a bit of copy and paste from elsewhere,
+e.g. `org.apache.hadoop.util.functional.TaskPool`
+is based on S3ACommitter's `org.apache.hadoop.fs.s3a.commit.Tasks`.
+
+The` _SUCCESS` file MUST be compatible with the S3A JSON file.
+This is to ensure any existing test suites which validate
+S3A committer output can be retargeted at jobs executed
+by the manifest committer without any changes.
+
+
+#### Progress callbacks in job commit.
+
+When? Proposed: heartbeat until renaming finally finishes.
+
+#### Error handling and aborting in job commit.
+
+We would want to stop the entire job commit. Some atomic boolean "abort job"
+would need to be checked in the processing of each task committer thread's
+iteraton through a directory (or processing of each file?)
+Failures in listing or renaming will need to be escalated to halting the entire
+job commit. This implies that any IOE raised in asynchronous rename operation or
+in a task committer thread must:
+
+1. be caught
+1. be stored in a shared field/variable
+1. trigger the abort
+1. be rethrown at the end of the `commitJob()` call
+
+#### Avoiding deadlocks
+
+If a job commit stage is using a thread pool for per-task operations, e.g. loading
+files, that same thread pool MUST NOT be used for parallel operations within
+the per-task stage.
+
+As every `JobStage` is executed in sequence within task or job commit, it
+is safe to share the same thread pool across stages.
+
+In the current implementation, there is no parallel "per manifest" operation
+in job commit other than for actually loading the files.
+The operations to create directories and to rename files are actually
+performed without performing parallel processing of individual manifests.
+
+Directory Preparation: merge the directory lists of all manifests,
+then queue for creation the (hopefully very much smaller) set of unique
+directories.
+
+Rename: iterate through all manifests and queue their renames into a pool for
+renaming.
+
+#### Thread pool lifetimes
+
+The lifespan of thread pools is constrained to that of the stage configuration,
+which will be limited to within each of the `PathOutputCommitter` methods
+to setup, commit, abort and cleanup.
+
+This avoids the thread pool lifecycle problems of the S3A Committers.
+
+#### Scale issues similar to S3A HADOOP-16570.
+
+This was a failure in terasorting where many tasks each generated many files;
... 7102 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org