Browse Source

HADOOP-13786 Add S3A committer for zero-rename commits to S3 endpoints.
Contributed by Steve Loughran and Ryan Blue.

Steve Loughran 7 years ago
parent
commit
de8b6ca5ef
100 changed files with 15163 additions and 627 deletions
  1. 7 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  2. 9 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
  3. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathExistsException.java
  4. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java
  5. 299 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java
  6. 116 1
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  7. 47 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  8. 23 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  9. 45 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/HadoopTestBase.java
  10. 130 14
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
  11. 185 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java
  12. 21 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java
  13. 184 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/BindingPathOutputCommitter.java
  14. 3 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
  15. 38 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitterFactory.java
  16. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
  17. 79 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/NamedCommitterFactory.java
  18. 17 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
  19. 204 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java
  20. 22 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  21. 22 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java
  22. 495 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitterFactory.java
  23. 40 6
      hadoop-tools/hadoop-aws/pom.xml
  24. 42 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java
  25. 2 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java
  26. 31 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java
  27. 38 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java
  28. 42 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java
  29. 37 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
  30. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
  31. 57 15
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  32. 197 35
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
  33. 485 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
  34. 7 19
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
  35. 92 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java
  36. 172 135
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
  37. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
  38. 379 128
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  39. 36 20
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
  40. 224 7
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
  41. 246 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
  42. 10 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
  43. 308 16
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  44. 11 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java
  45. 5 5
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
  46. 52 4
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
  47. 474 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
  48. 756 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
  49. 90 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java
  50. 240 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java
  51. 596 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
  52. 129 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java
  53. 192 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java
  54. 60 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Duration.java
  55. 59 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/DurationInfo.java
  56. 100 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java
  57. 80 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/LocalTempDir.java
  58. 182 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
  59. 229 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitPaths.java
  60. 43 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PathCommitException.java
  61. 100 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
  62. 129 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java
  63. 410 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java
  64. 53 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/ValidationFailure.java
  65. 192 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
  66. 69 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java
  67. 432 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
  68. 322 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java
  69. 45 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java
  70. 161 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
  71. 288 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
  72. 47 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitterFactory.java
  73. 27 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/package-info.java
  74. 28 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/package-info.java
  75. 33 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/ConflictResolution.java
  76. 116 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java
  77. 48 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitterFactory.java
  78. 159 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java
  79. 48 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitterFactory.java
  80. 300 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
  81. 851 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
  82. 64 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java
  83. 49 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterFactory.java
  84. 27 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/package-info.java
  85. 253 115
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
  86. 2 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
  87. 15 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
  88. 1951 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md
  89. 819 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md
  90. 172 16
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  91. 5 2
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
  92. 60 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
  93. 124 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
  94. 11 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
  95. 4 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
  96. 21 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
  97. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  98. 2 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
  99. 16 33
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
  100. 2 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java

+ 7 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -446,4 +446,11 @@
     <Method name="setInstance"/>
     <Bug pattern="ME_ENUM_FIELD_SETTER"/>
   </Match>
+
+  <!-- findbugs is complaining that a stream isn't closed. It will be. -->
+  <Match>
+    <Class name="org.apache.hadoop.util.JsonSerialization"/>
+    <Method name="save"/>
+    <Bug pattern="OBL_UNSATISFIED_OBLIGATION"/>
+  </Match>
 </FindBugsFilter>

+ 9 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java

@@ -101,6 +101,15 @@ public class FSDataOutputStream extends DataOutputStream
     out.close(); // This invokes PositionCache.close()
   }
 
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "FSDataOutputStream{");
+    sb.append("wrappedStream=").append(wrappedStream);
+    sb.append('}');
+    return sb.toString();
+  }
+
   /**
    * Get a reference to the wrapped output stream.
    *

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathExistsException.java

@@ -27,7 +27,7 @@ public class PathExistsException extends PathIOException {
     super(path, "File exists");
   }
   
-  protected PathExistsException(String path, String error) {
+  public PathExistsException(String path, String error) {
     super(path, error);
   }
-}
+}

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java

@@ -97,6 +97,11 @@ public abstract class StorageStatistics {
     public long getValue() {
       return value;
     }
+
+    @Override
+    public String toString() {
+      return name + " = " + value;
+    }
   }
 
   private final String name;

+ 299 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java

@@ -0,0 +1,299 @@
+/*
+ * 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.util;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.google.common.base.Preconditions;
+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.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Support for marshalling objects to and from JSON.
+ *
+ * It constructs an object mapper as an instance field.
+ * and synchronizes access to those methods
+ * which use the mapper.
+ *
+ * This class was extracted from
+ * {@code org.apache.hadoop.registry.client.binding.JsonSerDeser},
+ * which is now a subclass of this class.
+ * @param <T> Type to marshal.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class JsonSerialization<T> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JsonSerialization.class);
+  private static final String UTF_8 = "UTF-8";
+
+  private final Class<T> classType;
+  private final ObjectMapper mapper;
+
+  /**
+   * Create an instance bound to a specific type.
+   * @param classType class to marshall
+   * @param failOnUnknownProperties fail if an unknown property is encountered.
+   * @param pretty generate pretty (indented) output?
+   */
+  public JsonSerialization(Class<T> classType,
+      boolean failOnUnknownProperties, boolean pretty) {
+    Preconditions.checkArgument(classType != null, "null classType");
+    this.classType = classType;
+    this.mapper = new ObjectMapper();
+    mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES,
+        failOnUnknownProperties);
+    mapper.configure(SerializationFeature.INDENT_OUTPUT, pretty);
+  }
+
+  /**
+   * Get the simple name of the class type to be marshalled.
+   * @return the name of the class being marshalled
+   */
+  public String getName() {
+    return classType.getSimpleName();
+  }
+
+  /**
+   * Convert from JSON.
+   *
+   * @param json input
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonParseException If the input is not well-formatted
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings("unchecked")
+  public synchronized T fromJson(String json)
+      throws IOException, JsonParseException, JsonMappingException {
+    if (json.isEmpty()) {
+      throw new EOFException("No data");
+    }
+    try {
+      return mapper.readValue(json, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json : {}\n{}", e, json, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Read from an input stream.
+   * @param stream stream to read from
+   * @return the parsed entity
+   * @throws IOException IO problems
+   * @throws JsonParseException If the input is not well-formatted
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  public synchronized T fromJsonStream(InputStream stream) throws IOException {
+    return mapper.readValue(stream, classType);
+  }
+
+  /**
+   * Load from a JSON text file.
+   * @param jsonFile input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonParseException If the input is not well-formatted
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings("unchecked")
+  public synchronized T load(File jsonFile)
+      throws IOException, JsonParseException, JsonMappingException {
+    if (!jsonFile.isFile()) {
+      throw new FileNotFoundException("Not a file: " + jsonFile);
+    }
+    if (jsonFile.length() == 0) {
+      throw new EOFException("File is empty: " + jsonFile);
+    }
+    try {
+      return mapper.readValue(jsonFile, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json file {}", jsonFile, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Save to a local file. Any existing file is overwritten unless
+   * the OS blocks that.
+   * @param file file
+   * @param path path
+   * @throws IOException IO exception
+   */
+  public void save(File file, T instance) throws
+      IOException {
+    writeJsonAsBytes(instance, new FileOutputStream(file));
+  }
+
+  /**
+   * Convert from a JSON file.
+   * @param resource input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonParseException If the input is not well-formatted
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings({"IOResourceOpenedButNotSafelyClosed"})
+  public synchronized T fromResource(String resource)
+      throws IOException, JsonParseException, JsonMappingException {
+    try (InputStream resStream = this.getClass()
+        .getResourceAsStream(resource)) {
+      if (resStream == null) {
+        throw new FileNotFoundException(resource);
+      }
+      return mapper.readValue(resStream, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json resource {}", resource, e);
+      throw e;
+    }
+  }
+
+  /**
+   * clone by converting to JSON and back again.
+   * This is much less efficient than any Java clone process.
+   * @param instance instance to duplicate
+   * @return a new instance
+   * @throws IOException IO problems.
+   */
+  public T fromInstance(T instance) throws IOException {
+    return fromJson(toJson(instance));
+  }
+
+  /**
+   * Load from a Hadoop filesystem.
+   * There's a check for data availability after the file is open, by
+   * raising an EOFException if stream.available == 0.
+   * This allows for a meaningful exception without the round trip overhead
+   * of a getFileStatus call before opening the file. It may be brittle
+   * against an FS stream which doesn't return a value here, but the
+   * standard filesystems all do.
+   * JSON parsing and mapping problems
+   * are converted to IOEs.
+   * @param fs filesystem
+   * @param path path
+   * @return a loaded object
+   * @throws IOException IO or JSON parse problems
+   */
+  public T load(FileSystem fs, Path path) throws IOException {
+    try (FSDataInputStream dataInputStream = fs.open(path)) {
+      // throw an EOF exception if there is no data available.
+      if (dataInputStream.available() == 0) {
+        throw new EOFException("No data in " + path);
+      }
+      return fromJsonStream(dataInputStream);
+    } catch (JsonProcessingException e) {
+      throw new IOException(
+          String.format("Failed to read JSON file \"%s\": %s", path, e),
+          e);
+    }
+  }
+
+  /**
+   * Save to a Hadoop filesystem.
+   * @param fs filesystem
+   * @param path path
+   * @param overwrite should any existing file be overwritten
+   * @throws IOException IO exception
+   */
+  public void save(FileSystem fs, Path path, T instance,
+      boolean overwrite) throws
+      IOException {
+    writeJsonAsBytes(instance, fs.create(path, overwrite));
+  }
+
+  /**
+   * Write the JSON as bytes, then close the file.
+   * @param dataOutputStream an output stream that will always be closed
+   * @throws IOException on any failure
+   */
+  private void writeJsonAsBytes(T instance,
+      OutputStream dataOutputStream) throws IOException {
+    try {
+      dataOutputStream.write(toBytes(instance));
+    } finally {
+      dataOutputStream.close();
+    }
+  }
+
+  /**
+   * Convert JSON to bytes.
+   * @param instance instance to convert
+   * @return a byte array
+   * @throws IOException IO problems
+   */
+  public byte[] toBytes(T instance) throws IOException {
+    return mapper.writeValueAsBytes(instance);
+  }
+
+  /**
+   * Deserialize from a byte array.
+   * @param bytes byte array
+   * @throws IOException IO problems
+   * @throws EOFException not enough data
+   */
+  public T fromBytes(byte[] bytes) throws IOException {
+    return fromJson(new String(bytes, 0, bytes.length, UTF_8));
+  }
+
+  /**
+   * Convert an instance to a JSON string.
+   * @param instance instance to convert
+   * @return a JSON string description
+   * @throws JsonProcessingException Json generation problems
+   */
+  public synchronized String toJson(T instance) throws JsonProcessingException {
+    return mapper.writeValueAsString(instance);
+  }
+
+  /**
+   * Convert an instance to a string form for output. This is a robust
+   * operation which will convert any JSON-generating exceptions into
+   * error text.
+   * @param instance non-null instance
+   * @return a JSON string
+   */
+  public String toString(T instance) {
+    Preconditions.checkArgument(instance != null, "Null instance argument");
+    try {
+      return toJson(instance);
+    } catch (JsonProcessingException e) {
+      return "Failed to convert to a string: " + e;
+    }
+  }
+}

+ 116 - 1
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1123,7 +1123,8 @@
 <property>
   <name>fs.s3a.multipart.purge.age</name>
   <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge.
+  <description>Minimum age in seconds of multipart uploads to purge
+    on startup if "fs.s3a.multipart.purge" is true
   </description>
 </property>
 
@@ -1344,6 +1345,120 @@
   </description>
 </property>
 
+<property>
+  <name>fs.s3a.retry.limit</name>
+  <value>${fs.s3a.attempts.maximum}</value>
+  <description>
+    Number of times to retry any repeatable S3 client request on failure,
+    excluding throttling requests.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.interval</name>
+  <value>500ms</value>
+  <description>
+    Interval between attempts to retry operations for any reason other
+    than S3 throttle errors.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.throttle.limit</name>
+  <value>${fs.s3a.attempts.maximum}</value>
+  <description>
+    Number of times to retry any throttled request.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.throttle.interval</name>
+  <value>1000ms</value>
+  <description>
+    Interval between retry attempts on throttled requests.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.name</name>
+  <value>file</value>
+  <description>
+    Committer to create for output to S3A, one of:
+    "file", "directory", "partitioned", "magic".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.magic.enabled</name>
+  <value>false</value>
+  <description>
+    Enable support in the filesystem for the S3 "Magic" committer.
+    When working with AWS S3, S3Guard must be enabled for the destination
+    bucket, as consistent metadata listings are required.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.threads</name>
+  <value>8</value>
+  <description>
+    Number of threads in committers for parallel operations on files
+    (upload, commit, abort, delete...)
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.tmp.path</name>
+  <value>tmp/staging</value>
+  <description>
+    Path in the cluster filesystem for temporary data.
+    This is for HDFS, not the local filesystem.
+    It is only for the summary data of each file, not the actual
+    data being committed.
+    Using an unqualified path guarantees that the full path will be
+    generated relative to the home directory of the user creating the job,
+    hence private (assuming home directory permissions are secure).
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.unique-filenames</name>
+  <value>true</value>
+  <description>
+    Option for final files to have a unique name through job attempt info,
+    or the value of fs.s3a.committer.staging.uuid
+    When writing data with the "append" conflict option, this guarantees
+    that new data will not overwrite any existing data.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.conflict-mode</name>
+  <value>fail</value>
+  <description>
+    Staging committer conflict resolution policy.
+    Supported: "fail", "append", "replace".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.abort.pending.uploads</name>
+  <value>true</value>
+  <description>
+    Should the staging committers abort all pending uploads to the destination
+    directory?
+
+    Changing this if more than one partitioned committer is
+    writing to the same destination tree simultaneously; otherwise
+    the first job to complete will cancel all outstanding uploads from the
+    others. However, it may lead to leaked outstanding uploads from failed
+    tasks. If disabled, configure the bucket lifecycle to remove uploads
+    after a time period, and/or set up a workflow to explicitly delete
+    entries. Otherwise there is a risk that uncommitted uploads may run up
+    bills.
+  </description>
+</property>
+
 <property>
   <name>fs.AbstractFileSystem.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3A</value>

+ 47 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -725,6 +725,8 @@ public class ContractTestUtils extends Assert {
 
   /**
    * Read in "length" bytes, convert to an ascii string.
+   * This uses {@link #toChar(byte)} to escape bytes, so cannot be used
+   * for round trip operations.
    * @param fs filesystem
    * @param path path to read
    * @param length #of bytes to read.
@@ -741,6 +743,28 @@ public class ContractTestUtils extends Assert {
     }
   }
 
+  /**
+   * Read in "length" bytes, convert to UTF8 string.
+   * @param fs filesystem
+   * @param path path to read
+   * @param length #of bytes to read. If -1: use file length.
+   * @return the bytes read and converted to a string
+   * @throws IOException IO problems
+   */
+  public static String readUTF8(FileSystem fs,
+                                  Path path,
+                                  int length) throws IOException {
+    if (length < 0) {
+      FileStatus status = fs.getFileStatus(path);
+      length = (int) status.getLen();
+    }
+    try (FSDataInputStream in = fs.open(path)) {
+      byte[] buf = new byte[length];
+      in.readFully(0, buf);
+      return new String(buf, "UTF-8");
+    }
+  }
+
   /**
    * Take an array of filestats and convert to a string
    * (prefixed with/ a [%02d] counter).
@@ -857,11 +881,30 @@ public class ContractTestUtils extends Assert {
    */
   public static void assertPathExists(FileSystem fileSystem, String message,
                                Path path) throws IOException {
-    if (!fileSystem.exists(path)) {
+    verifyPathExists(fileSystem, message, path);
+  }
+
+  /**
+   * Verify that a path exists, returning the file status of the path.
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws FileNotFoundException raised if the path is missing
+   * @throws IOException IO problems
+   */
+  public static FileStatus verifyPathExists(FileSystem fileSystem,
+      String message,
+      Path path) throws IOException {
+    try {
+      return fileSystem.getFileStatus(path);
+    } catch (FileNotFoundException e) {
       //failure, report it
-      ls(fileSystem, path.getParent());
-      throw new FileNotFoundException(message + ": not found " + path
-                                      + " in " + path.getParent());
+      LOG.error("{}: not found {}; parent listing is:\n{}",
+          message, path, ls(fileSystem, path.getParent()));
+      throw (IOException)new FileNotFoundException(
+          message + ": not found " + path + " in " + path.getParent())
+          .initCause(e);
     }
   }
 

+ 23 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -316,20 +316,37 @@ public abstract class GenericTestUtils {
   /**
    * Assert that an exception's <code>toString()</code> value
    * contained the expected text.
-   * @param string expected string
+   * @param expectedText expected string
    * @param t thrown exception
    * @throws AssertionError if the expected string is not found
    */
-  public static void assertExceptionContains(String string, Throwable t) {
+  public static void assertExceptionContains(String expectedText, Throwable t) {
+    assertExceptionContains(expectedText, t, "");
+  }
+
+  /**
+   * Assert that an exception's <code>toString()</code> value
+   * contained the expected text.
+   * @param expectedText expected string
+   * @param t thrown exception
+   * @param message any extra text for the string
+   * @throws AssertionError if the expected string is not found
+   */
+  public static void assertExceptionContains(String expectedText,
+      Throwable t,
+      String message) {
     Assert.assertNotNull(E_NULL_THROWABLE, t);
     String msg = t.toString();
     if (msg == null) {
       throw new AssertionError(E_NULL_THROWABLE_STRING, t);
     }
-    if (!msg.contains(string)) {
-      throw new AssertionError("Expected to find '" + string + "' "
-          + E_UNEXPECTED_EXCEPTION + ":"
-          + StringUtils.stringifyException(t),
+    if (expectedText != null && !msg.contains(expectedText)) {
+      String prefix = org.apache.commons.lang.StringUtils.isEmpty(message)
+          ? "" : (message + ": ");
+      throw new AssertionError(
+          String.format("%s Expected to find '%s' %s: %s",
+              prefix, expectedText, E_UNEXPECTED_EXCEPTION,
+              StringUtils.stringifyException(t)),
           t);
     }
   }  

+ 45 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/HadoopTestBase.java

@@ -17,29 +17,37 @@
  */
 package org.apache.hadoop.test;
 
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
+import org.junit.rules.TestName;
 import org.junit.rules.Timeout;
 
 /**
  * A base class for JUnit4 tests that sets a default timeout for all tests
- * that subclass this test
+ * that subclass this test.
+ *
+ * Threads are named to the method being executed, for ease of diagnostics
+ * in logs and thread dumps.
  */
-public abstract class HadoopTestBase {
+public abstract class HadoopTestBase extends Assert {
+
   /**
-   * System property name to set the test timeout: {@value}
+   * System property name to set the test timeout: {@value}.
    */
   public static final String PROPERTY_TEST_DEFAULT_TIMEOUT =
-    "test.default.timeout";
+      "test.default.timeout";
 
   /**
    * The default timeout (in milliseconds) if the system property
    * {@link #PROPERTY_TEST_DEFAULT_TIMEOUT}
-   * is not set: {@value}
+   * is not set: {@value}.
    */
   public static final int TEST_DEFAULT_TIMEOUT_VALUE = 100000;
 
   /**
-   * The JUnit rule that sets the default timeout for tests
+   * The JUnit rule that sets the default timeout for tests.
    */
   @Rule
   public Timeout defaultTimeout = retrieveTestTimeout();
@@ -64,4 +72,35 @@ public abstract class HadoopTestBase {
     }
     return new Timeout(millis);
   }
+
+  /**
+   * The method name.
+   */
+  @Rule
+  public TestName methodName = new TestName();
+
+  /**
+   * Get the method name; defaults to the value of {@link #methodName}.
+   * Subclasses may wish to override it, which will tune the thread naming.
+   * @return the name of the method.
+   */
+  protected String getMethodName() {
+    return methodName.getMethodName();
+  }
+
+  /**
+   * Static initializer names this thread "JUnit".
+   */
+  @BeforeClass
+  public static void nameTestThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * Before each method, the thread is renamed to match the method name.
+   */
+  @Before
+  public void nameThreadToMethod() {
+    Thread.currentThread().setName("JUnit-" + getMethodName());
+  }
 }

+ 130 - 14
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java

@@ -19,11 +19,13 @@
 package org.apache.hadoop.test;
 
 import com.google.common.base.Preconditions;
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.util.Time;
 
+import java.util.Optional;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeoutException;
 
@@ -43,7 +45,7 @@ import java.util.concurrent.TimeoutException;
  * check; useful when checking the contents of the exception.
  */
 public final class LambdaTestUtils {
-  public static final Logger LOG =
+  private static final Logger LOG =
       LoggerFactory.getLogger(LambdaTestUtils.class);
 
   private LambdaTestUtils() {
@@ -60,6 +62,7 @@ public final class LambdaTestUtils {
    * Interface to implement for converting a timeout into some form
    * of exception to raise.
    */
+  @FunctionalInterface
   public interface TimeoutHandler {
 
     /**
@@ -371,16 +374,11 @@ public final class LambdaTestUtils {
       Class<E> clazz,
       Callable<T> eval)
       throws Exception {
-    try {
-      T result = eval.call();
-      throw new AssertionError("Expected an exception, got "
-          + robustToString(result));
-    } catch (Throwable e) {
-      if (clazz.isAssignableFrom(e.getClass())) {
-        return (E)e;
-      }
-      throw e;
-    }
+    return intercept(clazz,
+        null,
+        "Expected a " + clazz.getName() + " to be thrown," +
+            " but got the result: ",
+        eval);
   }
 
   /**
@@ -450,6 +448,59 @@ public final class LambdaTestUtils {
     return ex;
   }
 
+  /**
+   * Intercept an exception; throw an {@code AssertionError} if one not raised.
+   * The caught exception is rethrown if it is of the wrong class or
+   * does not contain the text defined in {@code contained}.
+   * <p>
+   * Example: expect deleting a nonexistent file to raise a
+   * {@code FileNotFoundException} with the {@code toString()} value
+   * containing the text {@code "missing"}.
+   * <pre>
+   * FileNotFoundException ioe = intercept(FileNotFoundException.class,
+   *   "missing",
+   *   "path should not be found",
+   *   () -> {
+   *     filesystem.delete(new Path("/missing"), false);
+   *   });
+   * </pre>
+   *
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param message any message tho include in exception/log messages
+   * @param eval expression to eval
+   * @param <T> return type of expression
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type and contents
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   * The error includes the {@code toString()} value of the result, if this
+   * can be determined.
+   * @see GenericTestUtils#assertExceptionContains(String, Throwable)
+   */
+  public static <T, E extends Throwable> E intercept(
+      Class<E> clazz,
+      String contained,
+      String message,
+      Callable<T> eval)
+      throws Exception {
+    E ex;
+    try {
+      T result = eval.call();
+      throw new AssertionError(message + ": " + robustToString(result));
+    } catch (Throwable e) {
+      if (!clazz.isAssignableFrom(e.getClass())) {
+        throw e;
+      } else {
+        ex = (E) e;
+      }
+    }
+    GenericTestUtils.assertExceptionContains(contained, ex, message);
+    return ex;
+  }
+
   /**
    * Variant of {@link #intercept(Class, Callable)} to simplify void
    * invocations.
@@ -468,9 +519,41 @@ public final class LambdaTestUtils {
       String contained,
       VoidCallable eval)
       throws Exception {
-    E ex = intercept(clazz, eval);
-    GenericTestUtils.assertExceptionContains(contained, ex);
-    return ex;
+    return intercept(clazz, contained,
+        "Expecting " + clazz.getName()
+        + (contained != null? (" with text " + contained) : "")
+        + " but got ",
+        () -> {
+          eval.call();
+          return "void";
+        });
+  }
+
+  /**
+   * Variant of {@link #intercept(Class, Callable)} to simplify void
+   * invocations.
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param message any message tho include in exception/log messages
+   * @param eval expression to eval
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   */
+  public static <E extends Throwable> E intercept(
+      Class<E> clazz,
+      String contained,
+      String message,
+      VoidCallable eval)
+      throws Exception {
+    return intercept(clazz, contained, message,
+        () -> {
+          eval.call();
+          return "void";
+        });
   }
 
   /**
@@ -494,6 +577,38 @@ public final class LambdaTestUtils {
     }
   }
 
+  /**
+   * Assert that an optional value matches an expected one;
+   * checks include null and empty on the actual value.
+   * @param message message text
+   * @param expected expected value
+   * @param actual actual optional value
+   * @param <T> type
+   */
+  public static <T> void assertOptionalEquals(String message,
+      T expected,
+      Optional<T> actual) {
+    Assert.assertNotNull(message, actual);
+    Assert.assertTrue(message +" -not present", actual.isPresent());
+    Assert.assertEquals(message, expected, actual.get());
+  }
+
+  /**
+   * Assert that an optional value matches an expected one;
+   * checks include null and empty on the actual value.
+   * @param message message text
+   * @param expected expected value
+   * @param actual actual optional value
+   * @param <T> type
+   */
+  public static <T> void assertOptionalUnset(String message,
+      Optional<T> actual) {
+    Assert.assertNotNull(message, actual);
+    if (actual.isPresent()) {
+      Assert.fail("Expected empty option, got " + actual.get().toString());
+    }
+  }
+
   /**
    * Returns {@code TimeoutException} on a timeout. If
    * there was a inner class passed in, includes it as the
@@ -638,6 +753,7 @@ public final class LambdaTestUtils {
    * A simple interface for lambdas, which returns nothing; this exists
    * to simplify lambda tests on operations with no return value.
    */
+  @FunctionalInterface
   public interface VoidCallable {
     void call() throws Exception;
   }

+ 185 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java

@@ -0,0 +1,185 @@
+/*
+ * 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.util;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.HadoopTestBase;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+/**
+ * Test the JSON serialization helper.
+ */
+public class TestJsonSerialization extends HadoopTestBase {
+
+  private final JsonSerialization<KeyVal> serDeser =
+      new JsonSerialization<>(KeyVal.class, true, true);
+
+  private final KeyVal source = new KeyVal("key", "1");
+
+  private static class KeyVal implements Serializable {
+    private String name;
+    private String value;
+
+    KeyVal(String name, String value) {
+      this.name = name;
+      this.value = value;
+    }
+
+    KeyVal() {
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("SimpleJson{");
+      sb.append("name='").append(name).append('\'');
+      sb.append(", value='").append(value).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;
+      }
+      KeyVal that = (KeyVal) o;
+      return Objects.equals(name, that.name) &&
+          Objects.equals(value, that.value);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(name, value);
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public void setName(String name) {
+      this.name = name;
+    }
+
+    public String getValue() {
+      return value;
+    }
+
+    public void setValue(String value) {
+      this.value = value;
+    }
+  }
+
+  @Test
+  public void testStringRoundTrip() throws Throwable {
+    String wire = serDeser.toJson(source);
+    KeyVal unmarshalled = serDeser.fromJson(wire);
+    assertEquals("Failed to unmarshall: " + wire, source, unmarshalled);
+  }
+
+  @Test
+  public void testBytesRoundTrip() throws Throwable {
+    byte[] wire = serDeser.toBytes(source);
+    KeyVal unmarshalled = serDeser.fromBytes(wire);
+    assertEquals(source, unmarshalled);
+  }
+
+  @Test
+  public void testBadBytesRoundTrip() throws Throwable {
+    LambdaTestUtils.intercept(JsonParseException.class,
+        "token",
+        () -> serDeser.fromBytes(new byte[]{'a'}));
+  }
+
+  @Test
+  public void testCloneViaJson() throws Throwable {
+    KeyVal unmarshalled = serDeser.fromInstance(source);
+    assertEquals(source, unmarshalled);
+  }
+
+  @Test
+  public void testFileRoundTrip() throws Throwable {
+    File tempFile = File.createTempFile("Keyval", ".json");
+    tempFile.delete();
+    try {
+      serDeser.save(tempFile, source);
+      assertEquals(source, serDeser.load(tempFile));
+    } finally {
+      tempFile.delete();
+    }
+  }
+
+  @Test
+  public void testEmptyFile() throws Throwable {
+    File tempFile = File.createTempFile("Keyval", ".json");
+    try {
+      LambdaTestUtils.intercept(EOFException.class,
+          "empty",
+          () -> serDeser.load(tempFile));
+    } finally {
+      tempFile.delete();
+    }
+  }
+
+  @Test
+  public void testFileSystemRoundTrip() throws Throwable {
+    File tempFile = File.createTempFile("Keyval", ".json");
+    tempFile.delete();
+    Path tempPath = new Path(tempFile.toURI());
+    LocalFileSystem fs = FileSystem.getLocal(new Configuration());
+    try {
+      serDeser.save(fs, tempPath, source, false);
+      assertEquals(source, serDeser.load(fs, tempPath));
+    } finally {
+      fs.delete(tempPath, false);
+    }
+  }
+
+  @Test
+  public void testFileSystemEmptyPath() throws Throwable {
+    File tempFile = File.createTempFile("Keyval", ".json");
+    Path tempPath = new Path(tempFile.toURI());
+    LocalFileSystem fs = FileSystem.getLocal(new Configuration());
+    try {
+      LambdaTestUtils.intercept(EOFException.class,
+          () -> serDeser.load(fs, tempPath));
+      fs.delete(tempPath, false);
+      LambdaTestUtils.intercept(FileNotFoundException.class,
+          () -> serDeser.load(fs, tempPath));
+    } finally {
+      fs.delete(tempPath, false);
+    }
+  }
+
+
+}

+ 21 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 
 /**
@@ -53,14 +54,23 @@ public class TestMapreduceConfigFields extends TestConfigurationFieldsBase {
   @SuppressWarnings("deprecation")
   @Override
   public void initializeMemberVariables() {
-    xmlFilename = new String("mapred-default.xml");
-    configurationClasses = new Class[] { MRJobConfig.class, MRConfig.class,
-        JHAdminConfig.class, ShuffleHandler.class, FileOutputFormat.class,
-	FileInputFormat.class, Job.class, NLineInputFormat.class,
-	JobConf.class, FileOutputCommitter.class };
+    xmlFilename = "mapred-default.xml";
+    configurationClasses = new Class[] {
+        MRJobConfig.class,
+        MRConfig.class,
+        JHAdminConfig.class,
+        ShuffleHandler.class,
+        FileOutputFormat.class,
+        FileInputFormat.class,
+        Job.class,
+        NLineInputFormat.class,
+        JobConf.class,
+        FileOutputCommitter.class,
+        PathOutputCommitterFactory.class
+    };
 
     // Initialize used variables
-    configurationPropsToSkipCompare = new HashSet<String>();
+    configurationPropsToSkipCompare = new HashSet<>();
 
     // Set error modes
     errorIfMissingConfigProps = true;
@@ -82,6 +92,11 @@ public class TestMapreduceConfigFields extends TestConfigurationFieldsBase {
         MRJobConfig.MAP_RESOURCE_TYPE_PREFIX);
     configurationPropsToSkipCompare.add(
         MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX);
+
+    // PathOutputCommitterFactory values
+    xmlPrefixToSkipCompare = new HashSet<>();
+    xmlPrefixToSkipCompare.add(
+        PathOutputCommitterFactory.COMMITTER_FACTORY_SCHEME);
   }
 
 }

+ 184 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/BindingPathOutputCommitter.java

@@ -0,0 +1,184 @@
+/*
+ * 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;
+
+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.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * This is a special committer which creates the factory for the committer and
+ * runs off that. Why does it exist? So that you can explicitly instantiate
+ * a committer by classname and yet still have the actual implementation
+ * driven dynamically by the factory options and destination filesystem.
+ * This simplifies integration
+ * with existing code which takes the classname of a committer.
+ * There's no factory for this, as that would lead to a loop.
+ *
+ * All commit protocol methods and accessors are delegated to the
+ * wrapped committer.
+ *
+ * How to use:
+ *
+ * <ol>
+ *   <li>
+ *     In applications which take a classname of committer in
+ *     a configuration option, set it to the canonical name of this class
+ *     (see {@link #NAME}). When this class is instantiated, it will
+ *     use the factory mechanism to locate the configured committer for the
+ *     destination.
+ *   </li>
+ *   <li>
+ *     In code, explicitly create an instance of this committer through
+ *     its constructor, then invoke commit lifecycle operations on it.
+ *     The dynamically configured committer will be created in the constructor
+ *     and have the lifecycle operations relayed to it.
+ *   </li>
+ * </ol>
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class BindingPathOutputCommitter extends PathOutputCommitter {
+
+  /**
+   * The classname for use in configurations.
+   */
+  public static final String NAME
+      = BindingPathOutputCommitter.class.getCanonicalName();
+
+  /**
+   * The bound committer.
+   */
+  private final PathOutputCommitter committer;
+
+  /**
+   * Instantiate.
+   * @param outputPath output path (may be null)
+   * @param context task context
+   * @throws IOException on any failure.
+   */
+  public BindingPathOutputCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    super(outputPath, context);
+    committer = PathOutputCommitterFactory.getCommitterFactory(outputPath,
+        context.getConfiguration())
+        .createOutputCommitter(outputPath, context);
+  }
+
+  @Override
+  public Path getOutputPath() {
+    return committer.getOutputPath();
+  }
+
+  @Override
+  public Path getWorkPath() throws IOException {
+    return committer.getWorkPath();
+  }
+
+  @Override
+  public void setupJob(JobContext jobContext) throws IOException {
+    committer.setupJob(jobContext);
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskContext) throws IOException {
+    committer.setupTask(taskContext);
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext taskContext)
+      throws IOException {
+    return committer.needsTaskCommit(taskContext);
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext taskContext) throws IOException {
+    committer.commitTask(taskContext);
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext taskContext) throws IOException {
+    committer.abortTask(taskContext);
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public void cleanupJob(JobContext jobContext) throws IOException {
+    super.cleanupJob(jobContext);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    committer.commitJob(jobContext);
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, JobStatus.State state)
+      throws IOException {
+    committer.abortJob(jobContext, state);
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public boolean isRecoverySupported() {
+    return committer.isRecoverySupported();
+  }
+
+  @Override
+  public boolean isCommitJobRepeatable(JobContext jobContext)
+      throws IOException {
+    return committer.isCommitJobRepeatable(jobContext);
+  }
+
+  @Override
+  public boolean isRecoverySupported(JobContext jobContext) throws IOException {
+    return committer.isRecoverySupported(jobContext);
+  }
+
+  @Override
+  public void recoverTask(TaskAttemptContext taskContext) throws IOException {
+    committer.recoverTask(taskContext);
+  }
+
+  @Override
+  public boolean hasOutputPath() {
+    return committer.hasOutputPath();
+  }
+
+  @Override
+  public String toString() {
+    return "BindingPathOutputCommitter{"
+        + "committer=" + committer +
+        '}';
+  }
+
+  /**
+   * Get the inner committer.
+   * @return the bonded committer.
+   */
+  public PathOutputCommitter getCommitter() {
+    return committer;
+  }
+}

+ 3 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java

@@ -155,17 +155,11 @@ public class FileOutputCommitter extends PathOutputCommitter {
    * @return the path where final output of the job should be placed.  This
    * could also be considered the committed application attempt path.
    */
-  private Path getOutputPath() {
+  @Override
+  public Path getOutputPath() {
     return this.outputPath;
   }
-  
-  /**
-   * @return true if we have an output path set, else false.
-   */
-  private boolean hasOutputPath() {
-    return this.outputPath != null;
-  }
-  
+
   /**
    * @return the path where the output of pending job attempts are
    * stored.

+ 38 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitterFactory.java

@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Creates a {@link FileOutputCommitter}, always.
+ */
+public final class FileOutputCommitterFactory
+    extends PathOutputCommitterFactory {
+
+  @Override
+  public PathOutputCommitter createOutputCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return createFileOutputCommitter(outputPath, context);
+  }
+
+}

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java

@@ -328,12 +328,14 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
     job.getConfiguration().set(BASE_OUTPUT_NAME, name);
   }
 
-  public synchronized 
-     OutputCommitter getOutputCommitter(TaskAttemptContext context
-                                        ) throws IOException {
+  public synchronized
+      OutputCommitter getOutputCommitter(TaskAttemptContext context)
+      throws IOException {
     if (committer == null) {
       Path output = getOutputPath(context);
-      committer = new FileOutputCommitter(output, context);
+      committer = PathOutputCommitterFactory.getCommitterFactory(
+          output,
+          context.getConfiguration()).createOutputCommitter(output, context);
     }
     return committer;
   }

+ 79 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/NamedCommitterFactory.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;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * A factory which creates any named committer identified
+ * in the option {@link PathOutputCommitterFactory#NAMED_COMMITTER_CLASS}.
+ */
+public final class NamedCommitterFactory extends
+    PathOutputCommitterFactory {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NamedCommitterFactory.class);
+
+  @SuppressWarnings("JavaReflectionMemberAccess")
+  @Override
+  public PathOutputCommitter createOutputCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    Class<? extends PathOutputCommitter> clazz = loadCommitterClass(context);
+    LOG.debug("Using PathOutputCommitter implementation {}", clazz);
+    try {
+      Constructor<? extends PathOutputCommitter> ctor
+          = clazz.getConstructor(Path.class, TaskAttemptContext.class);
+      return ctor.newInstance(outputPath, context);
+    } catch (NoSuchMethodException
+        | InstantiationException
+        | IllegalAccessException
+        | InvocationTargetException e) {
+      throw new IOException("Failed to create " + clazz
+          + ":" + e, e);
+    }
+  }
+
+  /**
+   * Load the class named in {@link #NAMED_COMMITTER_CLASS}.
+   * @param context job or task context
+   * @return the committer class
+   * @throws IOException if no committer was defined.
+   */
+  private Class<? extends PathOutputCommitter> loadCommitterClass(
+      JobContext context) throws IOException {
+    Preconditions.checkNotNull(context, "null context");
+    Configuration conf = context.getConfiguration();
+    String value = conf.get(NAMED_COMMITTER_CLASS, "");
+    if (value.isEmpty()) {
+      throw new IOException("No committer defined in " + NAMED_COMMITTER_CLASS);
+    }
+    return conf.getClass(NAMED_COMMITTER_CLASS,
+        FileOutputCommitter.class, PathOutputCommitter.class);
+  }
+}

+ 17 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java

@@ -75,10 +75,27 @@ public abstract class PathOutputCommitter extends OutputCommitter {
         + " {}", outputPath, context);
   }
 
+  /**
+   * Get the final directory where work will be placed once the job
+   * is committed. This may be null, in which case, there is no output
+   * path to write data to.
+   * @return the path where final output of the job should be placed.
+   */
+  public abstract Path getOutputPath();
+
+  /**
+   * Predicate: is there an output path?
+   * @return true if we have an output path set, else false.
+   */
+  public boolean hasOutputPath() {
+    return getOutputPath() != null;
+  }
+
   /**
    * Get the directory that the task should write results into.
    * Warning: there's no guarantee that this work path is on the same
    * FS as the final output, or that it's visible across machines.
+   * May be null.
    * @return the work directory
    * @throws IOException IO problem
    */

+ 204 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java

@@ -0,0 +1,204 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A factory for committers implementing the {@link PathOutputCommitter}
+ * methods, and so can be used from {@link FileOutputFormat}.
+ * The base implementation returns {@link FileOutputCommitter} instances.
+ *
+ * Algorithm:
+ * <ol>
+ *   <ul>If an explicit committer factory is named, it is used.</ul>
+ *   <ul>The output path is examined.
+ *   If is non null and there is an explicit schema for that filesystem,
+ *   its factory is instantiated.</ul>
+ *   <ul>Otherwise, an instance of {@link FileOutputCommitter} is
+ *   created.</ul>
+ * </ol>
+ *
+ * In {@link FileOutputFormat}, the created factory has its method
+ * {@link #createOutputCommitter(Path, TaskAttemptContext)} with a task
+ * attempt context and a possibly null path.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PathOutputCommitterFactory extends Configured {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PathOutputCommitterFactory.class);
+
+  /**
+   * Name of the configuration option used to configure the
+   * output committer factory to use unless there is a specific
+   * one for a schema.
+   */
+  public static final String COMMITTER_FACTORY_CLASS =
+      "mapreduce.outputcommitter.factory.class";
+
+  /**
+   * Scheme prefix for per-filesystem scheme committers.
+   */
+  public static final String COMMITTER_FACTORY_SCHEME =
+      "mapreduce.outputcommitter.factory.scheme";
+
+  /**
+   * String format pattern for per-filesystem scheme committers.
+   */
+  public static final String COMMITTER_FACTORY_SCHEME_PATTERN =
+      COMMITTER_FACTORY_SCHEME + ".%s";
+
+
+  /**
+   * The {@link FileOutputCommitter} factory.
+   */
+  public static final String FILE_COMMITTER_FACTORY  =
+      "org.apache.hadoop.mapreduce.lib.output.FileOutputCommitterFactory";
+
+  /**
+   * The {@link FileOutputCommitter} factory.
+   */
+  public static final String NAMED_COMMITTER_FACTORY  =
+      "org.apache.hadoop.mapreduce.lib.output.NamedCommitterFactory";
+
+  /**
+   * The named output committer.
+   * Creates any committer listed in
+   */
+  public static final String NAMED_COMMITTER_CLASS =
+      "mapreduce.outputcommitter.named.classname";
+
+  /**
+   * Default committer factory name: {@value}.
+   */
+  public static final String COMMITTER_FACTORY_DEFAULT =
+      FILE_COMMITTER_FACTORY;
+
+  /**
+   * Create an output committer for a task attempt.
+   * @param outputPath output path. This may be null.
+   * @param context context
+   * @return a new committer
+   * @throws IOException problems instantiating the committer
+   */
+  public PathOutputCommitter createOutputCommitter(
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return createFileOutputCommitter(outputPath, context);
+  }
+
+  /**
+   * Create an instance of the default committer, a {@link FileOutputCommitter}
+   * for a task.
+   * @param outputPath the task's output path, or or null if no output path
+   * has been defined.
+   * @param context the task attempt context
+   * @return the committer to use
+   * @throws IOException problems instantiating the committer
+   */
+  protected final PathOutputCommitter createFileOutputCommitter(
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    LOG.debug("Creating FileOutputCommitter for path {} and context {}",
+        outputPath, context);
+    return new FileOutputCommitter(outputPath, context);
+  }
+
+  /**
+   * Get the committer factory for a configuration.
+   * @param outputPath the job's output path. If null, it means that the
+   * schema is unknown and a per-schema factory cannot be determined.
+   * @param conf configuration
+   * @return an instantiated committer factory
+   */
+  public static PathOutputCommitterFactory getCommitterFactory(
+      Path outputPath,
+      Configuration conf) {
+    // determine which key to look up the overall one or a schema-specific
+    // key
+    LOG.debug("Looking for committer factory for path {}", outputPath);
+    String key = COMMITTER_FACTORY_CLASS;
+    if (StringUtils.isEmpty(conf.getTrimmed(key)) && outputPath != null) {
+      // there is no explicit factory and there's an output path
+      // Get the scheme of the destination
+      String scheme = outputPath.toUri().getScheme();
+
+      // and see if it has a key
+      String schemeKey = String.format(COMMITTER_FACTORY_SCHEME_PATTERN,
+          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);
+        key = schemeKey;
+      } else {
+        LOG.debug("No scheme-specific factory defined in {}", schemeKey);
+      }
+    }
+
+    // create the factory. Before using Configuration.getClass, check
+    // for an empty configuration value, as that raises ClassNotFoundException.
+    Class<? extends PathOutputCommitterFactory> factory;
+    String trimmedValue = conf.getTrimmed(key, "");
+    if (StringUtils.isEmpty(trimmedValue)) {
+      // empty/null value, use default
+      LOG.debug("No output committer factory defined,"
+          + " defaulting to FileOutputCommitterFactory");
+      factory = FileOutputCommitterFactory.class;
+    } else {
+      // key is set, get the class
+      factory = conf.getClass(key,
+          FileOutputCommitterFactory.class,
+          PathOutputCommitterFactory.class);
+      LOG.debug("Using OutputCommitter factory class {} from key {}",
+          factory, key);
+    }
+    return ReflectionUtils.newInstance(factory, conf);
+  }
+
+  /**
+   * Create the committer factory for a task attempt & destination, then
+   * create the committer from it.
+   * @param outputPath the task's output path, or or null if no output path
+   * has been defined.
+   * @param context the task attempt context
+   * @return the committer to use
+   * @throws IOException problems instantiating the committer
+   */
+  public static PathOutputCommitter createCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return getCommitterFactory(outputPath,
+        context.getConfiguration())
+        .createOutputCommitter(outputPath, context);
+  }
+
+}

+ 22 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -2043,4 +2043,26 @@
   <name>mapreduce.job.send-token-conf</name>
   <value></value>
 </property>
+
+<property>
+  <description>
+    The name of an output committer factory for MRv2 FileOutputFormat to use
+    for committing work. If set, overrides any per-filesystem committer
+    defined for the destination filesystem.
+  </description>
+  <name>mapreduce.outputcommitter.factory.class</name>
+  <value></value>
+</property>
+
+
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.s3a</name>
+  <value>org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory</value>
+  <description>
+    The committer factory to use when writing data to S3A filesystems.
+    If mapreduce.outputcommitter.factory.class is set, it will
+    override this property.
+  </description>
+</property>
+
 </configuration>

+ 22 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java

@@ -109,14 +109,34 @@ public class TestPathOutputCommitter extends Assert {
     public void abortTask(TaskAttemptContext taskContext) throws IOException {
 
     }
+
+    @Override
+    public Path getOutputPath() {
+      return null;
+    }
   }
 
   /**
    * Stub task context.
+   * The {@link #getConfiguration()} method returns the configuration supplied
+   * in the constructor; while {@link #setOutputCommitter(OutputCommitter)}
+   * sets the committer returned in {@link #getOutputCommitter()}.
+   * Otherwise, the methods are all no-ops.
    */
-  public class TaskContext
+  public static class TaskContext
       implements TaskInputOutputContext<String, String, String, String> {
 
+    private final Configuration configuration;
+
+    public TaskContext() {
+      this(new Configuration());
+    }
+
+    public TaskContext(Configuration conf) {
+      this.configuration = conf;
+    }
+
+
     private OutputCommitter outputCommitter;
 
     public void setOutputCommitter(OutputCommitter outputCommitter) {
@@ -180,7 +200,7 @@ public class TestPathOutputCommitter extends Assert {
 
     @Override
     public Configuration getConfiguration() {
-      return null;
+      return configuration;
     }
 
     @Override

+ 495 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitterFactory.java

@@ -0,0 +1,495 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Test the committer factory logic, looking at the override
+ * and fallback behavior.
+ */
+@SuppressWarnings("unchecked")
+public class TestPathOutputCommitterFactory extends Assert {
+
+  private static final String HTTP_COMMITTER_FACTORY = String.format(
+      COMMITTER_FACTORY_SCHEME_PATTERN, "http");
+
+  private static final Path HTTP_PATH = new Path("http://hadoop.apache.org/");
+  private static final Path HDFS_PATH = new Path("hdfs://localhost:8081/");
+
+  private TaskAttemptID taskAttemptID =
+      new TaskAttemptID("local", 0, TaskType.MAP, 1, 2);
+
+  /**
+   * Set a factory for a schema, verify it works.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testCommitterFactoryForSchema() throws Throwable {
+    createCommitterFactory(SimpleCommitterFactory.class,
+        HTTP_PATH,
+        newBondedConfiguration());
+  }
+
+  /**
+   * A schema factory only affects that filesystem.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testCommitterFactoryFallbackDefault() throws Throwable {
+    createCommitterFactory(FileOutputCommitterFactory.class,
+        HDFS_PATH,
+        newBondedConfiguration());
+  }
+
+  /**
+   * A schema factory only affects that filesystem; test through
+   * {@link PathOutputCommitterFactory#createCommitter(Path, TaskAttemptContext)}.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testCommitterFallbackDefault() throws Throwable {
+    createCommitter(FileOutputCommitter.class,
+        HDFS_PATH,
+        taskAttempt(newBondedConfiguration()));
+  }
+
+  /**
+   * Verify that you can override any schema with an explicit name.
+   */
+  @Test
+  public void testCommitterFactoryOverride() throws Throwable {
+    Configuration conf = newBondedConfiguration();
+    // set up for the schema factory
+    // and then set a global one which overrides the others.
+    conf.set(COMMITTER_FACTORY_CLASS, OtherFactory.class.getName());
+    createCommitterFactory(OtherFactory.class, HDFS_PATH, conf);
+    createCommitterFactory(OtherFactory.class, HTTP_PATH, conf);
+  }
+
+  /**
+   * Verify that if the factory class option is "", schema factory
+   * resolution still works.
+   */
+  @Test
+  public void testCommitterFactoryEmptyOption() throws Throwable {
+    Configuration conf = newBondedConfiguration();
+    // set up for the schema factory
+    // and then set a global one which overrides the others.
+    conf.set(COMMITTER_FACTORY_CLASS, "");
+    createCommitterFactory(SimpleCommitterFactory.class, HTTP_PATH, conf);
+
+    // and HDFS, with no schema, falls back to the default
+    createCommitterFactory(FileOutputCommitterFactory.class, HDFS_PATH, conf);
+  }
+
+  /**
+   * Verify that if the committer factory class is unknown, you cannot
+   * create committers.
+   */
+  @Test
+  public void testCommitterFactoryUnknown() throws Throwable {
+    Configuration conf = new Configuration();
+    // set the factory to an unknown class
+    conf.set(COMMITTER_FACTORY_CLASS, "unknown");
+    intercept(RuntimeException.class,
+        () -> getCommitterFactory(HDFS_PATH, conf));
+  }
+
+  /**
+   * Verify that if the committer output path is null, you get back
+   * a FileOutputCommitter with null output & work paths.
+   */
+  @Test
+  public void testCommitterNullOutputPath() throws Throwable {
+    // bind http to schema
+    Configuration conf = newBondedConfiguration();
+    // then ask committers for a null path
+    FileOutputCommitter committer = createCommitter(
+        FileOutputCommitterFactory.class,
+        FileOutputCommitter.class,
+        null, conf);
+    assertNull(committer.getOutputPath());
+    assertNull(committer.getWorkPath());
+  }
+
+  /**
+   * Verify that if you explicitly name a committer, that takes priority
+   * over any filesystem committer.
+   */
+  @Test
+  public void testNamedCommitterFactory() throws Throwable {
+    Configuration conf = new Configuration();
+    // set up for the schema factory
+    conf.set(COMMITTER_FACTORY_CLASS, NAMED_COMMITTER_FACTORY);
+    conf.set(NAMED_COMMITTER_CLASS, SimpleCommitter.class.getName());
+    SimpleCommitter sc = createCommitter(
+        NamedCommitterFactory.class,
+        SimpleCommitter.class, HDFS_PATH, conf);
+    assertEquals("Wrong output path from " + sc,
+        HDFS_PATH,
+        sc.getOutputPath());
+  }
+
+  /**
+   * Verify that if you explicitly name a committer and there's no
+   * path, the committer is picked up.
+   */
+  @Test
+  public void testNamedCommitterFactoryNullPath() throws Throwable {
+    Configuration conf = new Configuration();
+    // set up for the schema factory
+    conf.set(COMMITTER_FACTORY_CLASS, NAMED_COMMITTER_FACTORY);
+    conf.set(NAMED_COMMITTER_CLASS, SimpleCommitter.class.getName());
+    SimpleCommitter sc = createCommitter(
+        NamedCommitterFactory.class,
+        SimpleCommitter.class,
+        null, conf);
+    assertNull(sc.getOutputPath());
+  }
+
+  /**
+   * Verify that if you explicitly name a committer and there's no
+   * path, the committer is picked up.
+   */
+  @Test
+  public void testNamedCommitterNullPath() throws Throwable {
+    Configuration conf = new Configuration();
+    // set up for the schema factory
+    conf.set(COMMITTER_FACTORY_CLASS, NAMED_COMMITTER_FACTORY);
+    conf.set(NAMED_COMMITTER_CLASS, SimpleCommitter.class.getName());
+
+    SimpleCommitter sc = createCommitter(
+        SimpleCommitter.class,
+        null, taskAttempt(conf));
+    assertNull(sc.getOutputPath());
+  }
+
+  /**
+   * Create a factory then a committer, validating the type of both.
+   * @param <T> type of factory
+   * @param <U> type of committer
+   * @param factoryClass expected factory class
+   * @param committerClass expected committer class
+   * @param path output path (may be null)
+   * @param conf configuration
+   * @return the committer
+   * @throws IOException failure to create
+   */
+  private <T extends PathOutputCommitterFactory, U extends PathOutputCommitter>
+      U createCommitter(Class<T> factoryClass,
+      Class<U> committerClass,
+      Path path,
+      Configuration conf) throws IOException {
+    T f = createCommitterFactory(factoryClass, path, conf);
+    PathOutputCommitter committer = f.createOutputCommitter(path,
+        taskAttempt(conf));
+    assertEquals(" Wrong committer for path " + path + " from factory " + f,
+        committerClass, committer.getClass());
+    return (U) committer;
+  }
+
+  /**
+   * Create a committer from a task context, via
+   * {@link PathOutputCommitterFactory#createCommitter(Path, TaskAttemptContext)}.
+   * @param <U> type of committer
+   * @param committerClass expected committer class
+   * @param path output path (may be null)
+   * @param context task attempt context
+   * @return the committer
+   * @throws IOException failure to create
+   */
+  private <U extends PathOutputCommitter> U createCommitter(
+      Class<U> committerClass,
+      Path path,
+      TaskAttemptContext context) throws IOException {
+    PathOutputCommitter committer = PathOutputCommitterFactory
+        .createCommitter(path, context);
+    assertEquals(" Wrong committer for path " + path,
+        committerClass, committer.getClass());
+    return (U) committer;
+  }
+
+  /**
+   * Create a factory then a committer, validating its type.
+   * @param factoryClass expected factory class
+   * @param path output path (may be null)
+   * @param conf configuration
+   * @param <T> type of factory
+   * @return the factory
+   */
+  private <T extends PathOutputCommitterFactory> T createCommitterFactory(
+      Class<T> factoryClass,
+      Path path,
+      Configuration conf) {
+    PathOutputCommitterFactory factory = getCommitterFactory(path, conf);
+    assertEquals(" Wrong factory for path " + path,
+        factoryClass, factory.getClass());
+    return (T)factory;
+  }
+
+  /**
+   * Create a new task attempt context.
+   * @param conf config
+   * @return a new context
+   */
+  private TaskAttemptContext taskAttempt(Configuration conf) {
+    return new TaskAttemptContextImpl(conf, taskAttemptID);
+  }
+
+  /**
+   * Verify that if you explicitly name a committer, that takes priority
+   * over any filesystem committer.
+   */
+  @Test
+  public void testFileOutputCommitterFactory() throws Throwable {
+    Configuration conf = new Configuration();
+    // set up for the schema factory
+    conf.set(COMMITTER_FACTORY_CLASS, FILE_COMMITTER_FACTORY);
+    conf.set(NAMED_COMMITTER_CLASS, SimpleCommitter.class.getName());
+    getCommitterFactory(HDFS_PATH, conf);
+    createCommitter(
+        FileOutputCommitterFactory.class,
+        FileOutputCommitter.class, null, conf);
+  }
+
+  /**
+   * Follow the entire committer chain down and create a new committer from
+   * the output format.
+   * @throws Throwable on a failure.
+   */
+  @Test
+  public void testFileOutputFormatBinding() throws Throwable {
+    Configuration conf = newBondedConfiguration();
+    conf.set(FileOutputFormat.OUTDIR, HTTP_PATH.toUri().toString());
+    TextOutputFormat<String, String> off = new TextOutputFormat<>();
+    SimpleCommitter committer = (SimpleCommitter)
+        off.getOutputCommitter(taskAttempt(conf));
+    assertEquals("Wrong output path from "+ committer,
+        HTTP_PATH,
+        committer.getOutputPath());
+  }
+
+  /**
+   * Follow the entire committer chain down and create a new committer from
+   * the output format.
+   * @throws Throwable on a failure.
+   */
+  @Test
+  public void testFileOutputFormatBindingNoPath() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.unset(FileOutputFormat.OUTDIR);
+    // set up for the schema factory
+    conf.set(COMMITTER_FACTORY_CLASS, NAMED_COMMITTER_FACTORY);
+    conf.set(NAMED_COMMITTER_CLASS, SimpleCommitter.class.getName());
+    httpToSimpleFactory(conf);
+    TextOutputFormat<String, String> off = new TextOutputFormat<>();
+    SimpleCommitter committer = (SimpleCommitter)
+        off.getOutputCommitter(taskAttempt(conf));
+    assertNull("Output path from "+ committer,
+        committer.getOutputPath());
+  }
+
+  /**
+   * Bind the http schema CommitterFactory to {@link SimpleCommitterFactory}.
+   * @param conf config to patch
+   */
+  private Configuration httpToSimpleFactory(Configuration conf) {
+    conf.set(HTTP_COMMITTER_FACTORY, SimpleCommitterFactory.class.getName());
+    return conf;
+  }
+
+
+  /**
+   * Create a configuration with the http schema bonded to the simple factory.
+   * @return a new, patched configuration
+   */
+  private Configuration newBondedConfiguration() {
+    return httpToSimpleFactory(new Configuration());
+  }
+
+  /**
+   * Extract the (mandatory) cause of an exception.
+   * @param ex exception
+   * @param clazz expected class
+   * @return the cause, which will be of the expected type
+   * @throws AssertionError if there is a problem
+   */
+  private <E extends Throwable> E verifyCauseClass(Throwable ex,
+      Class<E> clazz) throws AssertionError {
+    Throwable cause = ex.getCause();
+    if (cause == null) {
+      throw new AssertionError("No cause", ex);
+    }
+    if (!cause.getClass().equals(clazz)) {
+      throw new AssertionError("Wrong cause class", cause);
+    }
+    return (E)cause;
+  }
+
+  @Test
+  public void testBadCommitterFactory() throws Throwable {
+    expectFactoryConstructionFailure(HTTP_COMMITTER_FACTORY);
+  }
+
+  @Test
+  public void testBoundCommitterWithSchema() throws Throwable {
+    // this verifies that a bound committer relays to the underlying committer
+    Configuration conf = newBondedConfiguration();
+    TestPathOutputCommitter.TaskContext tac
+        = new TestPathOutputCommitter.TaskContext(conf);
+    BindingPathOutputCommitter committer
+        = new BindingPathOutputCommitter(HTTP_PATH, tac);
+    intercept(IOException.class, "setupJob",
+        () -> committer.setupJob(tac));
+  }
+
+  @Test
+  public void testBoundCommitterWithDefault() throws Throwable {
+    // this verifies that a bound committer relays to the underlying committer
+    Configuration conf = newBondedConfiguration();
+    TestPathOutputCommitter.TaskContext tac
+        = new TestPathOutputCommitter.TaskContext(conf);
+    BindingPathOutputCommitter committer
+        = new BindingPathOutputCommitter(HDFS_PATH, tac);
+    assertEquals(FileOutputCommitter.class,
+        committer.getCommitter().getClass());
+  }
+
+  /**
+   * Set the specific key to a string which is not a factory class; expect
+   * a failure.
+   * @param key key to set
+   * @throws Throwable on a failure
+   */
+  @SuppressWarnings("ThrowableNotThrown")
+  protected void expectFactoryConstructionFailure(String key) throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(key, "Not a factory");
+    RuntimeException ex = intercept(RuntimeException.class,
+        () -> getCommitterFactory(HTTP_PATH, conf));
+    verifyCauseClass(
+        verifyCauseClass(ex, RuntimeException.class),
+        ClassNotFoundException.class);
+  }
+
+  /**
+   * A simple committer.
+   */
+  public static final class SimpleCommitter extends PathOutputCommitter {
+
+    private final Path outputPath;
+
+    public SimpleCommitter(Path outputPath,
+        TaskAttemptContext context) throws IOException {
+      super(outputPath, context);
+      this.outputPath = outputPath;
+    }
+
+    @Override
+    public Path getWorkPath() throws IOException {
+      return null;
+    }
+
+    /**
+     * Job setup throws an exception.
+     * @param jobContext Context of the job
+     * @throws IOException always
+     */
+    @Override
+    public void setupJob(JobContext jobContext) throws IOException {
+      throw new IOException("setupJob");
+    }
+
+    @Override
+    public void setupTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+
+    @Override
+    public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        throws IOException {
+      return false;
+    }
+
+    @Override
+    public void commitTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+
+    @Override
+    public void abortTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+
+    @Override
+    public Path getOutputPath() {
+      return outputPath;
+    }
+  }
+
+  /**
+   * The simple committer factory.
+   */
+  private static class SimpleCommitterFactory
+      extends PathOutputCommitterFactory {
+
+    @Override
+    public PathOutputCommitter createOutputCommitter(Path outputPath,
+        TaskAttemptContext context) throws IOException {
+      return new SimpleCommitter(outputPath, context);
+    }
+
+  }
+
+  /**
+   * Some other factory.
+   */
+  private static class OtherFactory extends PathOutputCommitterFactory {
+
+    /**
+     * {@inheritDoc}
+     * @param outputPath output path. This may be null.
+     * @param context context
+     * @return
+     * @throws IOException
+     */
+    @Override
+    public PathOutputCommitter createOutputCommitter(Path outputPath,
+        TaskAttemptContext context) throws IOException {
+      return new SimpleCommitter(outputPath, context);
+    }
+
+  }
+
+}

+ 40 - 6
hadoop-tools/hadoop-aws/pom.xml

@@ -129,7 +129,9 @@
                 <!-- surefire.forkNumber won't do the parameter -->
                 <!-- substitution.  Putting a prefix in front of it like -->
                 <!-- "fork-" makes it work. -->
-                <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                <!-- Important: Those leading 0s are needed to guarantee that -->
+                <!-- trailing three chars are always numeric and unique -->
+                <test.unique.fork.id>fork-000${surefire.forkNumber}</test.unique.fork.id>
                 <!-- Propagate scale parameters -->
                 <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                 <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
@@ -165,7 +167,7 @@
                     <!-- surefire.forkNumber won't do the parameter -->
                     <!-- substitution.  Putting a prefix in front of it like -->
                     <!-- "fork-" makes it work. -->
-                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <test.unique.fork.id>fork-000${surefire.forkNumber}</test.unique.fork.id>
                     <!-- Propagate scale parameters -->
                     <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
@@ -192,7 +194,6 @@
                     <include>**/ITest*.java</include>
                   </includes>
                   <excludes>
-                    <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
                     <exclude>**/ITestS3AContractRootDir.java</exclude>
                     <exclude>**/ITestS3AFileContextStatistics.java</exclude>
                     <exclude>**/ITestS3AEncryptionSSEC*.java</exclude>
@@ -225,7 +226,6 @@
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
                   <includes>
-                    <include>**/ITestJets3tNativeS3FileSystemContract.java</include>
                     <include>**/ITestS3AContractRootDir.java</include>
                     <include>**/ITestS3AFileContextStatistics.java</include>
                     <include>**/ITestS3AHuge*.java</include>
@@ -465,8 +465,8 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-      <scope>test</scope>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -474,12 +474,23 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+       <artifactId>hadoop-mapreduce-client-hs</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-examples</artifactId>
       <scope>test</scope>
       <type>jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-distcp</artifactId>
@@ -491,5 +502,28 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <!-- artifacts needed to bring up a Mini MR Yarn cluster-->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-app</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-app</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
   </dependencies>
 </project>

+ 42 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java

@@ -0,0 +1,42 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.AmazonServiceException;
+
+/**
+ * A 400 "Bad Request" exception was received.
+ * This is the general "bad parameters, headers, whatever" failure.
+ */
+public class AWSBadRequestException extends AWSServiceIOException {
+  /**
+   * HTTP status code which signals this failure mode was triggered: {@value}.
+   */
+  public static final int STATUS_CODE = 400;
+
+  /**
+   * Instantiate.
+   * @param operation operation which triggered this
+   * @param cause the underlying cause
+   */
+  public AWSBadRequestException(String operation,
+      AmazonServiceException cause) {
+    super(operation, cause);
+  }
+}

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
+import com.amazonaws.SdkBaseException;
 import com.google.common.base.Preconditions;
 
 import java.io.IOException;
@@ -31,7 +32,7 @@ public class AWSClientIOException extends IOException {
   private final String operation;
 
   public AWSClientIOException(String operation,
-      AmazonClientException cause) {
+      SdkBaseException cause) {
     super(cause);
     Preconditions.checkArgument(operation != null, "Null 'operation' argument");
     Preconditions.checkArgument(cause != null, "Null 'cause' argument");

+ 31 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java

@@ -0,0 +1,31 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.AmazonServiceException;
+
+/**
+ * Status code 443, no response from server. This is considered idempotent.
+ */
+public class AWSNoResponseException extends AWSServiceIOException {
+  public AWSNoResponseException(String operation,
+      AmazonServiceException cause) {
+    super(operation, cause);
+  }
+}

+ 38 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java

@@ -0,0 +1,38 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.AmazonServiceException;
+
+/**
+ * Request is redirected.
+ * If this gets as far as the user, it's unrecoverable
+ */
+public class AWSRedirectException extends AWSServiceIOException {
+
+  /**
+   * Instantiate.
+   * @param operation operation which triggered this
+   * @param cause the underlying cause
+   */
+  public AWSRedirectException(String operation,
+      AmazonServiceException cause) {
+    super(operation, cause);
+  }
+}

+ 42 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java

@@ -0,0 +1,42 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.AmazonServiceException;
+
+/**
+ * Exception raised when a service was throttled.
+ */
+public class AWSServiceThrottledException extends AWSServiceIOException {
+
+  /**
+   * HTTP status code which signals this failure mode was triggered: {@value}.
+   */
+  public static final int STATUS_CODE = 503;
+
+  /**
+   * Instantiate.
+   * @param operation operation which triggered this
+   * @param cause the underlying cause
+   */
+  public AWSServiceThrottledException(String operation,
+      AmazonServiceException cause) {
+    super(operation, cause);
+  }
+}

+ 37 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.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.fs.s3a;
+
+import com.amazonaws.AmazonServiceException;
+
+/**
+ * A 500 response came back from a service.
+ * This is considered <i>probably</i> retriable, That is, we assume
+ * <ol>
+ *   <li>whatever error happened in the service itself to have happened
+ *    before the infrastructure committed the operation.</li>
+ *    <li>Nothing else got through either.</li>
+ * </ol>
+ */
+public class AWSStatus500Exception extends AWSServiceIOException {
+  public AWSStatus500Exception(String operation,
+      AmazonServiceException cause) {
+    super(operation, cause);
+  }
+}

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 final class BlockingThreadPoolExecutorService
     extends SemaphoredDelegatingExecutor {
 
-  private static Logger LOG = LoggerFactory
+  private static final Logger LOG = LoggerFactory
       .getLogger(BlockingThreadPoolExecutorService.class);
 
   private static final AtomicInteger POOLNUMBER = new AtomicInteger(1);

+ 57 - 15
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -35,6 +35,11 @@ public final class Constants {
   private Constants() {
   }
 
+  /**
+   * default hadoop temp dir on local system: {@value}.
+   */
+  public static final String HADOOP_TMP_DIR = "hadoop.tmp.dir";
+
   /** The minimum multipart size which S3 supports. */
   public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024;
 
@@ -328,14 +333,6 @@ public final class Constants {
   @InterfaceAudience.Private
   public static final int MAX_MULTIPART_COUNT = 10000;
 
-  /**
-   * Classname of the S3A-specific output committer factory. This
-   * is what must be declared when attempting to use
-   */
-  @InterfaceStability.Unstable
-  public static final String S3A_OUTPUT_COMMITTER_FACTORY =
-      "org.apache.hadoop.fs.s3a.commit.S3AOutputCommitterFactory";
-
   /* Constants. */
   public static final String S3_METADATA_STORE_IMPL =
       "fs.s3a.metadatastore.impl";
@@ -410,13 +407,6 @@ public final class Constants {
       "fs.s3a.s3guard.ddb.background.sleep";
   public static final int S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT = 25;
 
-  /**
-   * V1 committer.
-   */
-  @InterfaceStability.Unstable
-  public static final String S3A_OUTPUT_COMMITTER_MRV1 =
-      "org.apache.hadoop.fs.s3a.commit.S3OutputCommitterMRv1";
-
   /**
    * The default "Null" metadata store: {@value}.
    */
@@ -463,4 +453,56 @@ public final class Constants {
   @InterfaceStability.Unstable
   public static final int DEFAULT_LIST_VERSION = 2;
 
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_THROTTLE_PROBABILITY =
+      "fs.s3a.failinject.throttle.probability";
+
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_CLIENT_FACTORY =
+      "org.apache.hadoop.fs.s3a.InconsistentS3ClientFactory";
+
+  /**
+   * Number of times to retry any repeatable S3 client request on failure,
+   * excluding throttling requests: {@value}.
+   */
+  public static final String RETRY_LIMIT = "fs.s3a.retry.limit";
+
+  /**
+   * Default retry limit: {@value}.
+   */
+  public static final int RETRY_LIMIT_DEFAULT = DEFAULT_MAX_ERROR_RETRIES;
+
+  /**
+   * Interval between retry attempts.: {@value}.
+   */
+  public static final String RETRY_INTERVAL = "fs.s3a.retry.interval";
+
+  /**
+   * Default retry interval: {@value}.
+   */
+  public static final String RETRY_INTERVAL_DEFAULT = "500ms";
+
+  /**
+   * Number of times to retry any throttled request: {@value}.
+   */
+  public static final String RETRY_THROTTLE_LIMIT =
+      "fs.s3a.retry.throttle.limit";
+
+  /**
+   * Default throttled retry limit: {@value}.
+   */
+  public static final int RETRY_THROTTLE_LIMIT_DEFAULT =
+      DEFAULT_MAX_ERROR_RETRIES;
+
+  /**
+   * Interval between retry attempts on throttled requests: {@value}.
+   */
+  public static final String RETRY_THROTTLE_INTERVAL =
+      "fs.s3a.retry.throttle.interval";
+
+  /**
+   * Default throttled retry interval: {@value}.
+   */
+  public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms";
+
 }

+ 197 - 35
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java

@@ -18,38 +18,50 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.ClientConfiguration;
+import com.amazonaws.SdkClientException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
 import com.amazonaws.services.s3.model.DeleteObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsResult;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
+import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ListObjectsV2Request;
 import com.amazonaws.services.s3.model.ListObjectsV2Result;
+import com.amazonaws.services.s3.model.MultipartUploadListing;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
 /**
  * A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects
  * inconsistency and/or errors.  Used for testing S3Guard.
@@ -87,6 +99,21 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   /** Time in milliseconds to delay visibility of newly modified object. */
   private long delayKeyMsec;
 
+  /**
+   * Probability of throttling a request.
+   */
+  private float throttleProbability;
+
+  /**
+   * Counter of failures since last reset.
+   */
+  private final AtomicLong failureCounter = new AtomicLong(0);
+
+  /**
+   * limit for failures before operations succeed; if 0 then "no limit".
+   */
+  private int failureLimit = 0;
+
   /**
    * Composite of data we need to track about recently deleted objects:
    * when it was deleted (same was with recently put objects) and the object
@@ -134,12 +161,25 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     if (delayKeySubstring.equals(MATCH_ALL_KEYS)) {
       delayKeySubstring = "";
     }
-    delayKeyProbability = conf.getFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY,
-        DEFAULT_DELAY_KEY_PROBABILITY);
+    delayKeyProbability = validProbability(
+        conf.getFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY,
+            DEFAULT_DELAY_KEY_PROBABILITY));
     delayKeyMsec = conf.getLong(FAIL_INJECT_INCONSISTENCY_MSEC,
         DEFAULT_DELAY_KEY_MSEC);
-    LOG.info("Enabled with {} msec delay, substring {}, probability {}",
-        delayKeyMsec, delayKeySubstring, delayKeyProbability);
+    setThrottleProbability(conf.getFloat(FAIL_INJECT_THROTTLE_PROBABILITY,
+        0.0f));
+    LOG.info("{}", this);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "Inconsistent S3 Client with"
+            + " %s msec delay, substring %s, delay probability %s;"
+            + " throttle probability %s"
+            + "; failure limit %d, failure count %d",
+        delayKeyMsec, delayKeySubstring, delayKeyProbability,
+        throttleProbability, failureLimit, failureCounter.get());
   }
 
   /**
@@ -174,10 +214,11 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   public DeleteObjectsResult deleteObjects(DeleteObjectsRequest
       deleteObjectsRequest)
       throws AmazonClientException, AmazonServiceException {
+    maybeFail();
     for (DeleteObjectsRequest.KeyVersion keyVersion :
         deleteObjectsRequest.getKeys()) {
-      registerDeleteObject(keyVersion.getKey(), deleteObjectsRequest
-          .getBucketName());
+      registerDeleteObject(keyVersion.getKey(),
+          deleteObjectsRequest.getBucketName());
     }
     return super.deleteObjects(deleteObjectsRequest);
   }
@@ -187,6 +228,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
       throws AmazonClientException, AmazonServiceException {
     String key = deleteObjectRequest.getKey();
     LOG.debug("key {}", key);
+    maybeFail();
     registerDeleteObject(key, deleteObjectRequest.getBucketName());
     super.deleteObject(deleteObjectRequest);
   }
@@ -196,6 +238,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   public PutObjectResult putObject(PutObjectRequest putObjectRequest)
       throws AmazonClientException, AmazonServiceException {
     LOG.debug("key {}", putObjectRequest.getKey());
+    maybeFail();
     registerPutObject(putObjectRequest);
     return super.putObject(putObjectRequest);
   }
@@ -204,6 +247,20 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   @Override
   public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
       throws AmazonClientException, AmazonServiceException {
+    maybeFail();
+    return innerlistObjects(listObjectsRequest);
+  }
+
+  /**
+   * Run the list object call without any failure probability.
+   * This stops a very aggressive failure rate from completely overloading
+   * the retry logic.
+   * @param listObjectsRequest request
+   * @return listing
+   * @throws AmazonClientException failure
+   */
+  private ObjectListing innerlistObjects(ListObjectsRequest listObjectsRequest)
+      throws AmazonClientException, AmazonServiceException {
     LOG.debug("prefix {}", listObjectsRequest.getPrefix());
     ObjectListing listing = super.listObjects(listObjectsRequest);
     listing = filterListObjects(listing);
@@ -215,6 +272,16 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   @Override
   public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request)
       throws AmazonClientException, AmazonServiceException {
+    maybeFail();
+    return innerListObjectsV2(request);
+  }
+
+  /**
+   * Non failing V2 list object request.
+   * @param request request
+   * @return result.
+   */
+  private ListObjectsV2Result innerListObjectsV2(ListObjectsV2Request request) {
     LOG.debug("prefix {}", request.getPrefix());
     ListObjectsV2Result listing = super.listObjectsV2(request);
     listing = filterListObjectsV2(listing);
@@ -222,17 +289,13 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     return listing;
   }
 
-
   private void addSummaryIfNotPresent(List<S3ObjectSummary> list,
       S3ObjectSummary item) {
     // Behavior of S3ObjectSummary
     String key = item.getKey();
-    for (S3ObjectSummary member : list) {
-      if (member.getKey().equals(key)) {
-        return;
-      }
+    if (list.stream().noneMatch((member) -> member.getKey().equals(key))) {
+      list.add(item);
     }
-    list.add(item);
   }
 
   /**
@@ -396,13 +459,9 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   }
 
   private List<String> filterPrefixes(List<String> prefixes) {
-    List<String> outputPrefixes = new ArrayList<>();
-    for (String key : prefixes) {
-      if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
-        outputPrefixes.add(key);
-      }
-    }
-    return outputPrefixes;
+    return prefixes.stream()
+        .filter(key -> !isKeyDelayed(delayedPutKeys.get(key), key))
+        .collect(Collectors.toList());
   }
 
   private boolean isKeyDelayed(Long enqueueTime, String key) {
@@ -425,14 +484,14 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
   private void registerDeleteObject(String key, String bucket) {
     if (shouldDelay(key)) {
       // Record summary so we can add it back for some time post-deletion
-      S3ObjectSummary summary = null;
-      ObjectListing list = listObjects(bucket, key);
-      for (S3ObjectSummary result : list.getObjectSummaries()) {
-        if (result.getKey().equals(key)) {
-          summary = result;
-          break;
-        }
-      }
+      ListObjectsRequest request = new ListObjectsRequest()
+              .withBucketName(bucket)
+              .withPrefix(key);
+      S3ObjectSummary summary = innerlistObjects(request).getObjectSummaries()
+          .stream()
+          .filter(result -> result.getKey().equals(key))
+          .findFirst()
+          .orElse(null);
       delayedDeletes.put(key, new Delete(System.currentTimeMillis(), summary));
     }
   }
@@ -471,7 +530,109 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     delayedPutKeys.put(key, System.currentTimeMillis());
   }
 
+  @Override
+  public CompleteMultipartUploadResult completeMultipartUpload(
+      CompleteMultipartUploadRequest completeMultipartUploadRequest)
+      throws SdkClientException, AmazonServiceException {
+    maybeFail();
+    return super.completeMultipartUpload(completeMultipartUploadRequest);
+  }
+
+  @Override
+  public UploadPartResult uploadPart(UploadPartRequest uploadPartRequest)
+      throws SdkClientException, AmazonServiceException {
+    maybeFail();
+    return super.uploadPart(uploadPartRequest);
+  }
+
+  @Override
+  public InitiateMultipartUploadResult initiateMultipartUpload(
+      InitiateMultipartUploadRequest initiateMultipartUploadRequest)
+      throws SdkClientException, AmazonServiceException {
+    maybeFail();
+    return super.initiateMultipartUpload(initiateMultipartUploadRequest);
+  }
+
+  @Override
+  public MultipartUploadListing listMultipartUploads(
+      ListMultipartUploadsRequest listMultipartUploadsRequest)
+      throws SdkClientException, AmazonServiceException {
+    maybeFail();
+    return super.listMultipartUploads(listMultipartUploadsRequest);
+  }
+
+  public float getDelayKeyProbability() {
+    return delayKeyProbability;
+  }
+
+  public long getDelayKeyMsec() {
+    return delayKeyMsec;
+  }
+
+  /**
+   * Get the probability of the request being throttled.
+   * @return a value 0 - 1.0f.
+   */
+  public float getThrottleProbability() {
+    return throttleProbability;
+  }
+
+  /**
+   * Set the probability of throttling a request.
+   * @param throttleProbability the probability of a request being throttled.
+   */
+  public void setThrottleProbability(float throttleProbability) {
+    this.throttleProbability = validProbability(throttleProbability);
+  }
+
+  /**
+   * Validate a probability option.
+   * @param p probability
+   * @return the probability, if valid
+   * @throws IllegalArgumentException if the probability is out of range.
+   */
+  private float validProbability(float p) {
+    Preconditions.checkArgument(p >= 0.0f && p <= 1.0f,
+        "Probability out of range 0 to 1 %s", p);
+    return p;
+  }
+
+  /**
+   * Conditionally fail the operation.
+   * @throws AmazonClientException if the client chooses to fail
+   * the request.
+   */
+  private void maybeFail() throws AmazonClientException {
+    // code structure here is to line up for more failures later
+    AmazonServiceException ex = null;
+    if (trueWithProbability(throttleProbability)) {
+      // throttle the request
+      ex = new AmazonServiceException("throttled"
+          + " count = " + (failureCounter.get() + 1), null);
+      ex.setStatusCode(503);
+    }
+
+    if (ex != null) {
+      long count = failureCounter.incrementAndGet();
+      if (failureLimit == 0
+          || (failureLimit > 0 && count < failureLimit)) {
+        throw ex;
+      }
+    }
+  }
+
+  /**
+   * Set the limit on failures before all operations pass through.
+   * This resets the failure count.
+   * @param limit limit; "0" means "no limit"
+   */
+  public void setFailureLimit(int limit) {
+    this.failureLimit = limit;
+    failureCounter.set(0);
+  }
+
   /** Since ObjectListing is immutable, we just override it with wrapper. */
+  @SuppressWarnings("serial")
   private static class CustomObjectListing extends ObjectListing {
 
     private final List<S3ObjectSummary> customListing;
@@ -506,6 +667,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
     }
   }
 
+  @SuppressWarnings("serial")
   private static class CustomListObjectsV2Result extends ListObjectsV2Result {
 
     private final List<S3ObjectSummary> customListing;

+ 485 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java

@@ -0,0 +1,485 @@
+/*
+ * 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.fs.s3a;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Optional;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.SdkBaseException;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.io.retry.RetryPolicy;
+
+/**
+ * Class to provide lambda expression invocation of AWS operations.
+ *
+ * The core retry logic is in
+ * {@link #retryUntranslated(String, boolean, Retried, Operation)};
+ * the other {@code retry() and retryUntranslated()} calls are wrappers.
+ *
+ * The static {@link #once(String, String, Operation)} and
+ * {@link #once(String, String, VoidOperation)} calls take an operation and
+ * return it with AWS exceptions translated to IOEs of some form.
+ *
+ * The retry logic on a failure is defined by the retry policy passed in
+ * the constructor; the standard retry policy is {@link S3ARetryPolicy},
+ * though others may be used.
+ *
+ * The constructor also takes two {@link Retried} callbacks.
+ * The {@code caughtCallback} is called whenever an exception (IOE or AWS)
+ * is caught, before the retry processing looks at it.
+ * The {@code retryCallback} is invoked after a retry is scheduled
+ * but before the sleep.
+ * These callbacks can be used for reporting and incrementing statistics.
+ *
+ * The static {@link #quietly(String, String, VoidOperation)} and
+ * {@link #quietlyEval(String, String, Operation)} calls exist to take any
+ * operation and quietly catch & log at debug. The return value of
+ * {@link #quietlyEval(String, String, Operation)} is a java 8 optional,
+ * which can then be used in java8-expressions.
+ */
+public class Invoker {
+  private static final Logger LOG = LoggerFactory.getLogger(Invoker.class);
+
+  /**
+   * Retry policy to use.
+   */
+  private final RetryPolicy retryPolicy;
+
+  /**
+   * Default retry handler.
+   */
+  private final Retried retryCallback;
+
+  /**
+   * Instantiate.
+   * @param retryPolicy retry policy for all operations.
+   * @param retryCallback standard retry policy
+   */
+  public Invoker(
+      RetryPolicy retryPolicy,
+      Retried retryCallback) {
+    this.retryPolicy = retryPolicy;
+    this.retryCallback = retryCallback;
+  }
+
+  public RetryPolicy getRetryPolicy() {
+    return retryPolicy;
+  }
+
+  public Retried getRetryCallback() {
+    return retryCallback;
+  }
+
+  /**
+   * Execute a function, translating any exception into an IOException.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param operation operation to execute
+   * @param <T> type of return value
+   * @return the result of the function call
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.OnceTranslated
+  public static <T> T once(String action, String path, Operation<T> operation)
+      throws IOException {
+    try {
+      return operation.execute();
+    } catch (AmazonClientException e) {
+      throw S3AUtils.translateException(action, path, e);
+    }
+  }
+
+  /**
+   * Execute an operation with no result.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param operation operation to execute
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.OnceTranslated
+  public static void once(String action, String path, VoidOperation operation)
+      throws IOException {
+    once(action, path,
+        () -> {
+          operation.execute();
+          return null;
+        });
+  }
+
+  /**
+   * Execute an operation and ignore all raised IOExceptions; log at INFO.
+   * @param log log to log at info.
+   * @param action action to include in log
+   * @param path optional path to include in log
+   * @param operation operation to execute
+   * @param <T> type of operation
+   */
+  public static <T> void ignoreIOExceptions(
+      Logger log,
+      String action,
+      String path,
+      Operation<T> operation) {
+    try {
+      once(action, path, operation);
+    } catch (IOException e) {
+      log.info("{}: {}", toDescription(action, path), e.toString(), e);
+    }
+  }
+
+  /**
+   * Execute an operation and ignore all raised IOExceptions; log at INFO.
+   * @param log log to log at info.
+   * @param action action to include in log
+   * @param path optional path to include in log
+   * @param operation operation to execute
+   */
+  public static void ignoreIOExceptions(
+      Logger log,
+      String action,
+      String path,
+      VoidOperation operation) {
+    ignoreIOExceptions(log, action, path,
+        () -> {
+          operation.execute();
+          return null;
+        });
+  }
+
+  /**
+   * Execute a void operation with retry processing.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param retrying callback on retries
+   * @param operation operation to execute
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.RetryTranslated
+  public void retry(String action,
+      String path,
+      boolean idempotent,
+      Retried retrying,
+      VoidOperation operation)
+      throws IOException {
+    retry(action, path, idempotent, retrying,
+        () -> {
+          operation.execute();
+          return null;
+        });
+  }
+
+  /**
+   * Execute a void operation with  the default retry callback invoked.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param operation operation to execute
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.RetryTranslated
+  public void retry(String action,
+      String path,
+      boolean idempotent,
+      VoidOperation operation)
+      throws IOException {
+    retry(action, path, idempotent, retryCallback, operation);
+  }
+
+  /**
+   * Execute a function with the default retry callback invoked.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param operation operation to execute
+   * @param <T> type of return value
+   * @return the result of the call
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.RetryTranslated
+  public <T> T retry(String action,
+      String path,
+      boolean idempotent,
+      Operation<T> operation)
+      throws IOException {
+
+    return retry(action, path, idempotent, retryCallback, operation);
+  }
+
+  /**
+   * Execute a function with retry processing.
+   * Uses {@link #once(String, String, Operation)} as the inner
+   * invocation mechanism before retry logic is performed.
+   * @param <T> type of return value
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param retrying callback on retries
+   * @param operation operation to execute
+   * @return the result of the call
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.RetryTranslated
+  public <T> T retry(
+      String action,
+      String path,
+      boolean idempotent,
+      Retried retrying,
+      Operation<T> operation)
+      throws IOException {
+    return retryUntranslated(
+        toDescription(action, path),
+        idempotent,
+        retrying,
+        () -> once(action, path, operation));
+  }
+
+  /**
+   * Execute a function with retry processing and no translation.
+   * and the default retry callback.
+   * @param text description for the catching callback
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param operation operation to execute
+   * @param <T> type of return value
+   * @return the result of the call
+   * @throws IOException any IOE raised
+   * @throws RuntimeException any Runtime exception raised
+   */
+  @Retries.RetryRaw
+  public <T> T retryUntranslated(
+      String text,
+      boolean idempotent,
+      Operation<T> operation) throws IOException {
+    return retryUntranslated(text, idempotent,
+        retryCallback, operation);
+  }
+
+  /**
+   * Execute a function with retry processing: AWS SDK Exceptions
+   * are <i>not</i> translated.
+   * This is method which the others eventually invoke.
+   * @param <T> type of return value
+   * @param text text to include in messages
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param retrying callback on retries
+   * @param operation operation to execute
+   * @return the result of the call
+   * @throws IOException any IOE raised
+   * @throws SdkBaseException any AWS exception raised
+   * @throws RuntimeException : these are never caught and retries.
+   */
+  @Retries.RetryRaw
+  public <T> T retryUntranslated(
+      String text,
+      boolean idempotent,
+      Retried retrying,
+      Operation<T> operation) throws IOException {
+
+    Preconditions.checkArgument(retrying != null, "null retrying argument");
+    int retryCount = 0;
+    Exception caught;
+    RetryPolicy.RetryAction retryAction;
+    boolean shouldRetry;
+    do {
+      try {
+        // execute the operation, returning if successful
+        return operation.execute();
+      } catch (IOException | SdkBaseException e) {
+        caught = e;
+      }
+      // you only get here if the operation didn't complete
+      // normally, hence caught != null
+
+      // translate the exception into an IOE for the retry logic
+      IOException translated;
+      if (caught instanceof IOException) {
+        translated = (IOException) caught;
+      } else {
+        translated = S3AUtils.translateException(text, "",
+            (SdkBaseException)caught);
+      }
+
+
+      int attempts = retryCount + 1;
+      try {
+        // decide action base on operation, invocation count, etc
+        retryAction = retryPolicy.shouldRetry(translated, retryCount, 0,
+            idempotent);
+        // is it a retry operation?
+        shouldRetry = retryAction.action.equals(
+            RetryPolicy.RetryAction.RETRY.action);
+        if (shouldRetry) {
+          // notify the callback
+          retrying.onFailure(text, translated, retryCount, idempotent);
+          // then sleep for the policy delay
+          Thread.sleep(retryAction.delayMillis);
+        }
+        // increment the retry count
+        retryCount++;
+      } catch (InterruptedException e) {
+        // sleep was interrupted
+        // change the exception
+        caught = new InterruptedIOException("Interrupted");
+        caught.initCause(e);
+        // no retry
+        shouldRetry = false;
+        // and re-interrupt the thread
+        Thread.currentThread().interrupt();
+      } catch (Exception e) {
+        // The retry policy raised an exception
+        // log that something happened
+        LOG.warn("{}: exception in retry processing", text, e);
+        // and fail the execution with the last execution exception.
+        shouldRetry = false;
+      }
+    } while (shouldRetry);
+
+    if (caught instanceof IOException) {
+      throw (IOException) caught;
+    } else {
+      throw (SdkBaseException) caught;
+    }
+  }
+
+
+  /**
+   * Execute an operation; any exception raised is simply caught and
+   * logged at debug.
+   * @param action action to execute
+   * @param path path (for exception construction)
+   * @param operation operation
+   */
+  public static void quietly(String action,
+      String path,
+      VoidOperation operation) {
+    try {
+      once(action, path, operation);
+    } catch (Exception e) {
+      LOG.debug("Action {} failed", action, e);
+    }
+  }
+
+  /**
+   * Execute an operation; any exception raised is caught and
+   * logged at debug.
+   * The result is only non-empty if the operation succeeded
+   * @param action action to execute
+   * @param path path (for exception construction)
+   * @param operation operation
+   */
+  public static <T> Optional<T> quietlyEval(String action,
+      String path,
+      Operation<T> operation) {
+    try {
+      return Optional.of(once(action, path, operation));
+    } catch (Exception e) {
+      LOG.debug("Action {} failed", action, e);
+      return Optional.empty();
+    }
+  }
+
+  /**
+   * Take an action and path and produce a string for logging.
+   * @param action action
+   * @param path path (may be null or empty)
+   * @return string for logs
+   */
+  private static String toDescription(String action, String path) {
+    return action +
+        (StringUtils.isNotEmpty(path) ? (" on " + path) : "");
+  }
+
+  /**
+   * Arbitrary operation throwing an IOException.
+   * @param <T> return type
+   */
+  @FunctionalInterface
+  public interface Operation<T> {
+    T execute() throws IOException;
+  }
+
+  /**
+   * Void operation which may raise an IOException.
+   */
+  @FunctionalInterface
+  public interface VoidOperation {
+    void execute() throws IOException;
+  }
+
+  /**
+   * Callback for retry and notification operations.
+   * Even if the interface is throwing up "raw" exceptions, this handler
+   * gets the translated one.
+   */
+  @FunctionalInterface
+  public interface Retried {
+    /**
+     * Retry event in progress (before any sleep).
+     * @param text text passed in to the retry() Call.
+     * @param exception the caught (and possibly translated) exception.
+     * @param retries number of retries so far
+     * @param idempotent is the request idempotent.
+     */
+    void onFailure(
+        String text,
+        IOException exception,
+        int retries,
+        boolean idempotent);
+  }
+
+  /**
+   * No op for a retrying callback.
+   */
+  public static final Retried NO_OP = new Retried() {
+    @Override
+    public void onFailure(String text,
+        IOException exception,
+        int retries,
+        boolean idempotent) {
+    }
+  };
+
+  /**
+   * Log summary at info, full stack at debug.
+   */
+  public static final Retried LOG_EVENT = new Retried() {
+    @Override
+    public void onFailure(String text,
+        IOException exception,
+        int retries,
+        boolean idempotent) {
+      LOG.debug("{}: " + exception, text);
+      if (retries == 1) {
+        // stack on first attempt, to keep noise down
+        LOG.debug("{}: " + exception, text, exception);
+      }
+    }
+  };
+}

+ 7 - 19
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java

@@ -385,8 +385,9 @@ public class Listing {
         status = statusBatchIterator.next();
         // We remove from provided list the file status listed by S3 so that
         // this does not return duplicate items.
-        LOG.debug("Removing the status from provided file status {}", status);
-        providedStatus.remove(status);
+        if (providedStatus.remove(status)) {
+          LOG.debug("Removed the status from provided file status {}", status);
+        }
       } else {
         if (providedStatusIterator.hasNext()) {
           status = providedStatusIterator.next();
@@ -540,10 +541,11 @@ public class Listing {
      * initial set of results/fail if there was a problem talking to the bucket.
      * @param listPath path of the listing
      * @param request initial request to make
-     * */
+     * @throws IOException if listObjects raises one.
+     */
     ObjectListingIterator(
         Path listPath,
-        S3ListRequest request) {
+        S3ListRequest request) throws IOException {
       this.listPath = listPath;
       this.maxKeys = owner.getMaxKeys();
       this.objects = owner.listObjects(request);
@@ -571,6 +573,7 @@ public class Listing {
      * @throws NoSuchElementException if there is no more data to list.
      */
     @Override
+    @Retries.RetryTranslated
     public S3ListResult next() throws IOException {
       if (firstListing) {
         // on the first listing, don't request more data.
@@ -814,19 +817,4 @@ public class Listing {
     }
   }
 
-  /**
-   * A Path filter which accepts all filenames.
-   */
-  static final PathFilter ACCEPT_ALL = new PathFilter() {
-    @Override
-    public boolean accept(Path file) {
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      return "ACCEPT_ALL";
-    }
-  };
-
 }

+ 92 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java

@@ -0,0 +1,92 @@
+/*
+ * 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.fs.s3a;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Declaration of retry policy for documentation only.
+ * This is purely for visibility in source and is currently package-scoped.
+ * Compare with {@link org.apache.hadoop.io.retry.AtMostOnce}
+ * and {@link org.apache.hadoop.io.retry.Idempotent}; these are real
+ * markers used by Hadoop RPC.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Retries {
+  /**
+   * No retry, exceptions are translated.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface OnceTranslated {
+    String value() default "";
+  }
+
+  /**
+   * No retry, exceptions are not translated.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface OnceRaw {
+    String value() default "";
+  }
+
+  /**
+   * No retry, expect a bit of both.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface OnceMixed {
+    String value() default "";
+  }
+
+  /**
+   * Retried, exceptions are translated.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface RetryTranslated {
+    String value() default "";
+  }
+
+  /**
+   * Retried, no translation.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface RetryRaw {
+    String value() default "";
+  }
+
+  /**
+   * Retried, mixed translation.
+   */
+  @Documented
+  @Retention(RetentionPolicy.SOURCE)
+  public @interface RetryMixed {
+    String value() default "";
+  }
+
+}

+ 172 - 135
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java

@@ -22,17 +22,16 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.Callable;
+import java.util.Locale;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEventType;
 import com.amazonaws.event.ProgressListener;
-import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
 import com.amazonaws.services.s3.model.PartETag;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
@@ -47,8 +46,9 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
+import org.apache.hadoop.fs.s3a.commit.PutTracker;
 import org.apache.hadoop.util.Progressable;
 
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
@@ -65,7 +65,8 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-class S3ABlockOutputStream extends OutputStream {
+class S3ABlockOutputStream extends OutputStream implements
+    StreamCapabilities {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(S3ABlockOutputStream.class);
@@ -86,14 +87,6 @@ class S3ABlockOutputStream extends OutputStream {
   private final ProgressListener progressListener;
   private final ListeningExecutorService executorService;
 
-  /**
-   * Retry policy for multipart commits; not all AWS SDK versions retry that.
-   */
-  private final RetryPolicy retryPolicy =
-      RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
-          5,
-          2000,
-          TimeUnit.MILLISECONDS);
   /**
    * Factory for blocks.
    */
@@ -120,7 +113,12 @@ class S3ABlockOutputStream extends OutputStream {
   /**
    * Write operation helper; encapsulation of the filesystem operations.
    */
-  private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
+  private final WriteOperationHelper writeOperationHelper;
+
+  /**
+   * Track multipart put operation.
+   */
+  private final PutTracker putTracker;
 
   /**
    * An S3A output stream which uploads partitions in a separate pool of
@@ -138,6 +136,7 @@ class S3ABlockOutputStream extends OutputStream {
    * @param blockFactory factory for creating stream destinations
    * @param statistics stats for this stream
    * @param writeOperationHelper state of the write operation.
+   * @param putTracker put tracking for commit support
    * @throws IOException on any problem
    */
   S3ABlockOutputStream(S3AFileSystem fs,
@@ -147,7 +146,8 @@ class S3ABlockOutputStream extends OutputStream {
       long blockSize,
       S3ADataBlocks.BlockFactory blockFactory,
       S3AInstrumentation.OutputStreamStatistics statistics,
-      S3AFileSystem.WriteOperationHelper writeOperationHelper)
+      WriteOperationHelper writeOperationHelper,
+      PutTracker putTracker)
       throws IOException {
     this.fs = fs;
     this.key = key;
@@ -155,6 +155,7 @@ class S3ABlockOutputStream extends OutputStream {
     this.blockSize = (int) blockSize;
     this.statistics = statistics;
     this.writeOperationHelper = writeOperationHelper;
+    this.putTracker = putTracker;
     Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
         "Block size is too small: %d", blockSize);
     this.executorService = MoreExecutors.listeningDecorator(executorService);
@@ -166,7 +167,11 @@ class S3ABlockOutputStream extends OutputStream {
     // writes a 0-byte entry.
     createBlockIfNeeded();
     LOG.debug("Initialized S3ABlockOutputStream for {}" +
-        " output to {}", writeOperationHelper, activeBlock);
+        " output to {}", key, activeBlock);
+    if (putTracker.initialize()) {
+      LOG.debug("Put tracker requests multipart upload");
+      initMultipartUpload();
+    }
   }
 
   /**
@@ -299,10 +304,7 @@ class S3ABlockOutputStream extends OutputStream {
   private synchronized void uploadCurrentBlock() throws IOException {
     Preconditions.checkState(hasActiveBlock(), "No active block");
     LOG.debug("Writing block # {}", blockCount);
-    if (multiPartUpload == null) {
-      LOG.debug("Initiating Multipart upload");
-      multiPartUpload = new MultiPartUpload();
-    }
+    initMultipartUpload();
     try {
       multiPartUpload.uploadBlockAsync(getActiveBlock());
       bytesSubmitted += getActiveBlock().dataSize();
@@ -312,6 +314,20 @@ class S3ABlockOutputStream extends OutputStream {
     }
   }
 
+  /**
+   * Init multipart upload. Assumption: this is called from
+   * a synchronized block.
+   * Note that this makes a blocking HTTPS request to the far end, so
+   * can take time and potentially fail.
+   * @throws IOException failure to initialize the upload
+   */
+  private void initMultipartUpload() throws IOException {
+    if (multiPartUpload == null) {
+      LOG.debug("Initiating Multipart upload");
+      multiPartUpload = new MultiPartUpload(key);
+    }
+  }
+
   /**
    * Close the stream.
    *
@@ -342,22 +358,35 @@ class S3ABlockOutputStream extends OutputStream {
           // This must happen even if there is no data, so that 0 byte files
           // are created.
           bytes = putObject();
+          bytesSubmitted = bytes;
         }
       } else {
-        // there has already been at least one block scheduled for upload;
-        // put up the current then wait
-        if (hasBlock && block.hasData()) {
+        // there's an MPU in progress';
+        // IF there is more data to upload, or no data has yet been uploaded,
+        // PUT the final block
+        if (hasBlock &&
+            (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
           //send last part
           uploadCurrentBlock();
         }
         // wait for the partial uploads to finish
         final List<PartETag> partETags =
             multiPartUpload.waitForAllPartUploads();
-        // then complete the operation
-        multiPartUpload.complete(partETags);
         bytes = bytesSubmitted;
+        // then complete the operation
+        if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
+            partETags,
+            bytes)) {
+          multiPartUpload.complete(partETags);
+        } else {
+          LOG.info("File {} will be visible when the job is committed", key);
+        }
+      }
+      if (!putTracker.outputImmediatelyVisible()) {
+        // track the number of bytes uploaded as commit operations.
+        statistics.commitUploaded(bytes);
       }
-      LOG.debug("Upload complete for {}", writeOperationHelper);
+      LOG.debug("Upload complete to {} by {}", key, writeOperationHelper);
     } catch (IOException ioe) {
       writeOperationHelper.writeFailed(ioe);
       throw ioe;
@@ -367,7 +396,7 @@ class S3ABlockOutputStream extends OutputStream {
       closeAll(LOG, statistics);
       clearActiveBlock();
     }
-    // All end of write operations, including deleting fake parent directories
+    // Note end of write. This does not change the state of the remote FS.
     writeOperationHelper.writeSuccessful(bytes);
   }
 
@@ -387,8 +416,9 @@ class S3ABlockOutputStream extends OutputStream {
     int size = block.dataSize();
     final S3ADataBlocks.BlockUploadData uploadData = block.startUpload();
     final PutObjectRequest putObjectRequest = uploadData.hasFile() ?
-        writeOperationHelper.newPutRequest(uploadData.getFile()) :
-        writeOperationHelper.newPutRequest(uploadData.getUploadStream(), size);
+        writeOperationHelper.createPutObjectRequest(key, uploadData.getFile())
+        : writeOperationHelper.createPutObjectRequest(key,
+            uploadData.getUploadStream(), size);
     long transferQueueTime = now();
     BlockUploadProgress callback =
         new BlockUploadProgress(
@@ -396,18 +426,13 @@ class S3ABlockOutputStream extends OutputStream {
     putObjectRequest.setGeneralProgressListener(callback);
     statistics.blockUploadQueued(size);
     ListenableFuture<PutObjectResult> putObjectResult =
-        executorService.submit(new Callable<PutObjectResult>() {
-          @Override
-          public PutObjectResult call() throws Exception {
-            PutObjectResult result;
-            try {
-              // the putObject call automatically closes the input
-              // stream afterwards.
-              result = writeOperationHelper.putObject(putObjectRequest);
-            } finally {
-              closeAll(LOG, uploadData, block);
-            }
-            return result;
+        executorService.submit(() -> {
+          try {
+            // the putObject call automatically closes the input
+            // stream afterwards.
+            return writeOperationHelper.putObject(putObjectRequest);
+          } finally {
+            closeAll(LOG, uploadData, block);
           }
         });
     clearActiveBlock();
@@ -459,20 +484,82 @@ class S3ABlockOutputStream extends OutputStream {
     return statistics;
   }
 
+  /**
+   * Return the stream capabilities.
+   * This stream always returns false when queried about hflush and hsync.
+   * If asked about {@link CommitConstants#STREAM_CAPABILITY_MAGIC_OUTPUT}
+   * it will return true iff this is an active "magic" output stream.
+   * @param capability string to query the stream support for.
+   * @return true if the capability is supported by this instance.
+   */
+  @Override
+  public boolean hasCapability(String capability) {
+    switch (capability.toLowerCase(Locale.ENGLISH)) {
+
+      // does the output stream have delayed visibility
+    case CommitConstants.STREAM_CAPABILITY_MAGIC_OUTPUT:
+      return !putTracker.outputImmediatelyVisible();
+
+      // The flush/sync options are absolutely not supported
+    case "hflush":
+    case "hsync":
+      return false;
+
+    default:
+      return false;
+    }
+  }
+
   /**
    * Multiple partition upload.
    */
   private class MultiPartUpload {
     private final String uploadId;
     private final List<ListenableFuture<PartETag>> partETagsFutures;
+    private int partsSubmitted;
+    private int partsUploaded;
+    private long bytesSubmitted;
 
-    MultiPartUpload() throws IOException {
-      this.uploadId = writeOperationHelper.initiateMultiPartUpload();
+    MultiPartUpload(String key) throws IOException {
+      this.uploadId = writeOperationHelper.initiateMultiPartUpload(key);
       this.partETagsFutures = new ArrayList<>(2);
       LOG.debug("Initiated multi-part upload for {} with " +
           "id '{}'", writeOperationHelper, uploadId);
     }
 
+    /**
+     * Get a count of parts submitted.
+     * @return the number of parts submitted; will always be >= the
+     * value of {@link #getPartsUploaded()}
+     */
+    public int getPartsSubmitted() {
+      return partsSubmitted;
+    }
+
+    /**
+     * Count of parts actually uploaded.
+     * @return the count of successfully completed part uploads.
+     */
+    public int getPartsUploaded() {
+      return partsUploaded;
+    }
+
+    /**
+     * Get the upload ID; will be null after construction completes.
+     * @return the upload ID
+     */
+    public String getUploadId() {
+      return uploadId;
+    }
+
+    /**
+     * Get the count of bytes submitted.
+     * @return the current upload size.
+     */
+    public long getBytesSubmitted() {
+      return bytesSubmitted;
+    }
+
     /**
      * Upload a block of data.
      * This will take the block
@@ -481,17 +568,22 @@ class S3ABlockOutputStream extends OutputStream {
      */
     private void uploadBlockAsync(final S3ADataBlocks.DataBlock block)
         throws IOException {
-      LOG.debug("Queueing upload of {}", block);
+      LOG.debug("Queueing upload of {} for upload {}", block, uploadId);
+      Preconditions.checkNotNull(uploadId, "Null uploadId");
+      partsSubmitted++;
       final int size = block.dataSize();
+      bytesSubmitted += size;
       final S3ADataBlocks.BlockUploadData uploadData = block.startUpload();
       final int currentPartNumber = partETagsFutures.size() + 1;
       final UploadPartRequest request =
           writeOperationHelper.newUploadPartRequest(
+              key,
               uploadId,
               currentPartNumber,
               size,
               uploadData.getUploadStream(),
-              uploadData.getFile());
+              uploadData.getFile(),
+              0L);
 
       long transferQueueTime = now();
       BlockUploadProgress callback =
@@ -500,24 +592,22 @@ class S3ABlockOutputStream extends OutputStream {
       request.setGeneralProgressListener(callback);
       statistics.blockUploadQueued(block.dataSize());
       ListenableFuture<PartETag> partETagFuture =
-          executorService.submit(new Callable<PartETag>() {
-            @Override
-            public PartETag call() throws Exception {
-              // this is the queued upload operation
-              LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
-                  uploadId);
-              // do the upload
-              PartETag partETag;
-              try {
-                partETag = fs.uploadPart(request).getPartETag();
-                LOG.debug("Completed upload of {} to part {}", block,
-                    partETag.getETag());
-                LOG.debug("Stream statistics of {}", statistics);
-              } finally {
-                // close the stream and block
-                closeAll(LOG, uploadData, block);
-              }
+          executorService.submit(() -> {
+            // this is the queued upload operation
+            // do the upload
+            try {
+              LOG.debug("Uploading part {} for id '{}'",
+                  currentPartNumber, uploadId);
+              PartETag partETag = writeOperationHelper.uploadPart(request)
+                  .getPartETag();
+              LOG.debug("Completed upload of {} to part {}",
+                  block, partETag.getETag());
+              LOG.debug("Stream statistics of {}", statistics);
+              partsUploaded++;
               return partETag;
+            } finally {
+              // close the stream and block
+              closeAll(LOG, uploadData, block);
             }
           });
       partETagsFutures.add(partETagFuture);
@@ -558,28 +648,18 @@ class S3ABlockOutputStream extends OutputStream {
      * @param partETags list of partial uploads
      * @throws IOException on any problem
      */
-    private CompleteMultipartUploadResult complete(List<PartETag> partETags)
+    private void complete(List<PartETag> partETags)
         throws IOException {
-      int retryCount = 0;
-      AmazonClientException lastException;
-      String operation =
-          String.format("Completing multi-part upload for key '%s'," +
-                  " id '%s' with %s partitions ",
-              key, uploadId, partETags.size());
-      do {
-        try {
-          LOG.debug(operation);
-          return writeOperationHelper.completeMultipartUpload(
-                  uploadId,
-                  partETags);
-        } catch (AmazonClientException e) {
-          lastException = e;
-          statistics.exceptionInMultipartComplete();
-        }
-      } while (shouldRetry(operation, lastException, retryCount++));
-      // this point is only reached if the operation failed more than
-      // the allowed retry count
-      throw translateException(operation, key, lastException);
+      AtomicInteger errorCount = new AtomicInteger(0);
+      try {
+        writeOperationHelper.completeMPUwithRetries(key,
+            uploadId,
+            partETags,
+            bytesSubmitted,
+            errorCount);
+      } finally {
+        statistics.exceptionInMultipartComplete(errorCount.get());
+      }
     }
 
     /**
@@ -590,57 +670,14 @@ class S3ABlockOutputStream extends OutputStream {
       int retryCount = 0;
       AmazonClientException lastException;
       fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
-      String operation =
-          String.format("Aborting multi-part upload for '%s', id '%s",
-              writeOperationHelper, uploadId);
-      do {
-        try {
-          LOG.debug(operation);
-          writeOperationHelper.abortMultipartUpload(uploadId);
-          return;
-        } catch (AmazonClientException e) {
-          lastException = e;
-          statistics.exceptionInMultipartAbort();
-        }
-      } while (shouldRetry(operation, lastException, retryCount++));
-      // this point is only reached if the operation failed more than
-      // the allowed retry count
-      LOG.warn("Unable to abort multipart upload, you may need to purge  " +
-          "uploaded parts", lastException);
-    }
-
-    /**
-     * Predicate to determine whether a failed operation should
-     * be attempted again.
-     * If a retry is advised, the exception is automatically logged and
-     * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented.
-     * The method then sleeps for the sleep time suggested by the sleep policy;
-     * if the sleep is interrupted then {@code Thread.interrupted()} is set
-     * to indicate the thread was interrupted; then false is returned.
-     *
-     * @param operation operation for log message
-     * @param e exception raised.
-     * @param retryCount  number of retries already attempted
-     * @return true if another attempt should be made
-     */
-    private boolean shouldRetry(String operation,
-        AmazonClientException e,
-        int retryCount) {
       try {
-        RetryPolicy.RetryAction retryAction =
-            retryPolicy.shouldRetry(e, retryCount, 0, true);
-        boolean retry = retryAction == RetryPolicy.RetryAction.RETRY;
-        if (retry) {
-          fs.incrementStatistic(IGNORED_ERRORS);
-          LOG.info("Retrying {} after exception ", operation, e);
-          Thread.sleep(retryAction.delayMillis);
-        }
-        return retry;
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-        return false;
-      } catch (Exception ignored) {
-        return false;
+        writeOperationHelper.abortMultipartUpload(key, uploadId,
+            (text, e, r, i) -> statistics.exceptionInMultipartAbort());
+      } catch (IOException e) {
+        // this point is only reached if the operation failed more than
+        // the allowed retry count
+        LOG.warn("Unable to abort multipart upload,"
+            + " you may need to purge uploaded parts", e);
       }
     }
 
@@ -718,7 +755,7 @@ class S3ABlockOutputStream extends OutputStream {
   private static class ProgressableListener implements ProgressListener {
     private final Progressable progress;
 
-    public ProgressableListener(Progressable progress) {
+    ProgressableListener(Progressable progress) {
       this.progress = progress;
     }
 

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java

@@ -401,7 +401,7 @@ final class S3ADataBlocks {
     }
 
     /**
-     * InputStream backed by the internal byte array
+     * InputStream backed by the internal byte array.
      *
      * @return
      */

File diff suppressed because it is too large
+ 379 - 128
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java


+ 36 - 20
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.S3ObjectInputStream;
 import com.amazonaws.services.s3.model.SSECustomerKey;
 import com.google.common.base.Preconditions;
@@ -39,7 +39,6 @@ import java.io.EOFException;
 import java.io.IOException;
 
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 
 /**
  * The input stream for an S3A object.
@@ -86,6 +85,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
   private String serverSideEncryptionKey;
   private final S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
+  private final Invoker invoker;
 
   /**
    * This is the actual position within the object, used by
@@ -104,14 +104,29 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    */
   private long contentRangeStart;
 
+  /**
+   * Create the stream.
+   * This does not attempt to open it; that is only done on the first
+   * actual read() operation.
+   * @param s3Attributes object attributes from a HEAD request
+   * @param contentLength length of content
+   * @param client S3 client to use
+   * @param stats statistics to update
+   * @param instrumentation instrumentation to update
+   * @param readahead readahead bytes
+   * @param inputPolicy IO policy
+   * @param invoker preconfigured invoker
+   */
   public S3AInputStream(S3ObjectAttributes s3Attributes,
       long contentLength,
       AmazonS3 client,
       FileSystem.Statistics stats,
       S3AInstrumentation instrumentation,
       long readahead,
-      S3AInputPolicy inputPolicy) {
-    Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), "No Bucket");
+      S3AInputPolicy inputPolicy,
+      Invoker invoker) {
+    Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
+        "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
     Preconditions.checkArgument(contentLength >= 0, "Negative content length");
     this.bucket = s3Attributes.getBucket();
@@ -126,6 +141,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
     this.inputPolicy = inputPolicy;
     setReadahead(readahead);
+    this.invoker = invoker;
   }
 
   /**
@@ -149,22 +165,22 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
         " streamPosition={}, nextReadPosition={}",
         uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos);
 
-    streamStatistics.streamOpened();
-    try {
-      GetObjectRequest request = new GetObjectRequest(bucket, key)
-          .withRange(targetPos, contentRangeFinish - 1);
-      if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
-          StringUtils.isNotBlank(serverSideEncryptionKey)){
-        request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));
-      }
-      wrappedStream = client.getObject(request).getObjectContent();
-      contentRangeStart = targetPos;
-      if (wrappedStream == null) {
-        throw new IOException("Null IO stream from reopen of (" + reason +  ") "
-            + uri);
-      }
-    } catch (AmazonClientException e) {
-      throw translateException("Reopen at position " + targetPos, uri, e);
+    long opencount = streamStatistics.streamOpened();
+    GetObjectRequest request = new GetObjectRequest(bucket, key)
+        .withRange(targetPos, contentRangeFinish - 1);
+    if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
+        StringUtils.isNotBlank(serverSideEncryptionKey)){
+      request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));
+    }
+    String text = String.format("Failed to %s %s at %d",
+        (opencount == 0 ? "open" : "re-open"), uri, targetPos);
+    S3Object object = invoker.retry(text, uri, true,
+        () -> client.getObject(request));
+    wrappedStream = object.getObjectContent();
+    contentRangeStart = targetPos;
+    if (wrappedStream == null) {
+      throw new IOException("Null IO stream from reopen of (" + reason +  ") "
+          + uri);
     }
 
     this.pos = targetPos;

+ 224 - 7
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -24,7 +24,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
@@ -122,8 +127,23 @@ public class S3AInstrumentation {
       STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
       STREAM_WRITE_TOTAL_TIME,
       STREAM_WRITE_TOTAL_DATA,
+      COMMITTER_COMMITS_CREATED,
+      COMMITTER_COMMITS_COMPLETED,
+      COMMITTER_JOBS_SUCCEEDED,
+      COMMITTER_JOBS_FAILED,
+      COMMITTER_TASKS_SUCCEEDED,
+      COMMITTER_TASKS_FAILED,
+      COMMITTER_BYTES_COMMITTED,
+      COMMITTER_BYTES_UPLOADED,
+      COMMITTER_COMMITS_FAILED,
+      COMMITTER_COMMITS_ABORTED,
+      COMMITTER_COMMITS_REVERTED,
+      COMMITTER_MAGIC_FILES_CREATED,
       S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
-      S3GUARD_METADATASTORE_INITIALIZATION
+      S3GUARD_METADATASTORE_INITIALIZATION,
+      S3GUARD_METADATASTORE_RETRY,
+      S3GUARD_METADATASTORE_THROTTLED,
+      STORE_IO_THROTTLED
   };
 
 
@@ -179,8 +199,11 @@ public class S3AInstrumentation {
       gauge(statistic.getSymbol(), statistic.getDescription());
     }
     //todo need a config for the quantiles interval?
+    int interval = 1;
     quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
-        "ops", "latency", 1);
+        "ops", "latency", interval);
+    quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
+        "events", "frequency (Hz)", interval);
   }
 
   /**
@@ -372,7 +395,7 @@ public class S3AInstrumentation {
   }
 
   /**
-   * Indicate that S3A deleted one or more file.s
+   * Indicate that S3A deleted one or more files.
    * @param count number of files.
    */
   public void fileDeleted(int count) {
@@ -505,6 +528,14 @@ public class S3AInstrumentation {
     return s3GuardInstrumentation;
   }
 
+  /**
+   * Create a new instance of the committer statistics.
+   * @return a new committer statistics instance
+   */
+  CommitterStatistics newCommitterStatistics() {
+    return new CommitterStatistics();
+  }
+
   /**
    * Merge in the statistics of a single input stream into
    * the filesystem-wide statistics.
@@ -584,9 +615,12 @@ public class S3AInstrumentation {
 
     /**
      * The inner stream was opened.
+     * @return the previous count
      */
-    public void streamOpened() {
+    public long streamOpened() {
+      long count = openOperations;
       openOperations++;
+      return count;
     }
 
     /**
@@ -810,10 +844,13 @@ public class S3AInstrumentation {
     }
 
     /**
-     * Note an exception in a multipart complete.
+     * Note exception in a multipart complete.
+     * @param count count of exceptions
      */
-    void exceptionInMultipartComplete() {
-      exceptionsInMultipartFinalize.incrementAndGet();
+    void exceptionInMultipartComplete(int count) {
+      if (count > 0) {
+        exceptionsInMultipartFinalize.addAndGet(count);
+      }
     }
 
     /**
@@ -831,6 +868,15 @@ public class S3AInstrumentation {
       return bytesPendingUpload.get();
     }
 
+    /**
+     * Data has been uploaded to be committed in a subsequent operation;
+     * to be called at the end of the write.
+     * @param size size in bytes
+     */
+    public void commitUploaded(long size) {
+      incrementCounter(COMMITTER_BYTES_UPLOADED, size);
+    }
+
     /**
      * Output stream has closed.
      * Trigger merge in of all statistics not updated during operation.
@@ -918,5 +964,176 @@ public class S3AInstrumentation {
     public void storeClosed() {
 
     }
+
+    /**
+     * Throttled request.
+     */
+    public void throttled() {
+      incrementCounter(S3GUARD_METADATASTORE_THROTTLED, 1);
+      addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, 1);
+    }
+
+    /**
+     * S3Guard is retrying after a (retryable) failure.
+     */
+    public void retrying() {
+      incrementCounter(S3GUARD_METADATASTORE_RETRY, 1);
+    }
+  }
+
+  /**
+   * Instrumentation exported to S3Guard Committers.
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public final class CommitterStatistics {
+
+    /** A commit has been created. */
+    public void commitCreated() {
+      incrementCounter(COMMITTER_COMMITS_CREATED, 1);
+    }
+
+    /**
+     * Data has been uploaded to be committed in a subsequent operation.
+     * @param size size in bytes
+     */
+    public void commitUploaded(long size) {
+      incrementCounter(COMMITTER_BYTES_UPLOADED, size);
+    }
+
+    /**
+     * A commit has been completed.
+     * @param size size in bytes
+     */
+    public void commitCompleted(long size) {
+      incrementCounter(COMMITTER_COMMITS_COMPLETED, 1);
+      incrementCounter(COMMITTER_BYTES_COMMITTED, size);
+    }
+
+    /** A commit has been aborted. */
+    public void commitAborted() {
+      incrementCounter(COMMITTER_COMMITS_ABORTED, 1);
+    }
+
+    public void commitReverted() {
+      incrementCounter(COMMITTER_COMMITS_REVERTED, 1);
+    }
+
+    public void commitFailed() {
+      incrementCounter(COMMITTER_COMMITS_FAILED, 1);
+    }
+
+    public void taskCompleted(boolean success) {
+      incrementCounter(
+          success ? COMMITTER_TASKS_SUCCEEDED
+              : COMMITTER_TASKS_FAILED,
+          1);
+    }
+
+    public void jobCompleted(boolean success) {
+      incrementCounter(
+          success ? COMMITTER_JOBS_SUCCEEDED
+              : COMMITTER_JOBS_FAILED,
+          1);
+    }
+  }
+
+  /**
+   * Copy all the metrics to a map of (name, long-value).
+   * @return a map of the metrics
+   */
+  public Map<String, Long> toMap() {
+    MetricsToMap metricBuilder = new MetricsToMap(null);
+    registry.snapshot(metricBuilder, true);
+    for (Map.Entry<String, MutableCounterLong> entry :
+        streamMetrics.entrySet()) {
+      metricBuilder.tuple(entry.getKey(), entry.getValue().value());
+    }
+    return metricBuilder.getMap();
+  }
+
+  /**
+   * Convert all metrics to a map.
+   */
+  private static class MetricsToMap extends MetricsRecordBuilder {
+    private final MetricsCollector parent;
+    private final Map<String, Long> map =
+        new HashMap<>(COUNTERS_TO_CREATE.length * 2);
+
+    MetricsToMap(MetricsCollector parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    public MetricsRecordBuilder tag(MetricsInfo info, String value) {
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder add(MetricsTag tag) {
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder add(AbstractMetric metric) {
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder setContext(String value) {
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addCounter(MetricsInfo info, int value) {
+      return tuple(info, value);
+    }
+
+    @Override
+    public MetricsRecordBuilder addCounter(MetricsInfo info, long value) {
+      return tuple(info, value);
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo info, int value) {
+      return tuple(info, value);
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo info, long value) {
+      return tuple(info, value);
+    }
+
+    public MetricsToMap tuple(MetricsInfo info, long value) {
+      return tuple(info.name(), value);
+    }
+
+    public MetricsToMap tuple(String name, long value) {
+      map.put(name, value);
+      return this;
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo info, float value) {
+      return tuple(info, (long) value);
+    }
+
+    @Override
+    public MetricsRecordBuilder addGauge(MetricsInfo info, double value) {
+      return tuple(info, (long) value);
+    }
+
+    @Override
+    public MetricsCollector parent() {
+      return parent;
+    }
+
+    /**
+     * Get the map.
+     * @return the map of metrics
+     */
+    public Map<String, Long> getMap() {
+      return map;
+    }
   }
 }

+ 246 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java

@@ -0,0 +1,246 @@
+/*
+ * 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.fs.s3a;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.NoRouteToHostException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.file.AccessDeniedException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputExceededException;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.net.ConnectTimeoutException;
+
+import static org.apache.hadoop.io.retry.RetryPolicies.*;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * The S3A request retry policy.
+ *
+ * This uses the retry options in the configuration file to determine retry
+ * count and delays for "normal" retries and separately, for throttling;
+ * the latter is best handled for longer with an exponential back-off.
+ *
+ * <ol>
+ * <li> Those exceptions considered unrecoverable (networking) are
+ *    failed fast.</li>
+ * <li>All non-IOEs are failed immediately. Assumed: bugs in code,
+ *    unrecoverable errors, etc</li>
+ * </ol>
+ *
+ * For non-idempotent operations, only failures due to throttling or
+ * from failures which are known to only arise prior to talking to S3
+ * are retried.
+ *
+ * The retry policy is all built around that of the normal IO exceptions,
+ * particularly those extracted from
+ * {@link S3AUtils#translateException(String, Path, AmazonClientException)}.
+ * Because the {@link #shouldRetry(Exception, int, int, boolean)} method
+ * does this translation if an {@code AmazonClientException} is processed,
+ * the policy defined for the IOEs also applies to the original exceptions.
+ *
+ * Put differently: this retry policy aims to work for handlers of the
+ * untranslated exceptions, as well as the translated ones.
+ * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html">S3 Error responses</a>
+ * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/dev/ErrorBestPractices.html">Amazon S3 Error Best Practices</a>
+ * @see <a href="http://docs.aws.amazon.com/amazondynamodb/latest/APIReference/CommonErrors.html">Dynamo DB Commmon errors</a>
+ */
+public class S3ARetryPolicy implements RetryPolicy {
+
+  private final RetryPolicy retryPolicy;
+
+  /**
+   * Instantiate.
+   * @param conf configuration to read.
+   */
+  public S3ARetryPolicy(Configuration conf) {
+    Preconditions.checkArgument(conf != null, "Null configuration");
+
+    // base policy from configuration
+    RetryPolicy fixedRetries = retryUpToMaximumCountWithFixedSleep(
+        conf.getInt(RETRY_LIMIT, RETRY_LIMIT_DEFAULT),
+        conf.getTimeDuration(RETRY_INTERVAL,
+            RETRY_INTERVAL_DEFAULT,
+            TimeUnit.MILLISECONDS),
+        TimeUnit.MILLISECONDS);
+
+    // which is wrapped by a rejection of all non-idempotent calls except
+    // for specific failures.
+    RetryPolicy retryIdempotentCalls = new FailNonIOEs(
+        new IdempotencyRetryFilter(fixedRetries));
+
+    // and a separate policy for throttle requests, which are considered
+    // repeatable, even for non-idempotent calls, as the service
+    // rejected the call entirely
+    RetryPolicy throttlePolicy = exponentialBackoffRetry(
+        conf.getInt(RETRY_THROTTLE_LIMIT, RETRY_THROTTLE_LIMIT_DEFAULT),
+        conf.getTimeDuration(RETRY_THROTTLE_INTERVAL,
+            RETRY_THROTTLE_INTERVAL_DEFAULT,
+            TimeUnit.MILLISECONDS),
+        TimeUnit.MILLISECONDS);
+
+    // no retry on network and tangible API issues
+    RetryPolicy fail = RetryPolicies.TRY_ONCE_THEN_FAIL;
+
+    // client connectivity: fixed retries without care for idempotency
+    RetryPolicy connectivityFailure = fixedRetries;
+
+    // the policy map maps the exact classname; subclasses do not
+    // inherit policies.
+    Map<Class<? extends Exception>, RetryPolicy> policyMap = new HashMap<>();
+
+    // failfast exceptions which we consider unrecoverable
+    policyMap.put(UnknownHostException.class, fail);
+    policyMap.put(NoRouteToHostException.class, fail);
+    policyMap.put(InterruptedException.class, fail);
+    // note this does not pick up subclasses (like socket timeout)
+    policyMap.put(InterruptedIOException.class, fail);
+    policyMap.put(AWSRedirectException.class, fail);
+    // interesting question: should this be retried ever?
+    policyMap.put(AccessDeniedException.class, fail);
+    policyMap.put(FileNotFoundException.class, fail);
+    policyMap.put(InvalidRequestException.class, fail);
+
+    // should really be handled by resubmitting to new location;
+    // that's beyond the scope of this retry policy
+    policyMap.put(AWSRedirectException.class, fail);
+
+    // throttled requests are can be retried, always
+    policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
+
+    // connectivity problems are retried without worrying about idempotency
+    policyMap.put(ConnectTimeoutException.class, connectivityFailure);
+
+    // this can be a sign of an HTTP connection breaking early.
+    // which can be reacted to by another attempt if the request was idempotent.
+    // But: could also be a sign of trying to read past the EOF on a GET,
+    // which isn't going to be recovered from
+    policyMap.put(EOFException.class, retryIdempotentCalls);
+
+    // policy on a 400/bad request still ambiguous. Given it
+    // comes and goes on test runs: try again
+    policyMap.put(AWSBadRequestException.class, connectivityFailure);
+
+    // Status 500 error code is also treated as a connectivity problem
+    policyMap.put(AWSStatus500Exception.class, connectivityFailure);
+
+    // server didn't respond.
+    policyMap.put(AWSNoResponseException.class, retryIdempotentCalls);
+
+    // other operations
+    policyMap.put(AWSClientIOException.class, retryIdempotentCalls);
+    policyMap.put(AWSServiceIOException.class, retryIdempotentCalls);
+    policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
+    policyMap.put(SocketTimeoutException.class, retryIdempotentCalls);
+
+    // Dynamo DB exceptions
+    // asking for more than you should get. It's a retry but should be logged
+    // trigger sleep
+    policyMap.put(ProvisionedThroughputExceededException.class, throttlePolicy);
+
+    retryPolicy = retryByException(retryIdempotentCalls, policyMap);
+  }
+
+  @Override
+  public RetryAction shouldRetry(Exception exception,
+      int retries,
+      int failovers,
+      boolean idempotent) throws Exception {
+    Exception ex = exception;
+    if (exception instanceof AmazonClientException) {
+      // uprate the amazon client exception for the purpose of exception
+      // processing.
+      ex = S3AUtils.translateException("", "",
+          (AmazonClientException) exception);
+    }
+    return retryPolicy.shouldRetry(ex, retries, failovers, idempotent);
+  }
+
+  /**
+   * Policy which fails fast any non-idempotent call; hands off
+   * all idempotent calls to the next retry policy.
+   */
+  private static final class IdempotencyRetryFilter implements RetryPolicy {
+
+    private final RetryPolicy next;
+
+    IdempotencyRetryFilter(RetryPolicy next) {
+      this.next = next;
+    }
+
+    @Override
+    public RetryAction shouldRetry(Exception e,
+        int retries,
+        int failovers,
+        boolean idempotent) throws Exception {
+      return
+          idempotent ?
+              next.shouldRetry(e, retries, failovers, true)
+              : RetryAction.FAIL;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "IdempotencyRetryFilter{");
+      sb.append("next=").append(next);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  /**
+   * All non-IOE exceptions are failed.
+   */
+  private static final class FailNonIOEs implements RetryPolicy {
+
+    private final RetryPolicy next;
+
+    private FailNonIOEs(RetryPolicy next) {
+      this.next = next;
+    }
+
+    @Override
+    public RetryAction shouldRetry(Exception e,
+        int retries,
+        int failovers,
+        boolean isIdempotentOrAtMostOnce) throws Exception {
+      return
+          e instanceof IOException ?
+              next.shouldRetry(e, retries, failovers, true)
+              : RetryAction.FAIL;
+    }
+  }
+
+}

+ 10 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Collections;
 import java.util.EnumMap;
@@ -35,8 +36,10 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AStorageStatistics extends StorageStatistics {
-  private static final Logger LOG = S3AFileSystem.LOG;
+public class S3AStorageStatistics extends StorageStatistics
+    implements Iterable<StorageStatistics.LongStatistic> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3AStorageStatistics.class);
 
   public static final String NAME = "S3AStorageStatistics";
   private final Map<Statistic, AtomicLong> opsCount =
@@ -96,6 +99,11 @@ public class S3AStorageStatistics extends StorageStatistics {
     return new LongIterator();
   }
 
+  @Override
+  public Iterator<LongStatistic> iterator() {
+    return getLongStatistics();
+  }
+
   @Override
   public Long getLong(String key) {
     final Statistic type = Statistic.fromSymbol(key);

+ 308 - 16
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import com.amazonaws.AbortedException;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
+import com.amazonaws.SdkBaseException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.auth.InstanceProfileCredentialsProvider;
+import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
+import com.amazonaws.services.dynamodbv2.model.LimitExceededException;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 
@@ -32,12 +38,18 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.ProviderUtils;
 
 import com.google.common.collect.Lists;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
@@ -46,12 +58,15 @@ import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
@@ -63,8 +78,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
 @InterfaceStability.Evolving
 public final class S3AUtils {
 
-  /** Reuse the S3AFileSystem log. */
-  private static final Logger LOG = S3AFileSystem.LOG;
+  private static final Logger LOG = LoggerFactory.getLogger(S3AUtils.class);
   static final String CONSTRUCTOR_EXCEPTION = "constructor exception";
   static final String INSTANTIATION_EXCEPTION
       = "instantiation exception";
@@ -95,6 +109,8 @@ public final class S3AUtils {
       S3AEncryptionMethods.SSE_S3.getMethod()
           + " is enabled but an encryption key was set in "
           + SERVER_SIDE_ENCRYPTION_KEY;
+  private static final String EOF_MESSAGE_IN_XML_PARSER
+      = "Failed to sanitize XML document destined for handler class";
 
 
   private S3AUtils() {
@@ -106,6 +122,10 @@ public final class S3AUtils {
    * {@link AmazonClientException} passed in, and any status codes included
    * in the operation. That is: HTTP error codes are examined and can be
    * used to build a more specific response.
+   *
+   * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html">S3 Error responses</a>
+   * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/dev/ErrorBestPractices.html">Amazon S3 Error Best Practices</a>
+   * @see <a href="http://docs.aws.amazon.com/amazondynamodb/latest/APIReference/CommonErrors.html">Dynamo DB Commmon errors</a>
    * @param operation operation
    * @param path path operated on (must not be null)
    * @param exception amazon exception raised
@@ -131,19 +151,28 @@ public final class S3AUtils {
   @SuppressWarnings("ThrowableInstanceNeverThrown")
   public static IOException translateException(String operation,
       String path,
-      AmazonClientException exception) {
+      SdkBaseException exception) {
     String message = String.format("%s%s: %s",
         operation,
         path != null ? (" on " + path) : "",
         exception);
     if (!(exception instanceof AmazonServiceException)) {
-      if (containsInterruptedException(exception)) {
-        return (IOException)new InterruptedIOException(message)
-            .initCause(exception);
+      Exception innerCause = containsInterruptedException(exception);
+      if (innerCause != null) {
+        // interrupted IO, or a socket exception underneath that class
+        return translateInterruptedException(exception, innerCause, message);
+      }
+      if (signifiesConnectionBroken(exception)) {
+        // call considered an sign of connectivity failure
+        return (EOFException)new EOFException(message).initCause(exception);
       }
       return new AWSClientIOException(message, exception);
     } else {
-
+      if (exception instanceof AmazonDynamoDBException) {
+        // special handling for dynamo DB exceptions
+        return translateDynamoDBException(message,
+            (AmazonDynamoDBException)exception);
+      }
       IOException ioe;
       AmazonServiceException ase = (AmazonServiceException) exception;
       // this exception is non-null if the service exception is an s3 one
@@ -151,9 +180,11 @@ public final class S3AUtils {
           ? (AmazonS3Exception) ase
           : null;
       int status = ase.getStatusCode();
+      message = message + ":" + ase.getErrorCode();
       switch (status) {
 
       case 301:
+      case 307:
         if (s3Exception != null) {
           if (s3Exception.getAdditionalDetails() != null &&
               s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) {
@@ -163,11 +194,16 @@ public final class S3AUtils {
                 + "the bucket.",
                 s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT);
           }
-          ioe = new AWSS3IOException(message, s3Exception);
+          ioe = new AWSRedirectException(message, s3Exception);
         } else {
-          ioe = new AWSServiceIOException(message, ase);
+          ioe = new AWSRedirectException(message, ase);
         }
         break;
+
+      case 400:
+        ioe = new AWSBadRequestException(message, ase);
+        break;
+
       // permissions
       case 401:
       case 403:
@@ -186,6 +222,25 @@ public final class S3AUtils {
       // a shorter one while it is being read.
       case 416:
         ioe = new EOFException(message);
+        ioe.initCause(ase);
+        break;
+
+      // this has surfaced as a "no response from server" message.
+      // so rare we haven't replicated it.
+      // Treating as an idempotent proxy error.
+      case 443:
+      case 444:
+        ioe = new AWSNoResponseException(message, ase);
+        break;
+
+      // throttling
+      case 503:
+        ioe = new AWSServiceThrottledException(message, ase);
+        break;
+
+      // internal error
+      case 500:
+        ioe = new AWSStatus500Exception(message, ase);
         break;
 
       default:
@@ -226,20 +281,98 @@ public final class S3AUtils {
    * Recurse down the exception loop looking for any inner details about
    * an interrupted exception.
    * @param thrown exception thrown
-   * @return true if down the execution chain the operation was an interrupt
+   * @return the actual exception if the operation was an interrupt
    */
-  static boolean containsInterruptedException(Throwable thrown) {
+  static Exception containsInterruptedException(Throwable thrown) {
     if (thrown == null) {
-      return false;
+      return null;
     }
     if (thrown instanceof InterruptedException ||
-        thrown instanceof InterruptedIOException) {
-      return true;
+        thrown instanceof InterruptedIOException ||
+        thrown instanceof AbortedException) {
+      return (Exception)thrown;
     }
     // tail recurse
     return containsInterruptedException(thrown.getCause());
   }
 
+  /**
+   * Handles translation of interrupted exception. This includes
+   * preserving the class of the fault for better retry logic
+   * @param exception outer exception
+   * @param innerCause inner cause (which is guaranteed to be some form
+   * of interrupted exception
+   * @param message message for the new exception.
+   * @return an IOE which can be rethrown
+   */
+  private static InterruptedIOException translateInterruptedException(
+      SdkBaseException exception,
+      final Exception innerCause,
+      String message) {
+    InterruptedIOException ioe;
+    if (innerCause instanceof SocketTimeoutException) {
+      ioe = new SocketTimeoutException(message);
+    } else {
+      String name = innerCause.getClass().getName();
+      if (name.endsWith(".ConnectTimeoutException")
+          || name.endsWith("$ConnectTimeoutException")) {
+        // TCP connection http timeout from the shaded or unshaded filenames
+        // com.amazonaws.thirdparty.apache.http.conn.ConnectTimeoutException
+        ioe = new ConnectTimeoutException(message);
+      } else {
+        // any other exception
+        ioe = new InterruptedIOException(message);
+      }
+    }
+    ioe.initCause(exception);
+    return ioe;
+  }
+
+  /**
+   * Is the exception an instance of a throttling exception. That
+   * is an AmazonServiceException with a 503 response, any
+   * exception from DynamoDB for limits exceeded, or an
+   * {@link AWSServiceThrottledException}.
+   * @param ex exception to examine
+   * @return true if it is considered a throttling exception
+   */
+  public static boolean isThrottleException(Exception ex) {
+    return ex instanceof AWSServiceThrottledException
+        || ex instanceof ProvisionedThroughputExceededException
+        || ex instanceof LimitExceededException
+        || (ex instanceof AmazonServiceException
+            && 503  == ((AmazonServiceException)ex).getStatusCode());
+  }
+
+  /**
+   * Cue that an AWS exception is likely to be an EOF Exception based
+   * on the message coming back from an XML/JSON parser. This is likely
+   * to be brittle, so only a hint.
+   * @param ex exception
+   * @return true if this is believed to be a sign the connection was broken.
+   */
+  public static boolean signifiesConnectionBroken(SdkBaseException ex) {
+    return ex.toString().contains(EOF_MESSAGE_IN_XML_PARSER);
+  }
+
+  /**
+   * Translate a DynamoDB exception into an IOException.
+   * @param message preformatted message for the exception
+   * @param ex exception
+   * @return an exception to throw.
+   */
+  public static IOException translateDynamoDBException(String message,
+      AmazonDynamoDBException ex) {
+    if (isThrottleException(ex)) {
+      return new AWSServiceThrottledException(message, ex);
+    }
+    if (ex instanceof ResourceNotFoundException) {
+      return (FileNotFoundException) new FileNotFoundException(message)
+          .initCause(ex);
+    }
+    return new AWSServiceIOException(message, ex);
+  }
+
   /**
    * Get low level details of an amazon exception for logging; multi-line.
    * @param e exception
@@ -587,7 +720,7 @@ public final class S3AUtils {
   }
 
   /**
-   * Get a long option >= the minimum allowed value, supporting memory
+   * Get a long option &gt;= the minimum allowed value, supporting memory
    * prefixes K,M,G,T,P.
    * @param conf configuration
    * @param key key to look up
@@ -596,7 +729,7 @@ public final class S3AUtils {
    * @return the value
    * @throws IllegalArgumentException if the value is below the minimum
    */
-  static long longBytesOption(Configuration conf,
+  public static long longBytesOption(Configuration conf,
                              String key,
                              long defVal,
                              long min) {
@@ -746,6 +879,133 @@ public final class S3AUtils {
     return dest;
   }
 
+
+  /**
+   * Delete a path quietly: failures are logged at DEBUG.
+   * @param fs filesystem
+   * @param path path
+   * @param recursive recursive?
+   */
+  public static void deleteQuietly(FileSystem fs,
+      Path path,
+      boolean recursive) {
+    try {
+      fs.delete(path, recursive);
+    } catch (IOException e) {
+      LOG.debug("Failed to delete {}", path, e);
+    }
+  }
+
+  /**
+   * Delete a path: failures are logged at WARN.
+   * @param fs filesystem
+   * @param path path
+   * @param recursive recursive?
+   */
+  public static void deleteWithWarning(FileSystem fs,
+      Path path,
+      boolean recursive) {
+    try {
+      fs.delete(path, recursive);
+    } catch (IOException e) {
+      LOG.warn("Failed to delete {}", path, e);
+    }
+  }
+
+
+  /**
+   * An interface for use in lambda-expressions working with
+   * directory tree listings.
+   */
+  @FunctionalInterface
+  public interface CallOnLocatedFileStatus {
+    void call(LocatedFileStatus status) throws IOException;
+  }
+
+  /**
+   * An interface for use in lambda-expressions working with
+   * directory tree listings.
+   */
+  @FunctionalInterface
+  public interface LocatedFileStatusMap<T> {
+    T call(LocatedFileStatus status) throws IOException;
+  }
+
+  /**
+   * Apply an operation to every {@link LocatedFileStatus} in a remote
+   * iterator.
+   * @param iterator iterator from a list
+   * @param eval closure to evaluate
+   * @throws IOException anything in the closure, or iteration logic.
+   */
+  public static void applyLocatedFiles(
+      RemoteIterator<LocatedFileStatus> iterator,
+      CallOnLocatedFileStatus eval) throws IOException {
+    while (iterator.hasNext()) {
+      eval.call(iterator.next());
+    }
+  }
+
+  /**
+   * Map an operation to every {@link LocatedFileStatus} in a remote
+   * iterator, returning a list of the results.
+   * @param iterator iterator from a list
+   * @param eval closure to evaluate
+   * @throws IOException anything in the closure, or iteration logic.
+   */
+  public static <T> List<T> mapLocatedFiles(
+      RemoteIterator<LocatedFileStatus> iterator,
+      LocatedFileStatusMap<T> eval) throws IOException {
+    final List<T> results = new ArrayList<>();
+    applyLocatedFiles(iterator,
+        (s) -> results.add(eval.call(s)));
+    return results;
+  }
+
+  /**
+   * Map an operation to every {@link LocatedFileStatus} in a remote
+   * iterator, returning a list of the all results which were not empty.
+   * @param iterator iterator from a list
+   * @param eval closure to evaluate
+   * @throws IOException anything in the closure, or iteration logic.
+   */
+  public static <T> List<T> flatmapLocatedFiles(
+      RemoteIterator<LocatedFileStatus> iterator,
+      LocatedFileStatusMap<Optional<T>> eval) throws IOException {
+    final List<T> results = new ArrayList<>();
+    applyLocatedFiles(iterator,
+        (s) -> eval.call(s).map(r -> results.add(r)));
+    return results;
+  }
+
+  /**
+   * List located files and filter them as a classic listFiles(path, filter)
+   * would do.
+   * @param fileSystem filesystem
+   * @param path path to list
+   * @param recursive recursive listing?
+   * @param filter filter for the filename
+   * @return the filtered list of entries
+   * @throws IOException IO failure.
+   */
+  public static List<LocatedFileStatus> listAndFilter(FileSystem fileSystem,
+      Path path, boolean recursive, PathFilter filter) throws IOException {
+    return flatmapLocatedFiles(fileSystem.listFiles(path, recursive),
+        status -> maybe(filter.accept(status.getPath()), status));
+  }
+
+  /**
+   * Convert a value into a non-empty Optional instance if
+   * the value of {@code include} is true.
+   * @param include flag to indicate the value is to be included.
+   * @param value value to return
+   * @param <T> type of option.
+   * @return if include is false, Optional.empty. Otherwise, the value.
+   */
+  public static <T> Optional<T> maybe(boolean include, T value) {
+    return include ? Optional.of(value) : Optional.empty();
+  }
+
   /**
    * Patch the security credential provider information in
    * {@link #CREDENTIAL_PROVIDER_PATH}
@@ -937,4 +1197,36 @@ public final class S3AUtils {
     return conf.get(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey);
   }
 
+
+  /**
+   * Path filter which ignores any file which starts with . or _.
+   */
+  public static final PathFilter HIDDEN_FILE_FILTER = new PathFilter() {
+    @Override
+    public boolean accept(Path path) {
+      String name = path.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+
+    @Override
+    public String toString() {
+      return "HIDDEN_FILE_FILTER";
+    }
+  };
+
+  /**
+   * A Path filter which accepts all filenames.
+   */
+  public static final PathFilter ACCEPT_ALL = new PathFilter() {
+    @Override
+    public boolean accept(Path file) {
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "ACCEPT_ALL";
+    }
+  };
+
 }

+ 11 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java

@@ -66,4 +66,15 @@ public class S3ListRequest {
   public ListObjectsV2Request getV2() {
     return v2Request;
   }
+
+  @Override
+  public String toString() {
+    if (isV1()) {
+      return String.format("List %s:/%s",
+          v1Request.getBucketName(), v1Request.getPrefix());
+    } else {
+      return String.format("List %s:/%s",
+          v2Request.getBucketName(), v2Request.getPrefix());
+    }
+  }
 }

+ 5 - 5
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java

@@ -30,7 +30,7 @@ class S3ObjectAttributes {
   private S3AEncryptionMethods serverSideEncryptionAlgorithm;
   private String serverSideEncryptionKey;
 
-  public S3ObjectAttributes(
+  S3ObjectAttributes(
       String bucket,
       String key,
       S3AEncryptionMethods serverSideEncryptionAlgorithm,
@@ -41,19 +41,19 @@ class S3ObjectAttributes {
     this.serverSideEncryptionKey = serverSideEncryptionKey;
   }
 
-  public String getBucket() {
+  String getBucket() {
     return bucket;
   }
 
-  public String getKey() {
+  String getKey() {
     return key;
   }
 
-  public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
+  S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
     return serverSideEncryptionAlgorithm;
   }
 
-  public String getServerSideEncryptionKey() {
+  String getServerSideEncryptionKey() {
     return serverSideEncryptionKey;
   }
 }

+ 52 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

@@ -77,6 +77,8 @@ public enum Statistic {
       "Number of continued object listings made"),
   OBJECT_METADATA_REQUESTS("object_metadata_requests",
       "Number of requests for object metadata"),
+  OBJECT_MULTIPART_UPLOAD_INITIATED("object_multipart_initiated",
+      "Object multipart upload initiated"),
   OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted",
       "Object multipart upload aborted"),
   OBJECT_PUT_REQUESTS("object_put_requests",
@@ -142,16 +144,62 @@ public enum Statistic {
   STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
       "Total queue duration of all block uploads"),
 
-  // S3Guard stats
+  // S3guard committer stats
+  COMMITTER_COMMITS_CREATED(
+      "committer_commits_created",
+      "Number of files to commit created"),
+  COMMITTER_COMMITS_COMPLETED(
+      "committer_commits_completed",
+      "Number of files committed"),
+  COMMITTER_JOBS_SUCCEEDED(
+      "committer_jobs_completed",
+      "Number of successful jobs"),
+  COMMITTER_JOBS_FAILED(
+      "committer_jobs_failed",
+      "Number of failed jobs"),
+  COMMITTER_TASKS_SUCCEEDED(
+      "committer_tasks_completed",
+      "Number of successful tasks"),
+  COMMITTER_TASKS_FAILED(
+      "committer_tasks_failed",
+      "Number of failed tasks"),
+  COMMITTER_BYTES_COMMITTED(
+      "committer_bytes_committed",
+      "Amount of data committed"),
+  COMMITTER_BYTES_UPLOADED(
+      "committer_bytes_uploaded",
+      "Number of bytes uploaded duing commit operations"),
+  COMMITTER_COMMITS_FAILED(
+      "committer_commits_failed",
+      "Number of commits failed"),
+  COMMITTER_COMMITS_ABORTED(
+      "committer_commits_aborted",
+      "Number of commits aborted"),
+  COMMITTER_COMMITS_REVERTED(
+      "committer_commits_reverted",
+      "Number of commits reverted"),
+  COMMITTER_MAGIC_FILES_CREATED(
+      "committer_magic_files_created",
+      "Number of files created under 'magic' paths"),
+
+  // S3guard stats
   S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
       "s3guard_metadatastore_put_path_request",
-      "s3guard metadata store put one metadata path request"),
+      "S3Guard metadata store put one metadata path request"),
   S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
       "s3guard_metadatastore_put_path_latency",
-      "s3guard metadata store put one metadata path lantency"),
+      "S3Guard metadata store put one metadata path latency"),
   S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
-      "s3guard metadata store initialization times");
+      "S3Guard metadata store initialization times"),
+  S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
+      "S3Guard metadata store retry events"),
+  S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",
+      "S3Guard metadata store throttled events"),
+  S3GUARD_METADATASTORE_THROTTLE_RATE(
+      "s3guard_metadatastore_throttle_rate",
+      "S3Guard metadata store throttle rate"),
 
+  STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried");
 
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);

+ 474 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java

@@ -0,0 +1,474 @@
+/*
+ * 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.fs.s3a;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
+import com.amazonaws.services.s3.transfer.model.UploadResult;
+import com.google.common.base.Preconditions;
+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.Path;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.s3a.Invoker.*;
+
+/**
+ * Helper for low-level operations against an S3 Bucket for writing data
+ * and creating and committing pending writes.
+ * <p>
+ * It hides direct access to the S3 API
+ * and is a location where the object upload process can be evolved/enhanced.
+ * <p>
+ * Features
+ * <ul>
+ *   <li>Methods to create and submit requests to S3, so avoiding
+ *   all direct interaction with the AWS APIs.</li>
+ *   <li>Some extra preflight checks of arguments, so failing fast on
+ *   errors.</li>
+ *   <li>Callbacks to let the FS know of events in the output stream
+ *   upload process.</li>
+ *   <li>Failure handling, including converting exceptions to IOEs.</li>
+ *   <li>Integration with instrumentation and S3Guard.</li>
+ * </ul>
+ *
+ * This API is for internal use only.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class WriteOperationHelper {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(WriteOperationHelper.class);
+  private final S3AFileSystem owner;
+  private final Invoker invoker;
+
+  /**
+   * Constructor.
+   * @param owner owner FS creating the helper
+   *
+   */
+  protected WriteOperationHelper(S3AFileSystem owner) {
+    this.owner = owner;
+    this.invoker = new Invoker(new S3ARetryPolicy(owner.getConf()),
+        this::operationRetried);
+  }
+
+  /**
+   * Callback from {@link Invoker} when an operation is retried.
+   * @param text text of the operation
+   * @param ex exception
+   * @param retries number of retries
+   * @param idempotent is the method idempotent
+   */
+  void operationRetried(String text, Exception ex, int retries,
+      boolean idempotent) {
+    owner.operationRetried(text, ex, retries, idempotent);
+  }
+
+  /**
+   * Execute a function with retry processing.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param idempotent does the operation have semantics
+   * which mean that it can be retried even if was already executed?
+   * @param operation operation to execute
+   * @param <T> type of return value
+   * @return the result of the call
+   * @throws IOException any IOE raised, or translated exception
+   */
+  public <T> T retry(String action,
+      String path,
+      boolean idempotent,
+      Invoker.Operation<T> operation)
+      throws IOException {
+
+    return invoker.retry(action, path, idempotent, operation);
+  }
+
+  /**
+   * Create a {@link PutObjectRequest} request against the specific key.
+   * @param destKey destination key
+   * @param inputStream source data.
+   * @param length size, if known. Use -1 for not known
+   * @return the request
+   */
+  public PutObjectRequest createPutObjectRequest(String destKey,
+      InputStream inputStream, long length) {
+    return owner.newPutObjectRequest(destKey,
+        newObjectMetadata(length),
+        inputStream);
+  }
+
+  /**
+   * Create a {@link PutObjectRequest} request to upload a file.
+   * @param dest key to PUT to.
+   * @param sourceFile source file
+   * @return the request
+   */
+  public PutObjectRequest createPutObjectRequest(String dest,
+      File sourceFile) {
+    Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE,
+        "File length is too big for a single PUT upload");
+    return owner.newPutObjectRequest(dest,
+        newObjectMetadata((int) sourceFile.length()),
+        sourceFile);
+  }
+
+  /**
+   * Callback on a successful write.
+   * @param length length of the write
+   */
+  public void writeSuccessful(long length) {
+  }
+
+  /**
+   * Callback on a write failure.
+   * @param ex Any exception raised which triggered the failure.
+   */
+  public void writeFailed(Exception ex) {
+    LOG.debug("Write to {} failed", this, ex);
+  }
+
+  /**
+   * Create a new object metadata instance.
+   * Any standard metadata headers are added here, for example:
+   * encryption.
+   * @param length size, if known. Use -1 for not known
+   * @return a new metadata instance
+   */
+  public ObjectMetadata newObjectMetadata(long length) {
+    return owner.newObjectMetadata(length);
+  }
+
+  /**
+   * Start the multipart upload process.
+   * Retry policy: retrying, translated.
+   * @return the upload result containing the ID
+   * @throws IOException IO problem
+   */
+  @Retries.RetryTranslated
+  public String initiateMultiPartUpload(String destKey) throws IOException {
+    LOG.debug("Initiating Multipart upload to {}", destKey);
+    final InitiateMultipartUploadRequest initiateMPURequest =
+        new InitiateMultipartUploadRequest(owner.getBucket(),
+            destKey,
+            newObjectMetadata(-1));
+    initiateMPURequest.setCannedACL(owner.getCannedACL());
+    owner.setOptionalMultipartUploadRequestParameters(initiateMPURequest);
+
+    return retry("initiate MultiPartUpload", destKey, true,
+        () -> owner.initiateMultipartUpload(initiateMPURequest).getUploadId());
+  }
+
+  /**
+   * Finalize a multipart PUT operation.
+   * This completes the upload, and, if that works, calls
+   * {@link S3AFileSystem#finishedWrite(String, long)} to update the filesystem.
+   * Retry policy: retrying, translated.
+   * @param destKey destination of the commit
+   * @param uploadId multipart operation Id
+   * @param partETags list of partial uploads
+   * @param length length of the upload
+   * @param retrying retrying callback
+   * @return the result of the operation.
+   * @throws IOException on problems.
+   */
+  @Retries.RetryTranslated
+  private CompleteMultipartUploadResult finalizeMultipartUpload(
+      String destKey,
+      String uploadId,
+      List<PartETag> partETags,
+      long length,
+      Retried retrying) throws IOException {
+    return invoker.retry("Completing multipart commit", destKey,
+        true,
+        retrying,
+        () -> {
+          // a copy of the list is required, so that the AWS SDK doesn't
+          // attempt to sort an unmodifiable list.
+          CompleteMultipartUploadResult result =
+              owner.getAmazonS3Client().completeMultipartUpload(
+                  new CompleteMultipartUploadRequest(owner.getBucket(),
+                      destKey,
+                      uploadId,
+                      new ArrayList<>(partETags)));
+          owner.finishedWrite(destKey, length);
+          return result;
+        }
+    );
+  }
+
+  /**
+   * This completes a multipart upload to the destination key via
+   * {@code finalizeMultipartUpload()}.
+   * Retry policy: retrying, translated.
+   * Retries increment the {@code errorCount} counter.
+   * @param destKey destination
+   * @param uploadId multipart operation Id
+   * @param partETags list of partial uploads
+   * @param length length of the upload
+   * @param errorCount a counter incremented by 1 on every error; for
+   * use in statistics
+   * @return the result of the operation.
+   * @throws IOException if problems arose which could not be retried, or
+   * the retry count was exceeded
+   */
+  @Retries.RetryTranslated
+  public CompleteMultipartUploadResult completeMPUwithRetries(
+      String destKey,
+      String uploadId,
+      List<PartETag> partETags,
+      long length,
+      AtomicInteger errorCount)
+      throws IOException {
+    checkNotNull(uploadId);
+    checkNotNull(partETags);
+    LOG.debug("Completing multipart upload {} with {} parts",
+        uploadId, partETags.size());
+    return finalizeMultipartUpload(destKey,
+        uploadId,
+        partETags,
+        length,
+        (text, e, r, i) -> errorCount.incrementAndGet());
+  }
+
+  /**
+   * Abort a multipart upload operation.
+   * @param destKey destination key of the upload
+   * @param uploadId multipart operation Id
+   * @param retrying callback invoked on every retry
+   * @throws IOException failure to abort
+   * @throws FileNotFoundException if the abort ID is unknown
+   */
+  @Retries.RetryTranslated
+  public void abortMultipartUpload(String destKey, String uploadId,
+      Retried retrying)
+      throws IOException {
+    invoker.retry("Aborting multipart upload", destKey, true,
+        retrying,
+        () -> owner.abortMultipartUpload(
+            destKey,
+            uploadId));
+  }
+
+  /**
+   * Abort a multipart commit operation.
+   * @param upload upload to abort.
+   * @throws IOException on problems.
+   */
+  @Retries.RetryTranslated
+  public void abortMultipartUpload(MultipartUpload upload)
+      throws IOException {
+    invoker.retry("Aborting multipart commit", upload.getKey(), true,
+        () -> owner.abortMultipartUpload(upload));
+  }
+
+
+  /**
+   * Abort multipart uploads under a path: limited to the first
+   * few hundred.
+   * @param prefix prefix for uploads to abort
+   * @return a count of aborts
+   * @throws IOException trouble; FileNotFoundExceptions are swallowed.
+   */
+  @Retries.RetryTranslated
+  public int abortMultipartUploadsUnderPath(String prefix)
+      throws IOException {
+    LOG.debug("Aborting multipart uploads under {}", prefix);
+    int count = 0;
+    List<MultipartUpload> multipartUploads = owner.listMultipartUploads(prefix);
+    LOG.debug("Number of outstanding uploads: {}", multipartUploads.size());
+    for (MultipartUpload upload: multipartUploads) {
+      try {
+        abortMultipartUpload(upload);
+        count++;
+      } catch (FileNotFoundException e) {
+        LOG.debug("Already aborted: {}", upload.getKey(), e);
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Abort a multipart commit operation.
+   * @param destKey destination key of ongoing operation
+   * @param uploadId multipart operation Id
+   * @throws IOException on problems.
+   * @throws FileNotFoundException if the abort ID is unknown
+   */
+  @Retries.RetryTranslated
+  public void abortMultipartCommit(String destKey, String uploadId)
+      throws IOException {
+    abortMultipartUpload(destKey, uploadId, invoker.getRetryCallback());
+  }
+
+  /**
+   * Create and initialize a part request of a multipart upload.
+   * Exactly one of: {@code uploadStream} or {@code sourceFile}
+   * must be specified.
+   * A subset of the file may be posted, by providing the starting point
+   * in {@code offset} and a length of block in {@code size} equal to
+   * or less than the remaining bytes.
+   * @param destKey destination key of ongoing operation
+   * @param uploadId ID of ongoing upload
+   * @param partNumber current part number of the upload
+   * @param size amount of data
+   * @param uploadStream source of data to upload
+   * @param sourceFile optional source file.
+   * @param offset offset in file to start reading.
+   * @return the request.
+   */
+  public UploadPartRequest newUploadPartRequest(
+      String destKey,
+      String uploadId,
+      int partNumber,
+      int size,
+      InputStream uploadStream,
+      File sourceFile,
+      Long offset) {
+    checkNotNull(uploadId);
+    // exactly one source must be set; xor verifies this
+    checkArgument((uploadStream != null) ^ (sourceFile != null),
+        "Data source");
+    checkArgument(size >= 0, "Invalid partition size %s", size);
+    checkArgument(partNumber > 0 && partNumber <= 10000,
+        "partNumber must be between 1 and 10000 inclusive, but is %s",
+        partNumber);
+
+    LOG.debug("Creating part upload request for {} #{} size {}",
+        uploadId, partNumber, size);
+    UploadPartRequest request = new UploadPartRequest()
+        .withBucketName(owner.getBucket())
+        .withKey(destKey)
+        .withUploadId(uploadId)
+        .withPartNumber(partNumber)
+        .withPartSize(size);
+    if (uploadStream != null) {
+      // there's an upload stream. Bind to it.
+      request.setInputStream(uploadStream);
+    } else {
+      checkArgument(sourceFile.exists(),
+          "Source file does not exist: %s", sourceFile);
+      checkArgument(offset >= 0, "Invalid offset %s", offset);
+      long length = sourceFile.length();
+      checkArgument(offset == 0 || offset < length,
+          "Offset %s beyond length of file %s", offset, length);
+      request.setFile(sourceFile);
+      request.setFileOffset(offset);
+    }
+    return request;
+  }
+
+  /**
+   * The toString method is intended to be used in logging/toString calls.
+   * @return a string description.
+   */
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "WriteOperationHelper {bucket=").append(owner.getBucket());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * PUT an object directly (i.e. not via the transfer manager).
+   * Byte length is calculated from the file length, or, if there is no
+   * file, from the content length of the header.
+   * @param putObjectRequest the request
+   * @return the upload initiated
+   * @throws IOException on problems
+   */
+  @Retries.RetryTranslated
+  public PutObjectResult putObject(PutObjectRequest putObjectRequest)
+      throws IOException {
+    return retry("put",
+        putObjectRequest.getKey(), true,
+        () -> owner.putObjectDirect(putObjectRequest));
+  }
+
+  /**
+   * PUT an object via the transfer manager.
+   * @param putObjectRequest the request
+   * @return the result of the operation
+   * @throws IOException on problems
+   */
+  @Retries.OnceTranslated
+  public UploadResult uploadObject(PutObjectRequest putObjectRequest)
+      throws IOException {
+    // no retry; rely on xfer manager logic
+    return retry("put",
+        putObjectRequest.getKey(), true,
+        () -> owner.executePut(putObjectRequest, null));
+  }
+
+  /**
+   * Revert a commit by deleting the file.
+   * Relies on retry code in filesystem
+   * @throws IOException on problems
+   * @param destKey destination key
+   */
+  @Retries.RetryTranslated
+  public void revertCommit(String destKey) throws IOException {
+    once("revert commit", destKey,
+        () -> {
+          Path destPath = owner.keyToQualifiedPath(destKey);
+          owner.deleteObjectAtPath(destPath,
+              destKey, true);
+          owner.maybeCreateFakeParentDirectory(destPath);
+        }
+    );
+  }
+
+  /**
+   * Upload part of a multi-partition file.
+   * @param request request
+   * @return the result of the operation.
+   * @throws IOException on problems
+   */
+  @Retries.RetryTranslated
+  public UploadPartResult uploadPart(UploadPartRequest request)
+      throws IOException {
+    return retry("upload part",
+        request.getKey(),
+        true,
+        () -> owner.uploadPart(request));
+  }
+
+}

+ 756 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java

@@ -0,0 +1,756 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+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.net.NetUtils;
+
+import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
+
+/**
+ * Abstract base class for S3A committers; allows for any commonality
+ * between different architectures.
+ *
+ * Although the committer APIs allow for a committer to be created without
+ * an output path, this is not supported in this class or its subclasses:
+ * a destination must be supplied. It is left to the committer factory
+ * to handle the creation of a committer when the destination is unknown.
+ *
+ * Requiring an output directory simplifies coding and testing.
+ */
+public abstract class AbstractS3ACommitter extends PathOutputCommitter {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractS3ACommitter.class);
+
+  /**
+   * Thread pool for task execution.
+   */
+  private ExecutorService threadPool;
+
+  /** Underlying commit operations. */
+  private final CommitOperations commitOperations;
+
+  /**
+   * Final destination of work.
+   */
+  private Path outputPath;
+
+  /**
+   * 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.
+   * <i>This may not be on the final file system</i>
+   */
+  private Path workPath;
+
+  /** Configuration of the job. */
+  private Configuration conf;
+
+  /** Filesystem of {@link #outputPath}. */
+  private FileSystem destFS;
+
+  /** 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;
+
+  /**
+   * Create a committer.
+   * This constructor binds the destination directory and configuration, but
+   * does not update the work path: That must be calculated by the
+   * implementation;
+   * It is omitted here to avoid subclass methods being called too early.
+   * @param outputPath the job's output path: MUST NOT be null.
+   * @param context the task's context
+   * @throws IOException on a failure
+   */
+  protected AbstractS3ACommitter(
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    super(outputPath, context);
+    Preconditions.checkArgument(outputPath != null, "null output path");
+    Preconditions.checkArgument(context != null, "null job context");
+    this.jobContext = context;
+    this.role = "Task committer " + context.getTaskAttemptID();
+    setConf(context.getConfiguration());
+    initOutput(outputPath);
+    LOG.debug("{} instantiated for job \"{}\" ID {} with destination {}",
+        role, jobName(context), jobIdString(context), outputPath);
+    S3AFileSystem fs = getDestS3AFS();
+    createJobMarker = context.getConfiguration().getBoolean(
+        CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
+        DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
+    commitOperations = new CommitOperations(fs);
+  }
+
+  /**
+   * Init the output filesystem and path.
+   * TESTING ONLY; allows mock FS to cheat.
+   * @param out output path
+   * @throws IOException failure to create the FS.
+   */
+  @VisibleForTesting
+  protected void initOutput(Path out) throws IOException {
+    FileSystem fs = getDestinationFS(out, getConf());
+    setDestFS(fs);
+    setOutputPath(fs.makeQualified(out));
+  }
+
+  /**
+   * Get the job/task context this committer was instantiated with.
+   * @return the context.
+   */
+  public final JobContext getJobContext() {
+    return jobContext;
+  }
+
+  /**
+   * Final path of output, in the destination FS.
+   * @return the path
+   */
+  @Override
+  public final Path getOutputPath() {
+    return outputPath;
+  }
+
+  /**
+   * Set the output path.
+   * @param outputPath new value
+   */
+  protected final void setOutputPath(Path outputPath) {
+    Preconditions.checkNotNull(outputPath, "Null output path");
+    this.outputPath = outputPath;
+  }
+
+  /**
+   * This is the critical method for {@code FileOutputFormat}; it declares
+   * the path for work.
+   * @return the working path.
+   */
+  @Override
+  public Path getWorkPath() {
+    return workPath;
+  }
+
+  /**
+   * Set the work path for this committer.
+   * @param workPath the work path to use.
+   */
+  protected void setWorkPath(Path workPath) {
+    LOG.debug("Setting work path to {}", workPath);
+    this.workPath = workPath;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  protected void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Get the destination FS, creating it on demand if needed.
+   * @return the filesystem; requires the output path to be set up
+   * @throws IOException if the FS cannot be instantiated.
+   */
+  public FileSystem getDestFS() throws IOException {
+    if (destFS == null) {
+      FileSystem fs = getDestinationFS(outputPath, getConf());
+      setDestFS(fs);
+    }
+    return destFS;
+  }
+
+  /**
+   * Get the destination as an S3A Filesystem; casting it.
+   * @return the dest S3A FS.
+   * @throws IOException if the FS cannot be instantiated.
+   */
+  public S3AFileSystem getDestS3AFS() throws IOException {
+    return (S3AFileSystem) getDestFS();
+  }
+
+  /**
+   * Set the destination FS: the FS of the final output.
+   * @param destFS destination FS.
+   */
+  protected void setDestFS(FileSystem destFS) {
+    this.destFS = destFS;
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param context the context of the job.  This is used to get the
+   * application attempt ID.
+   * @return the path to store job attempt data.
+   */
+  public Path getJobAttemptPath(JobContext context) {
+    return getJobAttemptPath(getAppAttemptId(context));
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param appAttemptId the ID of the application attempt for this job.
+   * @return the path to store job attempt data.
+   */
+  protected abstract Path getJobAttemptPath(int appAttemptId);
+
+  /**
+   * Compute the path where the output of a task attempt is stored until
+   * that task is committed. This may be the normal Task attempt path
+   * or it may be a subdirectory.
+   * The default implementation returns the value of
+   * {@link #getBaseTaskAttemptPath(TaskAttemptContext)};
+   * subclasses may return different values.
+   * @param context the context of the task attempt.
+   * @return the path where a task attempt should be stored.
+   */
+  public Path getTaskAttemptPath(TaskAttemptContext context) {
+    return getBaseTaskAttemptPath(context);
+  }
+
+  /**
+   * Compute the base path where the output of a task attempt is written.
+   * This is the path which will be deleted when a task is cleaned up and
+   * aborted.
+   *
+   * @param context the context of the task attempt.
+   * @return the path where a task attempt should be stored.
+   */
+  protected abstract Path getBaseTaskAttemptPath(TaskAttemptContext context);
+
+  /**
+   * Get a temporary directory for data. When a task is aborted/cleaned
+   * up, the contents of this directory are all deleted.
+   * @param context task context
+   * @return a path for temporary data.
+   */
+  public abstract Path getTempTaskAttemptPath(TaskAttemptContext context);
+
+  /**
+   * Get the name of this committer.
+   * @return the committer name.
+   */
+  public abstract String getName();
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AbstractS3ACommitter{");
+    sb.append("role=").append(role);
+    sb.append(", name").append(getName());
+    sb.append(", outputPath=").append(getOutputPath());
+    sb.append(", workPath=").append(workPath);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Get the destination filesystem from the output path and the configuration.
+   * @param out output path
+   * @param config job/task config
+   * @return the associated FS
+   * @throws PathCommitException output path isn't to an S3A FS instance.
+   * @throws IOException failure to instantiate the FS.
+   */
+  protected FileSystem getDestinationFS(Path out, Configuration config)
+      throws IOException {
+    return getS3AFileSystem(out, config,
+        requiresDelayedCommitOutputInFileSystem());
+  }
+
+  /**
+   * Flag to indicate whether or not the destination filesystem needs
+   * to be configured to support magic paths where the output isn't immediately
+   * visible. If the committer returns true, then committer setup will
+   * fail if the FS doesn't have the capability.
+   * Base implementation returns false.
+   * @return what the requirements of the committer are of the filesystem.
+   */
+  protected boolean requiresDelayedCommitOutputInFileSystem() {
+    return false;
+  }
+  /**
+   * Task recovery considered unsupported: Warn and fail.
+   * @param taskContext Context of the task whose output is being recovered
+   * @throws IOException always.
+   */
+  @Override
+  public void recoverTask(TaskAttemptContext taskContext) throws IOException {
+    LOG.warn("Cannot recover task {}", taskContext.getTaskAttemptID());
+    throw new PathCommitException(outputPath,
+        String.format("Unable to recover task %s",
+        taskContext.getTaskAttemptID()));
+  }
+
+  /**
+   * if the job requires a success marker on a successful job,
+   * create the file {@link CommitConstants#_SUCCESS}.
+   *
+   * While the classic committers create a 0-byte file, the S3Guard committers
+   * PUT up a the contents of a {@link SuccessData} file.
+   * @param context job context
+   * @param pending the pending commits
+   * @throws IOException IO failure
+   */
+  protected void maybeCreateSuccessMarkerFromCommits(JobContext context,
+      List<SinglePendingCommit> pending) throws IOException {
+    List<String> filenames = new ArrayList<>(pending.size());
+    for (SinglePendingCommit commit : pending) {
+      String key = commit.getDestinationKey();
+      if (!key.startsWith("/")) {
+        // fix up so that FS.makeQualified() sets up the path OK
+        key = "/" + key;
+      }
+      filenames.add(key);
+    }
+    maybeCreateSuccessMarker(context, filenames);
+  }
+
+  /**
+   * if the job requires a success marker on a successful job,
+   * create the file {@link CommitConstants#_SUCCESS}.
+   *
+   * While the classic committers create a 0-byte file, the S3Guard committers
+   * PUT up a the contents of a {@link SuccessData} file.
+   * @param context job context
+   * @param filenames list of filenames.
+   * @throws IOException IO failure
+   */
+  protected void maybeCreateSuccessMarker(JobContext context,
+      List<String> filenames)
+      throws IOException {
+    if (createJobMarker) {
+      // create a success data structure and then save it
+      SuccessData successData = new SuccessData();
+      successData.setCommitter(getName());
+      successData.setDescription(getRole());
+      successData.setHostname(NetUtils.getLocalHostname());
+      Date now = new Date();
+      successData.setTimestamp(now.getTime());
+      successData.setDate(now.toString());
+      successData.setFilenames(filenames);
+      commitOperations.createSuccessMarker(getOutputPath(), successData, true);
+    }
+  }
+
+  /**
+   * Base job setup deletes the success marker.
+   * TODO: Do we need this?
+   * @param context context
+   * @throws IOException IO failure
+   */
+/*
+
+  @Override
+  public void setupJob(JobContext context) throws IOException {
+    if (createJobMarker) {
+      try (DurationInfo d = new DurationInfo("Deleting _SUCCESS marker")) {
+        commitOperations.deleteSuccessMarker(getOutputPath());
+      }
+    }
+  }
+*/
+
+  @Override
+  public void setupTask(TaskAttemptContext context) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s",
+        context.getTaskAttemptID())) {
+      Path taskAttemptPath = getTaskAttemptPath(context);
+      FileSystem fs = getTaskAttemptFilesystem(context);
+      fs.mkdirs(taskAttemptPath);
+    }
+  }
+
+  /**
+   * Get the task attempt path filesystem. This may not be the same as the
+   * final destination FS, and so may not be an S3A FS.
+   * @param context task attempt
+   * @return the filesystem
+   * @throws IOException failure to instantiate
+   */
+  protected FileSystem getTaskAttemptFilesystem(TaskAttemptContext context)
+      throws IOException {
+    return getTaskAttemptPath(context).getFileSystem(getConf());
+  }
+
+  /**
+   * Commit a list of pending uploads.
+   * @param context job context
+   * @param pending list of pending uploads
+   * @throws IOException on any failure
+   */
+  protected void commitPendingUploads(JobContext context,
+      List<SinglePendingCommit> pending) throws IOException {
+    if (pending.isEmpty()) {
+      LOG.warn("{}: No pending uploads to commit", getRole());
+    }
+    LOG.debug("{}: committing the output of {} task(s)",
+        getRole(), pending.size());
+    Tasks.foreach(pending)
+        .stopOnFailure()
+        .executeWith(buildThreadPool(context))
+        .onFailure((commit, exception) ->
+                getCommitOperations().abortSingleCommit(commit))
+        .abortWith(commit -> getCommitOperations().abortSingleCommit(commit))
+        .revertWith(commit -> getCommitOperations().revertCommit(commit))
+        .run(commit -> getCommitOperations().commitOrFail(commit));
+  }
+
+  /**
+   * Try to read every pendingset file and build a list of them/
+   * In the case of a failure to read the file, exceptions are held until all
+   * reads have been attempted.
+   * @param context job context
+   * @param suppressExceptions whether to suppress exceptions.
+   * @param fs job attempt fs
+   * @param pendingCommitFiles list of files found in the listing scan
+   * @return the list of commits
+   * @throws IOException on a failure when suppressExceptions is false.
+   */
+  protected List<SinglePendingCommit> loadPendingsetFiles(
+      JobContext context,
+      boolean suppressExceptions,
+      FileSystem fs,
+      Iterable<? extends FileStatus> pendingCommitFiles) throws IOException {
+
+    final List<SinglePendingCommit> pending = Collections.synchronizedList(
+        Lists.newArrayList());
+    Tasks.foreach(pendingCommitFiles)
+        .suppressExceptions(suppressExceptions)
+        .executeWith(buildThreadPool(context))
+        .run(pendingCommitFile ->
+          pending.addAll(
+              PendingSet.load(fs, pendingCommitFile.getPath()).getCommits())
+      );
+    return pending;
+  }
+
+  /**
+   * Internal Job commit operation: where the S3 requests are made
+   * (potentially in parallel).
+   * @param context job context
+   * @param pending pending request
+   * @throws IOException any failure
+   */
+  protected void commitJobInternal(JobContext context,
+      List<SinglePendingCommit> pending)
+      throws IOException {
+
+    commitPendingUploads(context, pending);
+  }
+
+  @Override
+  public void abortJob(JobContext context, JobStatus.State state)
+      throws IOException {
+    LOG.info("{}: aborting job {} in state {}",
+        getRole(), jobIdString(context), state);
+    // final cleanup operations
+    abortJobInternal(context, false);
+  }
+
+
+  /**
+   * The internal job abort operation; can be overridden in tests.
+   * This must clean up operations; it is called when a commit fails, as
+   * well as in an {@link #abortJob(JobContext, JobStatus.State)} call.
+   * The base implementation calls {@link #cleanup(JobContext, boolean)}
+   * @param context job context
+   * @param suppressExceptions should exceptions be suppressed?
+   * @throws IOException any IO problem raised when suppressExceptions is false.
+   */
+  protected void abortJobInternal(JobContext context,
+      boolean suppressExceptions)
+      throws IOException {
+    cleanup(context, suppressExceptions);
+  }
+
+  /**
+   * Abort all pending uploads to the destination directory during
+   * job cleanup operations.
+   * @param suppressExceptions should exceptions be suppressed
+   */
+  protected void abortPendingUploadsInCleanup(
+      boolean suppressExceptions) throws IOException {
+    Path dest = getOutputPath();
+    try (DurationInfo d =
+             new DurationInfo(LOG, "Aborting all pending commits under %s",
+                 dest)) {
+      CommitOperations ops = getCommitOperations();
+      List<MultipartUpload> pending = ops
+          .listPendingUploadsUnderPath(dest);
+      Tasks.foreach(pending)
+          .executeWith(buildThreadPool(getJobContext()))
+          .suppressExceptions(suppressExceptions)
+          .run(u -> ops.abortMultipartCommit(u.getKey(), u.getUploadId()));
+    }
+  }
+
+  /**
+   * Subclass-specific pre commit actions.
+   * @param context job context
+   * @param pending the pending operations
+   * @throws IOException any failure
+   */
+  protected void preCommitJob(JobContext context,
+      List<SinglePendingCommit> pending) throws IOException {
+  }
+
+  /**
+   * Commit work.
+   * This consists of two stages: precommit and commit.
+   * <p>
+   * Precommit: identify pending uploads, then allow subclasses
+   * to validate the state of the destination and the pending uploads.
+   * Any failure here triggers an abort of all pending uploads.
+   * <p>
+   * Commit internal: do the final commit sequence.
+   * <p>
+   * The final commit action is to build the {@code __SUCCESS} file entry.
+   * </p>
+   * @param context job context
+   * @throws IOException any failure
+   */
+  @Override
+  public void commitJob(JobContext context) throws IOException {
+    String id = jobIdString(context);
+    try (DurationInfo d = new DurationInfo(LOG,
+        "%s: commitJob(%s)", getRole(), id)) {
+      List<SinglePendingCommit> pending
+          = listPendingUploadsToCommit(context);
+      preCommitJob(context, pending);
+      commitJobInternal(context, pending);
+      jobCompleted(true);
+      maybeCreateSuccessMarkerFromCommits(context, pending);
+      cleanup(context, false);
+    } catch (IOException e) {
+      LOG.warn("Commit failure for job {}", id, e);
+      jobCompleted(false);
+      abortJobInternal(context, true);
+      throw e;
+    }
+  }
+
+  /**
+   * Job completion outcome; this may be subclassed in tests.
+   * @param success did the job succeed.
+   */
+  protected void jobCompleted(boolean success) {
+    getCommitOperations().jobCompleted(success);
+  }
+
+  /**
+   * Clean up any staging directories.
+   * IOEs must be caught and swallowed.
+   */
+  public abstract void cleanupStagingDirs();
+
+  /**
+   * Get the list of pending uploads for this job attempt.
+   * @param context job context
+   * @return a list of pending uploads.
+   * @throws IOException Any IO failure
+   */
+  protected abstract List<SinglePendingCommit> listPendingUploadsToCommit(
+      JobContext context)
+      throws IOException;
+
+  /**
+   * Cleanup the job context, including aborting anything pending.
+   * @param context job context
+   * @param suppressExceptions should exceptions be suppressed?
+   * @throws IOException any failure if exceptions were not suppressed.
+   */
+  protected void cleanup(JobContext context,
+      boolean suppressExceptions) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Cleanup job %s", jobIdString(context))) {
+      abortPendingUploadsInCleanup(suppressExceptions);
+    } finally {
+      cleanupStagingDirs();
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public void cleanupJob(JobContext context) throws IOException {
+    String r = getRole();
+    String id = jobIdString(context);
+    LOG.warn("{}: using deprecated cleanupJob call for {}", r, id);
+    try (DurationInfo d = new DurationInfo(LOG, "%s: cleanup Job %s", r, id)) {
+      cleanup(context, true);
+    }
+  }
+
+  /**
+   * Execute an operation; maybe suppress any raised IOException.
+   * @param suppress should raised IOEs be suppressed?
+   * @param action action (for logging when the IOE is supressed.
+   * @param operation operation
+   * @throws IOException if operation raised an IOE and suppress == false
+   */
+  protected void maybeIgnore(
+      boolean suppress,
+      String action,
+      Invoker.VoidOperation operation) throws IOException {
+    if (suppress) {
+      ignoreIOExceptions(LOG, action, "", operation);
+    } else {
+      operation.execute();
+    }
+  }
+
+  /**
+   * Execute an operation; maybe suppress any raised IOException.
+   * @param suppress should raised IOEs be suppressed?
+   * @param action action (for logging when the IOE is suppressed.
+   * @param ex  exception
+   * @throws IOException if suppress == false
+   */
+  protected void maybeIgnore(
+      boolean suppress,
+      String action,
+      IOException ex) throws IOException {
+    if (suppress) {
+      LOG.info(action, ex);
+    } else {
+      throw ex;
+    }
+  }
+
+  /**
+   * Get the commit actions instance.
+   * Subclasses may provide a mock version of this.
+   * @return the commit actions instance to use for operations.
+   */
+  protected CommitOperations getCommitOperations() {
+    return commitOperations;
+  }
+
+  /**
+   * Used in logging and reporting to help disentangle messages.
+   * @return the committer's role.
+   */
+  protected String getRole() {
+    return role;
+  }
+
+  /**
+   * Returns an {@link ExecutorService} for parallel tasks. The number of
+   * threads in the thread-pool is set by s3.multipart.committer.num-threads.
+   * If num-threads is 0, this will return null;
+   *
+   * @param context the JobContext for this commit
+   * @return an {@link ExecutorService} or null for the number of threads
+   */
+  protected final synchronized ExecutorService buildThreadPool(
+      JobContext context) {
+
+    if (threadPool == null) {
+      int numThreads = context.getConfiguration().getInt(
+          FS_S3A_COMMITTER_THREADS,
+          DEFAULT_COMMITTER_THREADS);
+      LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads);
+      if (numThreads > 0) {
+        threadPool = Executors.newFixedThreadPool(numThreads,
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("s3-committer-pool-%d")
+                .build());
+      } else {
+        return null;
+      }
+    }
+    return threadPool;
+  }
+
+  /**
+   * Delete the task attempt path without raising any errors.
+   * @param context task context
+   */
+  protected void deleteTaskAttemptPathQuietly(TaskAttemptContext context) {
+    Path attemptPath = getBaseTaskAttemptPath(context);
+    ignoreIOExceptions(LOG, "Delete task attempt path", attemptPath.toString(),
+        () -> deleteQuietly(
+            getTaskAttemptFilesystem(context), attemptPath, true));
+  }
+
+  /**
+   * Abort all pending uploads in the list.
+   * @param context job context
+   * @param pending pending uploads
+   * @param suppressExceptions should exceptions be suppressed
+   * @throws IOException any exception raised
+   */
+  protected void abortPendingUploads(JobContext context,
+      List<SinglePendingCommit> pending,
+      boolean suppressExceptions)
+      throws IOException {
+    if (pending == null || pending.isEmpty()) {
+      LOG.info("{}: no pending commits to abort", getRole());
+    } else {
+      try (DurationInfo d = new DurationInfo(LOG,
+          "Aborting %s uploads", pending.size())) {
+        Tasks.foreach(pending)
+            .executeWith(buildThreadPool(context))
+            .suppressExceptions(suppressExceptions)
+            .run(commit -> getCommitOperations().abortSingleCommit(commit));
+      }
+    }
+  }
+
+}

+ 90 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java

@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.commit;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory;
+
+/**
+ * Dynamically create the output committer based on subclass type and settings.
+ */
+public abstract class AbstractS3ACommitterFactory
+    extends PathOutputCommitterFactory {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      AbstractS3ACommitterFactory.class);
+
+  @Override
+  public PathOutputCommitter createOutputCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    FileSystem fs = getDestinationFileSystem(outputPath, context);
+    PathOutputCommitter outputCommitter;
+    if (fs instanceof S3AFileSystem) {
+      outputCommitter = createTaskCommitter((S3AFileSystem)fs,
+          outputPath, context);
+    } else {
+      throw new PathCommitException(outputPath,
+          "Filesystem not supported by this committer");
+    }
+    LOG.info("Using Commmitter {} for {}",
+        outputCommitter,
+        outputPath);
+    return outputCommitter;
+  }
+
+  /**
+   * Get the destination filesystem, returning null if there is none.
+   * Code using this must explicitly or implicitly look for a null value
+   * in the response.
+   * @param outputPath output path
+   * @param context job/task context
+   * @return the destination filesystem, if it can be determined
+   * @throws IOException if the FS cannot be instantiated
+   */
+  protected FileSystem getDestinationFileSystem(Path outputPath,
+      JobContext context)
+      throws IOException {
+    return outputPath != null ?
+          FileSystem.get(outputPath.toUri(), context.getConfiguration())
+          : null;
+  }
+
+  /**
+   * Implementation point: create a task committer for a specific filesystem.
+   * @param fileSystem destination FS.
+   * @param outputPath final output path for work
+   * @param context task context
+   * @return a committer
+   * @throws IOException any problem, including the FS not supporting
+   * the desired committer
+   */
+  public abstract PathOutputCommitter createTaskCommitter(
+      S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException;
+}

+ 240 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.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.fs.s3a.commit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_SCHEME_PATTERN;
+
+/**
+ * Constants for working with committers.
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class CommitConstants {
+
+  private CommitConstants() {
+  }
+
+  /**
+   * Path for "magic" writes: path and {@link #PENDING_SUFFIX} files:
+   * {@value}.
+   */
+  public static final String MAGIC = "__magic";
+
+  /**
+   * Marker of the start of a directory tree for calculating
+   * the final path names: {@value}.
+   */
+  public static final String BASE = "__base";
+
+  /**
+   * Suffix applied to pending commit metadata: {@value}.
+   */
+  public static final String PENDING_SUFFIX = ".pending";
+
+  /**
+   * Suffix applied to multiple pending commit metadata: {@value}.
+   */
+  public static final String PENDINGSET_SUFFIX = ".pendingset";
+
+  /**
+   * Flag to indicate whether support for the Magic committer is enabled
+   * in the filesystem.
+   * Value: {@value}.
+   */
+  public static final String MAGIC_COMMITTER_PREFIX
+      = "fs.s3a.committer.magic";
+
+  /**
+   * Flag to indicate whether support for the Magic committer is enabled
+   * in the filesystem.
+   * Value: {@value}.
+   */
+  public static final String MAGIC_COMMITTER_ENABLED
+      = MAGIC_COMMITTER_PREFIX + ".enabled";
+
+  /**
+   * Flag to indicate whether a stream is a magic output stream;
+   * returned in {@code StreamCapabilities}
+   * Value: {@value}.
+   */
+  public static final String STREAM_CAPABILITY_MAGIC_OUTPUT
+      = "s3a:magic.output.stream";
+
+  /**
+   * Flag to indicate that a store supports magic committers.
+   * returned in {@code StreamCapabilities}
+   * Value: {@value}.
+   */
+  public static final String STORE_CAPABILITY_MAGIC_COMMITTER
+      = "s3a:magic.committer";
+
+  /**
+   * Is the committer enabled by default? No.
+   */
+  public static final boolean DEFAULT_MAGIC_COMMITTER_ENABLED = false;
+
+  /**
+   * This is the "Pending" directory of the {@code FileOutputCommitter};
+   * data written here is, in that algorithm, renamed into place.
+   * Value: {@value}.
+   */
+  public static final String TEMPORARY = "_temporary";
+
+  /**
+   * Temp data which is not auto-committed: {@value}.
+   * Uses a different name from normal just to make clear it is different.
+   */
+  public static final String TEMP_DATA = "__temp-data";
+
+
+  /**
+   * Flag to trigger creation of a marker file on job completion.
+   */
+  public static final String CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER
+      = "mapreduce.fileoutputcommitter.marksuccessfuljobs";
+
+  /**
+   * Marker file to create on success: {@value}.
+   */
+  public static final String _SUCCESS = "_SUCCESS";
+
+  /** Default job marker option: {@value}. */
+  public static final boolean DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER = true;
+
+  /**
+   * Key to set for the S3A schema to use the specific committer.
+   */
+  public static final String S3A_COMMITTER_FACTORY_KEY = String.format(
+      COMMITTER_FACTORY_SCHEME_PATTERN, "s3a");
+
+  /**
+   * S3 Committer factory: {@value}.
+   * This uses the value of {@link #FS_S3A_COMMITTER_NAME}
+   * to choose the final committer.
+   */
+  public static final String S3A_COMMITTER_FACTORY =
+      S3ACommitterFactory.CLASSNAME;
+
+  /**
+   * Option to identify the S3A committer:
+   * {@value}.
+   */
+  public static final String FS_S3A_COMMITTER_NAME =
+      "fs.s3a.committer.name";
+
+  /**
+   * Option for {@link #FS_S3A_COMMITTER_NAME}:
+   * classic/file output committer: {@value}.
+   */
+  public static final String COMMITTER_NAME_FILE = "file";
+
+  /**
+   * Option for {@link #FS_S3A_COMMITTER_NAME}:
+   * magic output committer: {@value}.
+   */
+  public static final String COMMITTER_NAME_MAGIC = "magic";
+
+  /**
+   * Option for {@link #FS_S3A_COMMITTER_NAME}:
+   * directory output committer: {@value}.
+   */
+  public static final String COMMITTER_NAME_DIRECTORY = "directory";
+
+  /**
+   * Option for {@link #FS_S3A_COMMITTER_NAME}:
+   * partition output committer: {@value}.
+   */
+  public static final String COMMITTER_NAME_PARTITIONED = "partitioned";
+
+  /**
+   * Option for final files to have a uniqueness name through job attempt info,
+   * falling back to a new UUID if there is no job attempt information to use.
+   * {@value}.
+   * When writing data with the "append" conflict option, this guarantees
+   * that new data will not overwrite any existing data.
+   */
+  public static final String FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES =
+      "fs.s3a.committer.staging.unique-filenames";
+  /**
+   * Default value for {@link #FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES}:
+   * {@value}.
+   */
+  public static final boolean DEFAULT_STAGING_COMMITTER_UNIQUE_FILENAMES = true;
+
+  /**
+   * Staging committer conflict resolution policy: {@value}.
+   * Supported: fail, append, replace.
+   */
+  public static final String FS_S3A_COMMITTER_STAGING_CONFLICT_MODE =
+      "fs.s3a.committer.staging.conflict-mode";
+
+  /** Conflict mode: {@value}. */
+  public static final String CONFLICT_MODE_FAIL = "fail";
+
+  /** Conflict mode: {@value}. */
+  public static final String CONFLICT_MODE_APPEND = "append";
+
+  /** Conflict mode: {@value}. */
+  public static final String CONFLICT_MODE_REPLACE = "replace";
+
+  /** Default conflict mode: {@value}. */
+  public static final String DEFAULT_CONFLICT_MODE = CONFLICT_MODE_FAIL;
+
+  /**
+   * Number of threads in committers for parallel operations on files
+   * (upload, commit, abort, delete...): {@value}.
+   */
+  public static final String FS_S3A_COMMITTER_THREADS =
+      "fs.s3a.committer.threads";
+  /**
+   * Default value for {@link #FS_S3A_COMMITTER_THREADS}: {@value}.
+   */
+  public static final int DEFAULT_COMMITTER_THREADS = 8;
+
+  /**
+   * Path  in the cluster filesystem for temporary data: {@value}.
+   * This is for HDFS, not the local filesystem.
+   * It is only for the summary data of each file, not the actual
+   * data being committed.
+   */
+  public static final String FS_S3A_COMMITTER_STAGING_TMP_PATH =
+      "fs.s3a.committer.staging.tmp.path";
+
+
+  /**
+   * Should the staging committers abort all pending uploads to the destination
+   * directory? Default: true.
+   *
+   * Changing this is if more than one partitioned committer is
+   * writing to the same destination tree simultaneously; otherwise
+   * the first job to complete will cancel all outstanding uploads from the
+   * others. However, it may lead to leaked outstanding uploads from failed
+   * tasks. If disabled, configure the bucket lifecycle to remove uploads
+   * after a time period, and/or set up a workflow to explicitly delete
+   * entries. Otherwise there is a risk that uncommitted uploads may run up
+   * bills.
+   */
+  public static final String FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS =
+      "fs.s3a.committer.staging.abort.pending.uploads";
+
+}

+ 596 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java

@@ -0,0 +1,596 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * The implementation of the various actions a committer needs.
+ * This doesn't implement the protocol/binding to a specific execution engine,
+ * just the operations needed to to build one.
+ *
+ * When invoking FS operations, it assumes that the underlying FS is
+ * handling retries and exception translation: it does not attempt to
+ * duplicate that work.
+ *
+ */
+public class CommitOperations {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      CommitOperations.class);
+
+  /**
+   * Destination filesystem.
+   */
+  private final S3AFileSystem fs;
+
+  /** Statistics. */
+  private final S3AInstrumentation.CommitterStatistics statistics;
+
+  /**
+   * Write operations for the destination fs.
+   */
+  private final WriteOperationHelper writeOperations;
+
+  /**
+   * Filter to find all {code .pendingset} files.
+   */
+  public static final PathFilter PENDINGSET_FILTER =
+      path -> path.toString().endsWith(CommitConstants.PENDINGSET_SUFFIX);
+
+  /**
+   * Filter to find all {code .pending} files.
+   */
+  public static final PathFilter PENDING_FILTER =
+      path -> path.toString().endsWith(CommitConstants.PENDING_SUFFIX);
+
+  /**
+   * Instantiate.
+   * @param fs FS to bind to
+   */
+  public CommitOperations(S3AFileSystem fs) {
+    Preconditions.checkArgument(fs != null, "null fs");
+    this.fs = fs;
+    statistics = fs.newCommitterStatistics();
+    writeOperations = fs.createWriteOperationHelper();
+  }
+
+  /**
+   * Convert an ordered list of strings to a list of index etag parts.
+   * @param tagIds list of tags
+   * @return same list, now in numbered tuples
+   */
+  public static List<PartETag> toPartEtags(List<String> tagIds) {
+    return IntStream.range(0, tagIds.size())
+        .mapToObj(i -> new PartETag(i + 1, tagIds.get(i)))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public String toString() {
+    return "CommitOperations{" + fs.getUri() + '}';
+  }
+
+  /** @return statistics. */
+  protected S3AInstrumentation.CommitterStatistics getStatistics() {
+    return statistics;
+  }
+
+  /**
+   * Commit the operation, throwing an exception on any failure.
+   * @param commit commit to execute
+   * @throws IOException on a failure
+   */
+  public void commitOrFail(SinglePendingCommit commit) throws IOException {
+    commit(commit, commit.getFilename()).maybeRethrow();
+  }
+
+  /**
+   * Commit a single pending commit; exceptions are caught
+   * and converted to an outcome.
+   * @param commit entry to commit
+   * @param origin origin path/string for outcome text
+   * @return the outcome
+   */
+  public MaybeIOE commit(SinglePendingCommit commit, String origin) {
+    LOG.debug("Committing single commit {}", commit);
+    MaybeIOE outcome;
+    String destKey = "unknown destination";
+    try {
+      commit.validate();
+      destKey = commit.getDestinationKey();
+      long l = innerCommit(commit);
+      LOG.debug("Successful commit of file length {}", l);
+      outcome = MaybeIOE.NONE;
+      statistics.commitCompleted(commit.getLength());
+    } catch (IOException e) {
+      String msg = String.format("Failed to commit upload against %s: %s",
+          destKey, e);
+      LOG.warn(msg, e);
+      outcome = new MaybeIOE(e);
+      statistics.commitFailed();
+    } catch (Exception e) {
+      String msg = String.format("Failed to commit upload against %s," +
+          " described in %s: %s", destKey, origin, e);
+      LOG.warn(msg, e);
+      outcome = new MaybeIOE(new PathCommitException(origin, msg, e));
+      statistics.commitFailed();
+    }
+    return outcome;
+  }
+
+  /**
+   * Inner commit operation.
+   * @param commit entry to commit
+   * @return bytes committed.
+   * @throws IOException failure
+   */
+  private long innerCommit(SinglePendingCommit commit) throws IOException {
+    // finalize the commit
+    writeOperations.completeMPUwithRetries(
+        commit.getDestinationKey(),
+              commit.getUploadId(),
+              toPartEtags(commit.getEtags()),
+              commit.getLength(),
+              new AtomicInteger(0));
+    return commit.getLength();
+  }
+
+  /**
+   * Locate all files with the pending suffix under a directory.
+   * @param pendingDir directory
+   * @param recursive recursive listing?
+   * @return the list of all located entries
+   * @throws IOException if there is a problem listing the path.
+   */
+  public List<LocatedFileStatus> locateAllSinglePendingCommits(
+      Path pendingDir,
+      boolean recursive) throws IOException {
+    return listAndFilter(fs, pendingDir, recursive, PENDING_FILTER);
+  }
+
+  /**
+   * Load all single pending commits in the directory.
+   * All load failures are logged and then added to list of files which would
+   * not load.
+   * @param pendingDir directory containing commits
+   * @param recursive do a recursive scan?
+   * @return tuple of loaded entries and those pending files which would
+   * not load/validate.
+   * @throws IOException on a failure to list the files.
+   */
+  public Pair<PendingSet,
+      List<Pair<LocatedFileStatus, IOException>>>
+      loadSinglePendingCommits(Path pendingDir, boolean recursive)
+      throws IOException {
+
+    List<LocatedFileStatus> statusList = locateAllSinglePendingCommits(
+        pendingDir, recursive);
+    PendingSet commits = new PendingSet(
+        statusList.size());
+    List<Pair<LocatedFileStatus, IOException>> failures = new ArrayList<>(1);
+    for (LocatedFileStatus status : statusList) {
+      try {
+        commits.add(SinglePendingCommit.load(fs, status.getPath()));
+      } catch (IOException e) {
+        LOG.warn("Failed to load commit file {}", status.getPath(), e);
+        failures.add(Pair.of(status, e));
+      }
+    }
+    return Pair.of(commits, failures);
+  }
+
+  /**
+   * Convert any exception to an IOE, if needed.
+   * @param key key to use in a path exception
+   * @param ex exception
+   * @return an IOE, either the passed in value or a new one wrapping the other
+   * exception.
+   */
+  public IOException makeIOE(String key, Exception ex) {
+    return ex instanceof IOException
+           ? (IOException) ex
+           : new PathCommitException(key, ex.toString(), ex);
+  }
+
+  /**
+   * Abort the multipart commit supplied. This is the lower level operation
+   * which doesn't generate an outcome, instead raising an exception.
+   * @param commit pending commit to abort
+   * @throws FileNotFoundException if the abort ID is unknown
+   * @throws IOException on any failure
+   */
+  public void abortSingleCommit(SinglePendingCommit commit)
+      throws IOException {
+    String destKey = commit.getDestinationKey();
+    String origin = commit.getFilename() != null
+                    ? (" defined in " + commit.getFilename())
+                    : "";
+    String uploadId = commit.getUploadId();
+    LOG.info("Aborting commit to object {}{}", destKey, origin);
+    abortMultipartCommit(destKey, uploadId);
+  }
+
+  /**
+   * Create an {@code AbortMultipartUpload} request and POST it to S3,
+   * incrementing statistics afterwards.
+   * @param destKey destination key
+   * @param uploadId upload to cancel
+   * @throws FileNotFoundException if the abort ID is unknown
+   * @throws IOException on any failure
+   */
+  public void abortMultipartCommit(String destKey, String uploadId)
+      throws IOException {
+    try {
+      writeOperations.abortMultipartCommit(destKey, uploadId);
+    } finally {
+      statistics.commitAborted();
+    }
+  }
+
+  /**
+   * Enumerate all pending files in a dir/tree, abort.
+   * @param pendingDir directory of pending operations
+   * @param recursive recurse?
+   * @return the outcome of all the abort operations
+   * @throws IOException if there is a problem listing the path.
+   */
+  public MaybeIOE abortAllSinglePendingCommits(Path pendingDir,
+      boolean recursive)
+      throws IOException {
+    Preconditions.checkArgument(pendingDir != null, "null pendingDir");
+    LOG.debug("Aborting all pending commit filess under {}"
+            + " (recursive={}", pendingDir, recursive);
+    RemoteIterator<LocatedFileStatus> pendingFiles;
+    try {
+      pendingFiles = ls(pendingDir, recursive);
+    } catch (FileNotFoundException fnfe) {
+      LOG.info("No directory to abort {}", pendingDir);
+      return MaybeIOE.NONE;
+    }
+    MaybeIOE outcome = MaybeIOE.NONE;
+    if (!pendingFiles.hasNext()) {
+      LOG.debug("No files to abort under {}", pendingDir);
+    }
+    while (pendingFiles.hasNext()) {
+      Path pendingFile = pendingFiles.next().getPath();
+      if (pendingFile.getName().endsWith(CommitConstants.PENDING_SUFFIX)) {
+        try {
+          abortSingleCommit(SinglePendingCommit.load(fs, pendingFile));
+        } catch (FileNotFoundException e) {
+          LOG.debug("listed file already deleted: {}", pendingFile);
+        } catch (IOException | IllegalArgumentException e) {
+          if (outcome == null) {
+            outcome = new MaybeIOE(makeIOE(pendingFile.toString(), e));
+          }
+        } finally {
+          // quietly try to delete the pending file
+          S3AUtils.deleteQuietly(fs, pendingFile, false);
+        }
+      }
+    }
+    return outcome;
+  }
+
+  /**
+   * List files.
+   * @param path path
+   * @param recursive recursive listing?
+   * @return iterator
+   * @throws IOException failure
+   */
+  protected RemoteIterator<LocatedFileStatus> ls(Path path, boolean recursive)
+      throws IOException {
+    return fs.listFiles(path, recursive);
+  }
+
+  /**
+   * List all pending uploads to the destination FS under a path.
+   * @param dest destination path
+   * @return A list of the pending uploads to any directory under that path.
+   * @throws IOException IO failure
+   */
+  public List<MultipartUpload> listPendingUploadsUnderPath(Path dest)
+      throws IOException {
+    return fs.listMultipartUploads(fs.pathToKey(dest));
+  }
+
+  /**
+   * Abort all pending uploads to the destination FS under a path.
+   * @param dest destination path
+   * @return a count of the number of uploads aborted.
+   * @throws IOException IO failure
+   */
+  public int abortPendingUploadsUnderPath(Path dest) throws IOException {
+    return writeOperations.abortMultipartUploadsUnderPath(fs.pathToKey(dest));
+  }
+
+  /**
+   * Delete any existing {@code _SUCCESS} file.
+   * @param outputPath output directory
+   * @throws IOException IO problem
+   */
+  public void deleteSuccessMarker(Path outputPath) throws IOException {
+    fs.delete(new Path(outputPath, _SUCCESS), false);
+  }
+
+  /**
+   * Save the success data to the {@code _SUCCESS} file.
+   * @param outputPath output directory
+   * @param successData success data to save.
+   * @param addMetrics should the FS metrics be added?
+   * @throws IOException IO problem
+   */
+  public void createSuccessMarker(Path outputPath,
+      SuccessData successData,
+      boolean addMetrics)
+      throws IOException {
+    Preconditions.checkArgument(outputPath != null, "null outputPath");
+
+    if (addMetrics) {
+      addFileSystemStatistics(successData.getMetrics());
+    }
+    // add any diagnostics
+    Configuration conf = fs.getConf();
+    successData.addDiagnostic(S3_METADATA_STORE_IMPL,
+        conf.getTrimmed(S3_METADATA_STORE_IMPL, ""));
+    successData.addDiagnostic(METADATASTORE_AUTHORITATIVE,
+        conf.getTrimmed(METADATASTORE_AUTHORITATIVE, "false"));
+    successData.addDiagnostic(MAGIC_COMMITTER_ENABLED,
+        conf.getTrimmed(MAGIC_COMMITTER_ENABLED, "false"));
+
+    // now write
+    Path markerPath = new Path(outputPath, _SUCCESS);
+    LOG.debug("Touching success marker for job {}: {}", markerPath,
+        successData);
+    successData.save(fs, markerPath, true);
+  }
+
+  /**
+   * Revert a pending commit by deleting the destination.
+   * @param commit pending commit
+   * @throws IOException failure
+   */
+  public void revertCommit(SinglePendingCommit commit) throws IOException {
+    LOG.warn("Revert {}", commit);
+    try {
+      writeOperations.revertCommit(commit.getDestinationKey());
+    } finally {
+      statistics.commitReverted();
+    }
+  }
+
+  /**
+   * Upload all the data in the local file, returning the information
+   * needed to commit the work.
+   * @param localFile local file (be  a file)
+   * @param destPath destination path
+   * @param partition partition/subdir. Not used
+   * @param uploadPartSize size of upload
+   * @return a pending upload entry
+   * @throws IOException failure
+   */
+  public SinglePendingCommit uploadFileToPendingCommit(File localFile,
+      Path destPath,
+      String partition,
+      long uploadPartSize)
+      throws IOException {
+
+    LOG.debug("Initiating multipart upload from {} to {}",
+        localFile, destPath);
+    Preconditions.checkArgument(destPath != null);
+    if (!localFile.isFile()) {
+      throw new FileNotFoundException("Not a file: " + localFile);
+    }
+    String destURI = destPath.toString();
+    String destKey = fs.pathToKey(destPath);
+    String uploadId = null;
+
+    boolean threw = true;
+    try {
+      statistics.commitCreated();
+      uploadId = writeOperations.initiateMultiPartUpload(destKey);
+      long length = localFile.length();
+
+      SinglePendingCommit commitData = new SinglePendingCommit();
+      commitData.setDestinationKey(destKey);
+      commitData.setBucket(fs.getBucket());
+      commitData.touch(System.currentTimeMillis());
+      commitData.setUploadId(uploadId);
+      commitData.setUri(destURI);
+      commitData.setText(partition != null ? "partition: " + partition : "");
+      commitData.setLength(length);
+
+      long offset = 0;
+      long numParts = (length / uploadPartSize +
+          ((length % uploadPartSize) > 0 ? 1 : 0));
+      // always write one part, even if it is just an empty one
+      if (numParts == 0) {
+        numParts = 1;
+      }
+
+      List<PartETag> parts = new ArrayList<>((int) numParts);
+
+      LOG.debug("File size is {}, number of parts to upload = {}",
+          length, numParts);
+      for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
+        long size = Math.min(length - offset, uploadPartSize);
+        UploadPartRequest part;
+        part = writeOperations.newUploadPartRequest(
+            destKey,
+            uploadId,
+            partNumber,
+            (int) size,
+            null,
+            localFile,
+            offset);
+        part.setLastPart(partNumber == numParts);
+        UploadPartResult partResult = writeOperations.uploadPart(part);
+        offset += uploadPartSize;
+        parts.add(partResult.getPartETag());
+      }
+
+      commitData.bindCommitData(parts);
+      statistics.commitUploaded(length);
+      threw = false;
+      return commitData;
+    } finally {
+      if (threw && uploadId != null) {
+        statistics.commitAborted();
+        try {
+          abortMultipartCommit(destKey, uploadId);
+        } catch (IOException e) {
+          LOG.error("Failed to abort upload {} to {}", uploadId, destKey, e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Add the filesystem statistics to the map; overwriting anything
+   * with the same name.
+   * @param dest destination map
+   */
+  public void addFileSystemStatistics(Map<String, Long> dest) {
+    dest.putAll(fs.getInstrumentation().toMap());
+  }
+
+  /**
+   * Note that a task has completed.
+   * @param success success flag
+   */
+  public void taskCompleted(boolean success) {
+    statistics.taskCompleted(success);
+  }
+
+  /**
+   * Note that a job has completed.
+   * @param success success flag
+   */
+  public void jobCompleted(boolean success) {
+    statistics.jobCompleted(success);
+  }
+
+  /**
+   * A holder for a possible IOException; the call {@link #maybeRethrow()}
+   * will throw any exception passed into the constructor, and be a no-op
+   * if none was.
+   *
+   * Why isn't a Java 8 optional used here? The main benefit would be that
+   * {@link #maybeRethrow()} could be done as a map(), but because Java doesn't
+   * allow checked exceptions in a map, the following code is invalid
+   * <pre>
+   *   exception.map((e) -> {throw e;}
+   * </pre>
+   * As a result, the code to work with exceptions would be almost as convoluted
+   * as the original.
+   */
+  public static class MaybeIOE {
+    private final IOException exception;
+
+    public static final MaybeIOE NONE = new MaybeIOE(null);
+
+    /**
+     * Construct with an exception.
+     * @param exception exception
+     */
+    public MaybeIOE(IOException exception) {
+      this.exception = exception;
+    }
+
+    /**
+     * Get any exception.
+     * @return the exception.
+     */
+    public IOException getException() {
+      return exception;
+    }
+
+    /**
+     * Is there an exception in this class?
+     * @return true if there is an exception
+     */
+    public boolean hasException() {
+      return exception != null;
+    }
+
+    /**
+     * Rethrow any exception.
+     * @throws IOException the exception field, if non-null.
+     */
+    public void maybeRethrow() throws IOException {
+      if (exception != null) {
+        throw exception;
+      }
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("MaybeIOE{");
+      sb.append(hasException() ? exception : "");
+      sb.append('}');
+      return sb.toString();
+    }
+
+    /**
+     * Get an instance based on the exception: either a value
+     * or a reference to {@link #NONE}.
+     * @param ex exception
+     * @return an instance.
+     */
+    public static MaybeIOE of(IOException ex) {
+      return ex != null ? new MaybeIOE(ex) : NONE;
+    }
+  }
+
+
+}

+ 129 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java

@@ -0,0 +1,129 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+
+import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
+
+/**
+ * Static utility methods related to S3A commitment processing, both
+ * staging and magic.
+ *
+ * <b>Do not use in any codepath intended to be used from the S3AFS
+ * except in the committers themselves.</b>
+ */
+public final class CommitUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommitUtils.class);
+
+  private CommitUtils() {
+  }
+
+  /**
+   * Verify that the path is a magic one.
+   * @param fs filesystem
+   * @param path path
+   * @throws PathCommitException if the path isn't a magic commit path
+   */
+  public static void verifyIsMagicCommitPath(S3AFileSystem fs,
+      Path path) throws PathCommitException {
+    verifyIsMagicCommitFS(fs);
+    if (!fs.isMagicCommitPath(path)) {
+      throw new PathCommitException(path, E_BAD_PATH);
+    }
+  }
+
+  /**
+   * Verify that an S3A FS instance is a magic commit FS.
+   * @param fs filesystem
+   * @throws PathCommitException if the FS isn't a magic commit FS.
+   */
+  public static void verifyIsMagicCommitFS(S3AFileSystem fs)
+      throws PathCommitException {
+    if (!fs.isMagicCommitEnabled()) {
+      // dump out details to console for support diagnostics
+      String fsUri = fs.getUri().toString();
+      LOG.error("{}: {}:\n{}", E_NORMAL_FS, fsUri, fs);
+      // then fail
+      throw new PathCommitException(fsUri, E_NORMAL_FS);
+    }
+  }
+
+  /**
+   * Verify that an FS is an S3A FS.
+   * @param fs filesystem
+   * @param path path to to use in exception
+   * @return the typecast FS.
+   * @throws PathCommitException if the FS is not an S3A FS.
+   */
+  public static S3AFileSystem verifyIsS3AFS(FileSystem fs, Path path)
+      throws PathCommitException {
+    if (!(fs instanceof S3AFileSystem)) {
+      throw new PathCommitException(path, E_WRONG_FS);
+    }
+    return (S3AFileSystem) fs;
+  }
+
+  /**
+   * Get the S3A FS of a path.
+   * @param path path to examine
+   * @param conf config
+   * @param magicCommitRequired is magic complete required in the FS?
+   * @return the filesystem
+   * @throws PathCommitException output path isn't to an S3A FS instance, or
+   * if {@code magicCommitRequired} is set, if doesn't support these commits.
+   * @throws IOException failure to instantiate the FS
+   */
+  public static S3AFileSystem getS3AFileSystem(Path path,
+      Configuration conf,
+      boolean magicCommitRequired)
+      throws PathCommitException, IOException {
+    S3AFileSystem s3AFS = verifyIsS3AFS(path.getFileSystem(conf), path);
+    if (magicCommitRequired) {
+      verifyIsMagicCommitFS(s3AFS);
+    }
+    return s3AFS;
+  }
+
+  /**
+   * Verify that all instances in a collection are of the given class.
+   * @param it iterator
+   * @param classname classname to require
+   * @throws ValidationFailure on a failure
+   */
+  public static void validateCollectionClass(Iterable it, Class classname)
+      throws ValidationFailure {
+    for (Object o : it) {
+      verify(o.getClass().equals(classname),
+          "Collection element is not a %s: %s", classname, o.getClass());
+    }
+  }
+
+
+}

+ 192 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java

@@ -0,0 +1,192 @@
+/*
+ * 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.fs.s3a.commit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.BASE;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TEMP_DATA;
+
+/**
+ * These are commit utility methods which import classes from
+ * hadoop-mapreduce, and so only work when that module is on the
+ * classpath.
+ *
+ * <b>Do not use in any codepath intended to be used from the S3AFS
+ * except in the committers themselves.</b>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class CommitUtilsWithMR {
+
+  private CommitUtilsWithMR() {
+  }
+
+  /**
+   * Get the location of magic job attempts.
+   * @param out the base output directory.
+   * @return the location of magic job attempts.
+   */
+  public static Path getMagicJobAttemptsPath(Path out) {
+    return new Path(out, MAGIC);
+  }
+
+  /**
+   * 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 context.getConfiguration().getInt(
+        MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
+  }
+
+  /**
+   * Compute the "magic" path for a job attempt.
+   * @param appAttemptId the ID of the application attempt for this job.
+   * @param dest the final output directory
+   * @return the path to store job attempt data.
+   */
+  public static Path getMagicJobAttemptPath(int appAttemptId, Path dest) {
+    return new Path(getMagicJobAttemptsPath(dest),
+        formatAppAttemptDir(appAttemptId));
+  }
+
+  /**
+   * Format the application attempt directory.
+   * @param attemptId attempt ID
+   * @return the directory name for the application attempt
+   */
+  public static String formatAppAttemptDir(int attemptId) {
+    return String.format("app-attempt-%04d", attemptId);
+  }
+
+  /**
+   * Compute the path where the output of magic task attempts are stored.
+   * @param context the context of the job with magic tasks.
+   * @param dest destination of work
+   * @return the path where the output of magic task attempts are stored.
+   */
+  public static Path getMagicTaskAttemptsPath(JobContext context, Path dest) {
+    return new Path(getMagicJobAttemptPath(
+        getAppAttemptId(context), dest), "tasks");
+  }
+
+  /**
+   * Compute the path where the output of a task attempt is stored until
+   * that task is committed.
+   * This path is marked as a base path for relocations, so subdirectory
+   * information is preserved.
+   * @param context the context of the task attempt.
+   * @param dest The output path to commit work into
+   * @return the path where a task attempt should be stored.
+   */
+  public static Path getMagicTaskAttemptPath(TaskAttemptContext context,
+      Path dest) {
+    return new Path(getBaseMagicTaskAttemptPath(context, dest), BASE);
+  }
+
+  /**
+   * Get the base Magic attempt path, without any annotations to mark relative
+   * references.
+   * @param context task context.
+   * @param dest The output path to commit work into
+   * @return the path under which all attempts go
+   */
+  public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context,
+      Path dest) {
+    return new Path(getMagicTaskAttemptsPath(context, dest),
+          String.valueOf(context.getTaskAttemptID()));
+  }
+
+  /**
+   * Compute a path for temporary data associated with a job.
+   * This data is <i>not magic</i>
+   * @param appAttemptId the ID of the application attempt for this job.
+   * @param out output directory of job
+   * @return the path to store temporary job attempt data.
+   */
+  public static Path getTempJobAttemptPath(int appAttemptId, Path out) {
+    return new Path(new Path(out, TEMP_DATA),
+        formatAppAttemptDir(appAttemptId));
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param context task context
+   * @param out output directory of job
+   * @return the path to store temporary job attempt data.
+   */
+  public static Path getTempTaskAttemptPath(TaskAttemptContext context,
+      Path out) {
+    return new Path(getTempJobAttemptPath(getAppAttemptId(context), out),
+        String.valueOf(context.getTaskAttemptID()));
+  }
+
+  /**
+   * Get a string value of a job ID; returns meaningful text if there is no ID.
+   * @param context job context
+   * @return a string for logs
+   */
+  public static String jobIdString(JobContext context) {
+    JobID jobID = context.getJobID();
+    return jobID != null ? jobID.toString() : "(no job ID)";
+  }
+
+  /**
+   * Get a job name; returns meaningful text if there is no name.
+   * @param context job context
+   * @return a string for logs
+   */
+  public static String jobName(JobContext context) {
+    String name = context.getJobName();
+    return (name != null && !name.isEmpty()) ? name : "(anonymous)";
+  }
+
+  /**
+   * Get a configuration option, with any value in the job configuration
+   * taking priority over that in the filesystem.
+   * This allows for per-job override of FS parameters.
+   *
+   * Order is: job context, filesystem config, default value
+   *
+   * @param context job/task context
+   * @param fsConf filesystem configuration. Get this from the FS to guarantee
+   * per-bucket parameter propagation
+   * @param key key to look for
+   * @param defVal default value
+   * @return the configuration option.
+   */
+  public static String getConfigurationOption(
+      JobContext context,
+      Configuration fsConf,
+      String key,
+      String defVal) {
+    return context.getConfiguration().getTrimmed(key,
+        fsConf.getTrimmed(key, defVal));
+  }
+}

+ 60 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Duration.java

@@ -0,0 +1,60 @@
+/*
+ * 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.fs.s3a.commit;
+
+/**
+ * Little duration counter.
+ */
+public class Duration {
+
+  private final long started;
+  private long finished;
+
+  public Duration() {
+    started = time();
+    finished = started;
+  }
+
+  protected long time() {
+    return System.currentTimeMillis();
+  }
+
+  public void finished() {
+    finished = time();
+  }
+
+  public String getDurationString() {
+    return humanTime(value());
+  }
+
+  public static String humanTime(long time) {
+    long seconds = (time / 1000);
+    long minutes = (seconds / 60);
+    return String.format("%d:%02d.%03ds", minutes, seconds % 60, time % 1000);
+  }
+
+  @Override
+  public String toString() {
+    return getDurationString();
+  }
+
+  public long value() {
+    return finished -started;
+  }
+}

+ 59 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/DurationInfo.java

@@ -0,0 +1,59 @@
+/*
+ * 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.fs.s3a.commit;
+
+import org.slf4j.Logger;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A duration with logging of final state at info in the {@code close()} call.
+ * This allows it to be used in a try-with-resources clause, and have the
+ * duration automatically logged.
+ */
+@InterfaceAudience.Private
+public class DurationInfo extends Duration
+    implements AutoCloseable {
+  private final String text;
+
+  private final Logger log;
+
+  /**
+   * Create the duration text from a {@code String.format()} code call.
+   * @param log log to write to
+   * @param format format string
+   * @param args list of arguments
+   */
+  public DurationInfo(Logger log, String format, Object... args) {
+    this.text = String.format(format, args);
+    this.log = log;
+    log.info("Starting: {}", text);
+  }
+
+  @Override
+  public String toString() {
+    return text + ": duration " + super.toString();
+  }
+
+  @Override
+  public void close() {
+    finished();
+    log.info(this.toString());
+  }
+}

+ 100 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java

@@ -0,0 +1,100 @@
+/*
+ * 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.fs.s3a.commit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.PartitionedStagingCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterFactory;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED;
+
+/**
+ * These are internal constants not intended for public use.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class InternalCommitterConstants {
+
+  private InternalCommitterConstants() {
+  }
+
+  /**
+   * This is the staging committer base class; only used for testing.
+   */
+  public static final String COMMITTER_NAME_STAGING = "staging";
+
+  /**
+   * A unique identifier to use for this work: {@value}.
+   */
+  public static final String FS_S3A_COMMITTER_STAGING_UUID =
+      "fs.s3a.committer.staging.uuid";
+
+  /**
+   * Directory committer factory: {@value}.
+   */
+  public static final String STAGING_COMMITTER_FACTORY =
+      StagingCommitterFactory.CLASSNAME;
+
+  /**
+   * Directory committer factory: {@value}.
+   */
+  public static final String DIRECTORY_COMMITTER_FACTORY =
+      DirectoryStagingCommitterFactory.CLASSNAME;
+
+  /**
+   * Partitioned committer factory: {@value}.
+   */
+  public static final String PARTITION_COMMITTER_FACTORY =
+      PartitionedStagingCommitterFactory.CLASSNAME;
+
+  /**
+   * Magic committer factory: {@value}.
+   */
+  public static final String MAGIC_COMMITTER_FACTORY =
+      MagicS3GuardCommitterFactory.CLASSNAME;
+
+  /**
+   * Error text when the destination path exists and the committer
+   * must abort the job/task {@value}.
+   */
+  public static final String E_DEST_EXISTS =
+      "Destination path exists and committer conflict resolution mode is "
+          + "\"fail\"";
+
+  /** Error message for bad path: {@value}. */
+  public static final String E_BAD_PATH
+      = "Path does not represent a magic-commit path";
+
+  /** Error message if filesystem isn't magic: {@value}. */
+  public static final String E_NORMAL_FS
+      = "Filesystem does not have support for 'magic' committer enabled"
+      + " in configuration option " + MAGIC_COMMITTER_ENABLED;
+
+  /** Error message if the dest FS isn't S3A: {@value}. */
+  public static final String E_WRONG_FS
+      = "Output path is not on an S3A Filesystem";
+
+  /** Error message for a path without a magic element in the list: {@value}. */
+  public static final String E_NO_MAGIC_PATH_ELEMENT
+      = "No " + MAGIC + " element in path";
+}

+ 80 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/LocalTempDir.java

@@ -0,0 +1,80 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
+
+/**
+ * A class which manages access to a temporary directory store, uses the
+ * directories listed in {@link Constants#BUFFER_DIR} for this.
+ */
+final class LocalTempDir {
+
+  private LocalTempDir() {
+  }
+
+  private static LocalDirAllocator directoryAllocator;
+
+  private static synchronized LocalDirAllocator getAllocator(
+      Configuration conf, String key) {
+    if (directoryAllocator != null) {
+      String bufferDir = conf.get(key) != null
+          ? key : Constants.HADOOP_TMP_DIR;
+      directoryAllocator = new LocalDirAllocator(bufferDir);
+    }
+    return directoryAllocator;
+  }
+
+  /**
+   * Create a temp file.
+   * @param conf configuration to use when creating the allocator
+   * @param prefix filename prefix
+   * @param size file size, or -1 if not known
+   * @return the temp file. The file has been created.
+   * @throws IOException IO failure
+   */
+  public static File tempFile(Configuration conf, String prefix, long size)
+      throws IOException {
+    return getAllocator(conf, BUFFER_DIR).createTmpFileForWrite(
+        prefix, size, conf);
+  }
+
+  /**
+   * Get a temporary path.
+   * @param conf configuration to use when creating the allocator
+   * @param prefix filename prefix
+   * @param size file size, or -1 if not known
+   * @return the temp path.
+   * @throws IOException IO failure
+   */
+  public static Path tempPath(Configuration conf, String prefix, long size)
+      throws IOException {
+    return getAllocator(conf, BUFFER_DIR)
+        .getLocalPathForWrite(prefix, size, conf);
+  }
+
+}

+ 182 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java

@@ -0,0 +1,182 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
+
+import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
+
+/**
+ * Adds the code needed for S3A to support magic committers.
+ * It's pulled out to keep S3A FS class slightly less complex.
+ * This class can be instantiated even when magic commit is disabled;
+ * in this case:
+ * <ol>
+ *   <li>{@link #isMagicCommitPath(Path)} will always return false.</li>
+ *   <li>{@link #createTracker(Path, String)} will always return an instance
+ *   of {@link PutTracker}.</li>
+ * </ol>
+ *
+ * <p>Important</p>: must not directly or indirectly import a class which
+ * uses any datatype in hadoop-mapreduce.
+ */
+public class MagicCommitIntegration {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MagicCommitIntegration.class);
+  private final S3AFileSystem owner;
+  private final boolean magicCommitEnabled;
+
+  /**
+   * Instantiate.
+   * @param owner owner class
+   * @param magicCommitEnabled is magic commit enabled.
+   */
+  public MagicCommitIntegration(S3AFileSystem owner,
+      boolean magicCommitEnabled) {
+    this.owner = owner;
+    this.magicCommitEnabled = magicCommitEnabled;
+  }
+
+  /**
+   * Given an (elements, key) pair, return the key of the final destination of
+   * the PUT, that is: where the final path is expected to go?
+   * @param elements path split to elements
+   * @param key key
+   * @return key for final put. If this is not a magic commit, the
+   * same as the key in.
+   */
+  public String keyOfFinalDestination(List<String> elements, String key) {
+    if (isMagicCommitPath(elements)) {
+      return elementsToKey(finalDestination(elements));
+    } else {
+      return key;
+    }
+  }
+
+  /**
+   * Given a path and a key to that same path, create a tracker for it.
+   * This specific tracker will be chosen based on whether or not
+   * the path is a magic one.
+   * @param path path of nominal write
+   * @param key key of path of nominal write
+   * @return the tracker for this operation.
+   */
+  public PutTracker createTracker(Path path, String key) {
+    final List<String> elements = splitPathToElements(path);
+    PutTracker tracker;
+
+    if(isMagicFile(elements)) {
+      // path is of a magic file
+      if (isMagicCommitPath(elements)) {
+        final String destKey = keyOfFinalDestination(elements, key);
+        String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
+        owner.getInstrumentation()
+            .incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1);
+        tracker = new MagicCommitTracker(path,
+            owner.getBucket(),
+            key,
+            destKey,
+            pendingsetPath,
+            owner.createWriteOperationHelper());
+        LOG.debug("Created {}", tracker);
+      } else {
+        LOG.warn("File being created has a \"magic\" path, but the filesystem"
+            + " has magic file support disabled: {}", path);
+        // downgrade to standard multipart tracking
+        tracker = new PutTracker(key);
+      }
+    } else {
+      // standard multipart tracking
+      tracker = new PutTracker(key);
+    }
+    return tracker;
+  }
+
+  /**
+   * This performs the calculation of the final destination of a set
+   * of elements.
+   *
+   * @param elements original (do not edit after this call)
+   * @return a list of elements, possibly empty
+   */
+  private List<String> finalDestination(List<String> elements) {
+    return magicCommitEnabled ?
+        MagicCommitPaths.finalDestination(elements)
+        : elements;
+  }
+
+  /**
+   * Is magic commit enabled?
+   * @return true if magic commit is turned on.
+   */
+  public boolean isMagicCommitEnabled() {
+    return magicCommitEnabled;
+  }
+
+  /**
+   * Predicate: is a path a magic commit path?
+   * @param path path to examine
+   * @return true if the path is or is under a magic directory
+   */
+  public boolean isMagicCommitPath(Path path) {
+    return isMagicCommitPath(splitPathToElements(path));
+  }
+
+  /**
+   * Is this path a magic commit path in this filesystem?
+   * True if magic commit is enabled, the path is magic
+   * and the path is not actually a commit metadata file.
+   * @param elements element list
+   * @return true if writing path is to be uprated to a magic file write
+   */
+  private boolean isMagicCommitPath(List<String> elements) {
+    return magicCommitEnabled && isMagicFile(elements);
+  }
+
+  /**
+   * Is the file a magic file: this predicate doesn't check
+   * for the FS actually having the magic bit being set.
+   * @param elements path elements
+   * @return true if the path is one a magic file write expects.
+   */
+  private boolean isMagicFile(List<String> elements) {
+    return isMagicPath(elements) &&
+        !isCommitMetadataFile(elements);
+  }
+
+  /**
+   * Does this file contain all the commit metadata?
+   * @param elements path element list
+   * @return true if this file is one of the commit metadata files.
+   */
+  private boolean isCommitMetadataFile(List<String> elements) {
+    String last = elements.get(elements.size() - 1);
+    return last.endsWith(CommitConstants.PENDING_SUFFIX)
+        || last.endsWith(CommitConstants.PENDINGSET_SUFFIX);
+  }
+
+}

+ 229 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitPaths.java

@@ -0,0 +1,229 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.BASE;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC;
+import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_MAGIC_PATH_ELEMENT;
+
+/**
+ * Operations on (magic) paths.
+ */
+public final class MagicCommitPaths {
+
+  private MagicCommitPaths() {
+  }
+
+  /**
+   * Take an absolute path, split it into a list of elements.
+   * If empty, the path is the root path.
+   * @param path input path
+   * @return a possibly empty list of elements.
+   * @throws IllegalArgumentException if the path is invalid -relative, empty...
+   */
+  public static List<String> splitPathToElements(Path path) {
+    checkArgument(path.isAbsolute(), "path is relative");
+    String uriPath = path.toUri().getPath();
+    checkArgument(!uriPath.isEmpty(), "empty path");
+    if ("/".equals(uriPath)) {
+      // special case: empty list
+      return new ArrayList<>(0);
+    }
+    List<String> elements = new ArrayList<>();
+    int len = uriPath.length();
+    int firstElementChar = 1;
+    int endOfElement = uriPath.indexOf('/', firstElementChar);
+    while (endOfElement > 0) {
+      elements.add(uriPath.substring(firstElementChar, endOfElement));
+      firstElementChar = endOfElement + 1;
+      endOfElement = firstElementChar == len ? -1
+          : uriPath.indexOf('/', firstElementChar);
+    }
+    // expect a possible child element here
+    if (firstElementChar != len) {
+      elements.add(uriPath.substring(firstElementChar));
+    }
+    return elements;
+  }
+
+  /**
+   * Is a path in the magic tree?
+   * @param elements element list
+   * @return true if a path is considered magic
+   */
+  public static boolean isMagicPath(List<String> elements) {
+    return elements.contains(MAGIC);
+  }
+
+  /**
+   * Does the list of magic elements contain a base path marker?
+   * @param elements element list, already stripped out
+   * from the magic tree.
+   * @return true if a path has a base directory
+   */
+  public static boolean containsBasePath(List<String> elements) {
+    return elements.contains(BASE);
+  }
+
+  /**
+   * Get the index of the magic path element.
+   * @param elements full path element list
+   * @return the index.
+   * @throws IllegalArgumentException if there is no magic element
+   */
+  public static int magicElementIndex(List<String> elements) {
+    int index = elements.indexOf(MAGIC);
+    checkArgument(index >= 0, E_NO_MAGIC_PATH_ELEMENT);
+    return index;
+  }
+
+  /**
+   * Get the parent path elements of the magic path.
+   * The list may be immutable or may be a view of the underlying list.
+   * Both the parameter list and the returned list MUST NOT be modified.
+   * @param elements full path element list
+   * @return the parent elements; may be empty
+   */
+  public static List<String> magicPathParents(List<String> elements) {
+    return elements.subList(0, magicElementIndex(elements));
+  }
+
+  /**
+   * Get the child path elements under the magic path.
+   * The list may be immutable or may be a view of the underlying list.
+   * Both the parameter list and the returned list MUST NOT be modified.
+   * @param elements full path element list
+   * @return the child elements; may be empty
+   */
+  public static List<String> magicPathChildren(List<String> elements) {
+    int index = magicElementIndex(elements);
+    int len = elements.size();
+    if (index == len - 1) {
+      // empty index
+      return Collections.emptyList();
+    } else {
+      return elements.subList(index + 1, len);
+    }
+  }
+
+  /**
+   * Get any child path elements under any {@code __base} path,
+   * or an empty list if there is either: no {@code __base} path element,
+   * or no child entries under it.
+   * The list may be immutable or may be a view of the underlying list.
+   * Both the parameter list and the returned list MUST NOT be modified.
+   * @param elements full path element list
+   * @return the child elements; may be empty
+   */
+  public static List<String> basePathChildren(List<String> elements) {
+    int index = elements.indexOf(BASE);
+    if (index < 0) {
+      return Collections.emptyList();
+    }
+    int len = elements.size();
+    if (index == len - 1) {
+      // empty index
+      return Collections.emptyList();
+    } else {
+      return elements.subList(index + 1, len);
+    }
+  }
+
+  /**
+   * Take a list of elements and create an S3 key by joining them
+   * with "/" between each one.
+   * @param elements path elements
+   * @return a path which can be used in the AWS API
+   */
+  public static String elementsToKey(List<String> elements) {
+    return StringUtils.join("/", elements);
+  }
+
+  /**
+   * Get the filename of a path: the last element.
+   * @param elements element list.
+   * @return the filename; the last element.
+   */
+  public static String filename(List<String> elements) {
+    return lastElement(elements);
+  }
+
+  /**
+   * Last element of a (non-empty) list.
+   * @param strings strings in
+   * @return the last one.
+   */
+  public static String lastElement(List<String> strings) {
+    checkArgument(!strings.isEmpty(), "empty list");
+    return strings.get(strings.size() - 1);
+  }
+
+  /**
+   * Get the magic subdirectory of a destination directory.
+   * @param destDir the destination directory
+   * @return a new path.
+   */
+  public static Path magicSubdir(Path destDir) {
+    return new Path(destDir, MAGIC);
+  }
+
+  /**
+   * Calculates the final destination of a file.
+   * This is the parent of any {@code __magic} element, and the filename
+   * of the path. That is: all intermediate child path elements are discarded.
+   * Why so? paths under the magic path include job attempt and task attempt
+   * subdirectories, which need to be skipped.
+   *
+   * If there is a {@code __base} directory in the children, then it becomes
+   * a base for unflattened paths, that is: all its children are pulled into
+   * the final destination.
+   * @param elements element list.
+   * @return the path
+   */
+  public static List<String> finalDestination(List<String> elements) {
+    if (isMagicPath(elements)) {
+      List<String> destDir = magicPathParents(elements);
+      List<String> children = magicPathChildren(elements);
+      checkArgument(!children.isEmpty(), "No path found under " +
+          MAGIC);
+      ArrayList<String> dest = new ArrayList<>(destDir);
+      if (containsBasePath(children)) {
+        // there's a base marker in the path
+        List<String> baseChildren = basePathChildren(children);
+        checkArgument(!baseChildren.isEmpty(),
+            "No path found under " + BASE);
+        dest.addAll(baseChildren);
+      } else {
+        dest.add(filename(children));
+      }
+      return dest;
+    } else {
+      return elements;
+    }
+  }
+
+}

+ 43 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PathCommitException.java

@@ -0,0 +1,43 @@
+/*
+ * 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.fs.s3a.commit;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Path exception to use for various commit issues.
+ */
+public class PathCommitException extends PathIOException {
+  public PathCommitException(String path, Throwable cause) {
+    super(path, cause);
+  }
+
+  public PathCommitException(String path, String error) {
+    super(path, error);
+  }
+
+  public PathCommitException(Path path, String error) {
+    super(path != null ? path.toString() : "", error);
+  }
+
+  public PathCommitException(String path, String error, Throwable cause) {
+    super(path, error, cause);
+  }
+}

+ 100 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java

@@ -0,0 +1,100 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.amazonaws.services.s3.model.PartETag;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Multipart put tracker.
+ * Base class does nothing except declare that any
+ * MPU must complete in the {@code close()} operation.
+ *
+ */
+@InterfaceAudience.Private
+public class PutTracker {
+
+  /** The destination. */
+  private final String destKey;
+
+  /**
+   * Instantiate.
+   * @param destKey destination key
+   */
+  public PutTracker(String destKey) {
+    this.destKey = destKey;
+  }
+
+  /**
+   * Startup event.
+   * @return true if the multipart should start immediately.
+   * @throws IOException any IO problem.
+   */
+  public boolean initialize() throws IOException {
+    return false;
+  }
+
+  /**
+   * Flag to indicate that output is not immediately visible after the stream
+   * is closed. Default: false.
+   * @return true if the output's visibility will be delayed.
+   */
+  public boolean outputImmediatelyVisible() {
+    return true;
+  }
+
+  /**
+   * Callback when the upload is is about to complete.
+   * @param uploadId Upload ID
+   * @param parts list of parts
+   * @param bytesWritten bytes written
+   * @return true if the commit is to be initiated immediately.
+   * False implies the output stream does not need to worry about
+   * what happens.
+   * @throws IOException I/O problem or validation failure.
+   */
+  public boolean aboutToComplete(String uploadId,
+      List<PartETag> parts,
+      long bytesWritten)
+      throws IOException {
+    return true;
+  }
+
+  /**
+   * get the destination key. The default implementation returns the
+   * key passed in: there is no adjustment of the destination.
+   * @return the destination to use in PUT requests.
+   */
+  public String getDestKey() {
+    return destKey;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "DefaultPutTracker{");
+    sb.append("destKey='").append(destKey).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

+ 129 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java

@@ -0,0 +1,129 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.PartitionedStagingCommitterFactory;
+import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterFactory;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+
+/**
+ * The S3A committer factory which chooses the committer based on the
+ * specific option chosen in a per-bucket basis from the property
+ * {@link CommitConstants#FS_S3A_COMMITTER_NAME}.
+ *
+ * This should be instantiated by using the property value {@link #CLASSNAME}
+ * as the committer for the job, then set the filesystem property
+ * {@link CommitConstants#FS_S3A_COMMITTER_NAME} to one of
+ * <ul>
+ *   <li>{@link CommitConstants#COMMITTER_NAME_FILE}: File committer.</li>
+ *   <li>{@link CommitConstants#COMMITTER_NAME_DIRECTORY}:
+ *   Staging directory committer.</li>
+ *   <li>{@link CommitConstants#COMMITTER_NAME_PARTITIONED}:
+ *   Staging partitioned committer.</li>
+ *   <li>{@link CommitConstants#COMMITTER_NAME_MAGIC}:
+ *   the "Magic" committer</li>
+ *   <li>{@link InternalCommitterConstants#COMMITTER_NAME_STAGING}:
+ *   the "staging" committer, which isn't intended for use outside tests.</li>
+ * </ul>
+ * There are no checks to verify that the filesystem is compatible with
+ * the committer.
+ */
+public class S3ACommitterFactory extends AbstractS3ACommitterFactory {
+
+  /**
+   * Name of this class: {@value}.
+   */
+  public static final String CLASSNAME
+      = "org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory";
+
+  /**
+   * Create a task committer.
+   * @param fileSystem destination FS.
+   * @param outputPath final output path for work
+   * @param context job context
+   * @return a committer
+   * @throws IOException instantiation failure
+   */
+  @Override
+  public PathOutputCommitter createTaskCommitter(S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    AbstractS3ACommitterFactory factory = chooseCommitterFactory(fileSystem,
+        outputPath,
+        context.getConfiguration());
+    return factory != null ?
+      factory.createTaskCommitter(fileSystem, outputPath, context)
+      : createFileOutputCommitter(outputPath, context);
+  }
+
+  /**
+   * Choose a committer from the FS and task configurations. Task Configuration
+   * takes priority, allowing execution engines to dynamically change
+   * committer on a query-by-query basis.
+   * @param fileSystem FS
+   * @param outputPath destination path
+   * @param taskConf configuration from the task
+   * @return An S3A committer if chosen, or "null" for the classic value
+   * @throws PathCommitException on a failure to identify the committer
+   */
+  private AbstractS3ACommitterFactory chooseCommitterFactory(
+      S3AFileSystem fileSystem,
+      Path outputPath,
+      Configuration taskConf) throws PathCommitException {
+    AbstractS3ACommitterFactory factory;
+
+    // the FS conf will have had its per-bucket values resolved, unlike
+    // job/task configurations.
+    Configuration fsConf = fileSystem.getConf();
+
+    String name = fsConf.getTrimmed(FS_S3A_COMMITTER_NAME, COMMITTER_NAME_FILE);
+    name = taskConf.getTrimmed(FS_S3A_COMMITTER_NAME, name);
+    switch (name) {
+    case COMMITTER_NAME_FILE:
+      factory = null;
+      break;
+    case COMMITTER_NAME_DIRECTORY:
+      factory = new DirectoryStagingCommitterFactory();
+      break;
+    case COMMITTER_NAME_PARTITIONED:
+      factory = new PartitionedStagingCommitterFactory();
+      break;
+    case COMMITTER_NAME_MAGIC:
+      factory = new MagicS3GuardCommitterFactory();
+      break;
+    case InternalCommitterConstants.COMMITTER_NAME_STAGING:
+      factory = new StagingCommitterFactory();
+      break;
+    default:
+      throw new PathCommitException(outputPath,
+          "Unknown committer: \"" + name + "\"");
+    }
+    return factory;
+  }
+}

+ 410 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java

@@ -0,0 +1,410 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for parallel execution, takes closures for the various
+ * actions.
+ * There is no retry logic: it is expected to be handled by the closures.
+ */
+public final class Tasks {
+  private static final Logger LOG = LoggerFactory.getLogger(Tasks.class);
+
+  private Tasks() {
+  }
+
+  /**
+   * Callback invoked to process an item.
+   * @param <I> item type being processed
+   * @param <E> exception class which may be raised
+   */
+  @FunctionalInterface
+  public interface Task<I, E extends Exception> {
+    void run(I item) throws E;
+  }
+
+  /**
+   * Callback invoked on a failure.
+   * @param <I> item type being processed
+   * @param <E> exception class which may be raised
+   */
+  @FunctionalInterface
+  public interface FailureTask<I, E extends Exception> {
+
+    /**
+     * process a failure.
+     * @param item item the task is processing
+     * @param exception the exception which was raised.
+     * @throws E Exception of type E
+     */
+    void run(I item, Exception exception) throws E;
+  }
+
+  /**
+   * Builder for task execution.
+   * @param <I> item type
+   */
+  public static class Builder<I> {
+    private final Iterable<I> items;
+    private ExecutorService service = null;
+    private FailureTask<I, ?> onFailure = null;
+    private boolean stopOnFailure = false;
+    private boolean suppressExceptions = false;
+    private Task<I, ?> revertTask = null;
+    private boolean stopRevertsOnFailure = false;
+    private Task<I, ?> abortTask = null;
+    private boolean stopAbortsOnFailure = false;
+
+    /**
+     * Create the builder.
+     * @param items items to process
+     */
+    Builder(Iterable<I> items) {
+      this.items = items;
+    }
+
+    /**
+     * Declare executor service: if null, the tasks are executed in a single
+     * thread.
+     * @param executorService service to schedule tasks with.
+     * @return this builder.
+     */
+    public Builder<I> executeWith(ExecutorService executorService) {
+      this.service = executorService;
+      return this;
+    }
+
+    public Builder<I> onFailure(FailureTask<I, ?> task) {
+      this.onFailure = task;
+      return this;
+    }
+
+    public Builder<I> stopOnFailure() {
+      this.stopOnFailure = true;
+      return this;
+    }
+
+    public Builder<I> suppressExceptions() {
+      return suppressExceptions(true);
+    }
+
+    public Builder<I> suppressExceptions(boolean suppress) {
+      this.suppressExceptions = suppress;
+      return this;
+    }
+
+    public Builder<I> revertWith(Task<I, ?> task) {
+      this.revertTask = task;
+      return this;
+    }
+
+    public Builder<I> stopRevertsOnFailure() {
+      this.stopRevertsOnFailure = true;
+      return this;
+    }
+
+    public Builder<I> abortWith(Task<I, ?> task) {
+      this.abortTask = task;
+      return this;
+    }
+
+    public Builder<I> stopAbortsOnFailure() {
+      this.stopAbortsOnFailure = true;
+      return this;
+    }
+
+    public <E extends Exception> boolean run(Task<I, E> task) throws E {
+      if (service != null) {
+        return runParallel(task);
+      } else {
+        return runSingleThreaded(task);
+      }
+    }
+
+    private <E extends Exception> boolean runSingleThreaded(Task<I, E> task)
+        throws E {
+      List<I> succeeded = new ArrayList<>();
+      List<Exception> exceptions = new ArrayList<>();
+
+      Iterator<I> iterator = items.iterator();
+      boolean threw = true;
+      try {
+        while (iterator.hasNext()) {
+          I item = iterator.next();
+          try {
+            task.run(item);
+            succeeded.add(item);
+
+          } catch (Exception e) {
+            exceptions.add(e);
+
+            if (onFailure != null) {
+              try {
+                onFailure.run(item, e);
+              } catch (Exception failException) {
+                LOG.error("Failed to clean up on failure", e);
+                // keep going
+              }
+            }
+
+            if (stopOnFailure) {
+              break;
+            }
+          }
+        }
+
+        threw = false;
+
+      } finally {
+        // threw handles exceptions that were *not* caught by the catch block,
+        // and exceptions that were caught and possibly handled by onFailure
+        // are kept in exceptions.
+        if (threw || !exceptions.isEmpty()) {
+          if (revertTask != null) {
+            boolean failed = false;
+            for (I item : succeeded) {
+              try {
+                revertTask.run(item);
+              } catch (Exception e) {
+                LOG.error("Failed to revert task", e);
+                failed = true;
+                // keep going
+              }
+              if (stopRevertsOnFailure && failed) {
+                break;
+              }
+            }
+          }
+
+          if (abortTask != null) {
+            boolean failed = false;
+            while (iterator.hasNext()) {
+              try {
+                abortTask.run(iterator.next());
+              } catch (Exception e) {
+                failed = true;
+                LOG.error("Failed to abort task", e);
+                // keep going
+              }
+              if (stopAbortsOnFailure && failed) {
+                break;
+              }
+            }
+          }
+        }
+      }
+
+      if (!suppressExceptions && !exceptions.isEmpty()) {
+        Tasks.<E>throwOne(exceptions);
+      }
+
+      return !threw && exceptions.isEmpty();
+    }
+
+    private <E extends Exception> boolean runParallel(final Task<I, E> task)
+        throws E {
+      final Queue<I> succeeded = new ConcurrentLinkedQueue<>();
+      final Queue<Exception> exceptions = new ConcurrentLinkedQueue<>();
+      final AtomicBoolean taskFailed = new AtomicBoolean(false);
+      final AtomicBoolean abortFailed = new AtomicBoolean(false);
+      final AtomicBoolean revertFailed = new AtomicBoolean(false);
+
+      List<Future<?>> futures = new ArrayList<>();
+
+      for (final I item : items) {
+        // submit a task for each item that will either run or abort the task
+        futures.add(service.submit(new Runnable() {
+          @Override
+          public void run() {
+            if (!(stopOnFailure && taskFailed.get())) {
+              // run the task
+              boolean threw = true;
+              try {
+                LOG.debug("Executing task");
+                task.run(item);
+                succeeded.add(item);
+                LOG.debug("Task succeeded");
+
+                threw = false;
+
+              } catch (Exception e) {
+                taskFailed.set(true);
+                exceptions.add(e);
+                LOG.info("Task failed", e);
+
+                if (onFailure != null) {
+                  try {
+                    onFailure.run(item, e);
+                  } catch (Exception failException) {
+                    LOG.error("Failed to clean up on failure", e);
+                    // swallow the exception
+                  }
+                }
+              } finally {
+                if (threw) {
+                  taskFailed.set(true);
+                }
+              }
+
+            } else if (abortTask != null) {
+              // abort the task instead of running it
+              if (stopAbortsOnFailure && abortFailed.get()) {
+                return;
+              }
+
+              boolean failed = true;
+              try {
+                LOG.info("Aborting task");
+                abortTask.run(item);
+                failed = false;
+              } catch (Exception e) {
+                LOG.error("Failed to abort task", e);
+                // swallow the exception
+              } finally {
+                if (failed) {
+                  abortFailed.set(true);
+                }
+              }
+            }
+          }
+        }));
+      }
+
+      // let the above tasks complete (or abort)
+      waitFor(futures);
+      int futureCount = futures.size();
+      futures.clear();
+
+      if (taskFailed.get() && revertTask != null) {
+        // at least one task failed, revert any that succeeded
+        LOG.info("Reverting all {} succeeded tasks from {} futures",
+            succeeded.size(), futureCount);
+        for (final I item : succeeded) {
+          futures.add(service.submit(() -> {
+            if (stopRevertsOnFailure && revertFailed.get()) {
+              return;
+            }
+
+            boolean failed = true;
+            try {
+              revertTask.run(item);
+              failed = false;
+            } catch (Exception e) {
+              LOG.error("Failed to revert task", e);
+              // swallow the exception
+            } finally {
+              if (failed) {
+                revertFailed.set(true);
+              }
+            }
+          }));
+        }
+
+        // let the revert tasks complete
+        waitFor(futures);
+      }
+
+      if (!suppressExceptions && !exceptions.isEmpty()) {
+        Tasks.<E>throwOne(exceptions);
+      }
+
+      return !taskFailed.get();
+    }
+  }
+
+  /**
+   * Wait for all the futures to complete; there's a small sleep between
+   * each iteration; enough to yield the CPU.
+   * @param futures futures.
+   */
+  private static void waitFor(Collection<Future<?>> futures) {
+    int size = futures.size();
+    LOG.debug("Waiting for {} tasks to complete", size);
+    int oldNumFinished = 0;
+    while (true) {
+      int numFinished = (int) futures.stream().filter(Future::isDone).count();
+
+      if (oldNumFinished != numFinished) {
+        LOG.debug("Finished count -> {}/{}", numFinished, size);
+        oldNumFinished = numFinished;
+      }
+
+      if (numFinished == size) {
+        // all of the futures are done, stop looping
+        break;
+      } else {
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          futures.forEach(future -> future.cancel(true));
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
+  }
+
+  public static <I> Builder<I> foreach(Iterable<I> items) {
+    return new Builder<>(items);
+  }
+
+  public static <I> Builder<I> foreach(I[] items) {
+    return new Builder<>(Arrays.asList(items));
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <E extends Exception> void throwOne(
+      Collection<Exception> exceptions)
+      throws E {
+    Iterator<Exception> iter = exceptions.iterator();
+    Exception e = iter.next();
+    Class<? extends Exception> exceptionClass = e.getClass();
+
+    while (iter.hasNext()) {
+      Exception other = iter.next();
+      if (!exceptionClass.isInstance(other)) {
+        e.addSuppressed(other);
+      }
+    }
+
+    Tasks.<E>castAndThrow(e);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <E extends Exception> void castAndThrow(Exception e) throws E {
+    if (e instanceof RuntimeException) {
+      throw (RuntimeException) e;
+    }
+    throw (E) e;
+  }
+}

+ 53 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/ValidationFailure.java

@@ -0,0 +1,53 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+
+/**
+ * Exception raised on validation failures; kept as an IOException
+ * for consistency with other failures.
+ */
+public class ValidationFailure extends IOException {
+
+  /**
+   * Create an instance with string formatting applied to the message
+   * and arguments.
+   * @param message error message
+   * @param args optional list of arguments
+   */
+  public ValidationFailure(String message, Object... args) {
+    super(String.format(message, args));
+  }
+
+  /**
+   * 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 ValidationFailure on a failure
+   */
+  public static void verify(boolean expression,
+      String message,
+      Object... args) throws ValidationFailure {
+    if (!expression) {
+      throw new ValidationFailure(message, args);
+    }
+  }
+}

+ 192 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java

@@ -0,0 +1,192 @@
+/*
+ * 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.fs.s3a.commit.files;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.s3a.commit.ValidationFailure;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
+import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
+
+/**
+ * Persistent format for multiple pending commits.
+ * Contains 0 or more {@link SinglePendingCommit} entries; validation logic
+ * checks those values on load.
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PendingSet extends PersistentCommitData {
+  private static final Logger LOG = LoggerFactory.getLogger(PendingSet.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 = 0x11000 + VERSION;
+
+
+  /** Version marker. */
+  private int version = VERSION;
+
+  /**
+   * Commit list.
+   */
+  private List<SinglePendingCommit> commits;
+
+  /**
+   * Any custom extra data committer subclasses may choose to add.
+   */
+  private final Map<String, String> extraData = new HashMap<>(0);
+
+  public PendingSet() {
+    this(0);
+  }
+
+
+  public PendingSet(int size) {
+    commits = new ArrayList<>(size);
+  }
+
+  /**
+   * Get a JSON serializer for this class.
+   * @return a serializer.
+   */
+  public static JsonSerialization<PendingSet> serializer() {
+    return new JsonSerialization<>(PendingSet.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
+   * @throws ValidationFailure if the data is invalid
+   */
+  public static PendingSet load(FileSystem fs, Path path)
+      throws IOException {
+    LOG.debug("Reading pending commits in file {}", path);
+    PendingSet instance = serializer().load(fs, path);
+    instance.validate();
+    return instance;
+  }
+
+  /**
+   * Add a commit.
+   * @param commit the single commit
+   */
+  public void add(SinglePendingCommit commit) {
+    commits.add(commit);
+  }
+
+  /**
+   * Deserialize via java Serialization API: deserialize the instance
+   * and then call {@link #validate()} to verify that the deserialized
+   * data is valid.
+   * @param inStream input stream
+   * @throws IOException IO problem or validation failure
+   * @throws ClassNotFoundException reflection problems
+   */
+  private void readObject(ObjectInputStream inStream) throws IOException,
+      ClassNotFoundException {
+    inStream.defaultReadObject();
+    validate();
+  }
+
+  /**
+   * Validate the data: those fields which must be non empty, must be set.
+   * @throws ValidationFailure if the data is invalid
+   */
+  public void validate() throws ValidationFailure {
+    verify(version == VERSION, "Wrong version: %s", version);
+    validateCollectionClass(extraData.keySet(), String.class);
+    validateCollectionClass(extraData.values(), String.class);
+    Set<String> destinations = new HashSet<>(commits.size());
+    validateCollectionClass(commits, SinglePendingCommit.class);
+    for (SinglePendingCommit c : commits) {
+      c.validate();
+      verify(!destinations.contains(c.getDestinationKey()),
+          "Destination %s is written to by more than one pending commit",
+          c.getDestinationKey());
+      destinations.add(c.getDestinationKey());
+    }
+  }
+
+  @Override
+  public byte[] toBytes() throws IOException {
+    return serializer().toBytes(this);
+  }
+
+  /**
+   * Number of commits.
+   * @return the number of commits in this structure.
+   */
+  public int size() {
+    return commits != null ? commits.size() : 0;
+  }
+
+  @Override
+  public void save(FileSystem fs, Path path, boolean overwrite)
+      throws IOException {
+    serializer().save(fs, path, this, overwrite);
+  }
+
+  /** @return the version marker. */
+  public int getVersion() {
+    return version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  /**
+   * @return commit list.
+   */
+  public List<SinglePendingCommit> getCommits() {
+    return commits;
+  }
+
+  public void setCommits(List<SinglePendingCommit> commits) {
+    this.commits = commits;
+  }
+}

+ 69 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java

@@ -0,0 +1,69 @@
+/*
+ * 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.fs.s3a.commit.files;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+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.s3a.commit.ValidationFailure;
+
+/**
+ * Class for single/multiple commit data structures.
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class PersistentCommitData implements Serializable {
+
+  /**
+   * 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;
+
+  /**
+   * Validate the data: those fields which must be non empty, must be set.
+   * @throws ValidationFailure if the data is invalid
+   */
+  public abstract void validate() throws ValidationFailure;
+
+  /**
+   * 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;
+
+}

+ 432 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java

@@ -0,0 +1,432 @@
+/*
+ * 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.fs.s3a.commit.files;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import com.amazonaws.services.s3.model.PartETag;
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
+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.s3a.commit.ValidationFailure;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
+import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
+import static org.apache.hadoop.util.StringUtils.join;
+
+/**
+ * This is the serialization format for uploads yet to be committerd.
+ *
+ * It's marked as {@link Serializable} so that it can be passed in RPC
+ * calls; for this to work it relies on the fact that java.io ArrayList
+ * and LinkedList are serializable. If any other list type is used for etags,
+ * it must also be serialized. Jackson expects lists, and it is used
+ * to persist to disk.
+ *
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SinglePendingCommit extends PersistentCommitData
+    implements Iterable<String> {
+
+  /**
+   * Serialization ID: {@value}.
+   */
+  private static final long serialVersionUID = 0x10000 + VERSION;
+
+  /** Version marker. */
+  private int version = VERSION;
+
+  /**
+   * This is the filename of the pending file itself.
+   * Used during processing; it's persistent value, if any, is ignored.
+   */
+  private String filename;
+
+  /** Path URI of the destination. */
+  private String uri = "";
+
+  /** ID of the upload. */
+  private String uploadId;
+
+  /** Destination bucket. */
+  private String bucket;
+
+  /** Destination key in the bucket. */
+  private String destinationKey;
+
+  /** When was the upload created? */
+  private long created;
+
+  /** When was the upload saved? */
+  private long saved;
+
+  /** timestamp as date; no expectation of parseability. */
+  private String date;
+
+  /** Job ID, if known. */
+  private String jobId = "";
+
+  /** Task ID, if known. */
+  private String taskId = "";
+
+  /** Arbitrary notes. */
+  private String text = "";
+
+  /** Ordered list of etags. */
+  private List<String> etags;
+
+  /**
+   * Any custom extra data committer subclasses may choose to add.
+   */
+  private Map<String, String> extraData = new HashMap<>(0);
+
+  /** Destination file size. */
+  private long length;
+
+  public SinglePendingCommit() {
+  }
+
+  /**
+   * Get a JSON serializer for this class.
+   * @return a serializer.
+   */
+  public static JsonSerialization<SinglePendingCommit> serializer() {
+    return new JsonSerialization<>(SinglePendingCommit.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
+   * @throws ValidationFailure if the data is invalid
+   */
+  public static SinglePendingCommit load(FileSystem fs, Path path)
+      throws IOException {
+    SinglePendingCommit instance = serializer().load(fs, path);
+    instance.filename = path.toString();
+    instance.validate();
+    return instance;
+  }
+
+  /**
+   * Deserialize via java Serialization API: deserialize the instance
+   * and then call {@link #validate()} to verify that the deserialized
+   * data is valid.
+   * @param inStream input stream
+   * @throws IOException IO problem
+   * @throws ClassNotFoundException reflection problems
+   * @throws ValidationFailure validation failure
+   */
+  private void readObject(ObjectInputStream inStream) throws IOException,
+      ClassNotFoundException {
+    inStream.defaultReadObject();
+    validate();
+  }
+
+  /**
+   * Set the various timestamp fields to the supplied value.
+   * @param millis time in milliseconds
+   */
+  public void touch(long millis) {
+    created = millis;
+    saved = millis;
+    date = new Date(millis).toString();
+  }
+
+  /**
+   * Set the commit data.
+   * @param parts ordered list of etags.
+   * @throws ValidationFailure if the data is invalid
+   */
+  public void bindCommitData(List<PartETag> parts) throws ValidationFailure {
+    etags = new ArrayList<>(parts.size());
+    int counter = 1;
+    for (PartETag part : parts) {
+      verify(part.getPartNumber() == counter,
+          "Expected part number %s but got %s", counter, part.getPartNumber());
+      etags.add(part.getETag());
+      counter++;
+    }
+  }
+
+  @Override
+  public void validate() throws ValidationFailure {
+    verify(version == VERSION, "Wrong version: %s", version);
+    verify(StringUtils.isNotEmpty(bucket), "Empty bucket");
+    verify(StringUtils.isNotEmpty(destinationKey),
+        "Empty destination");
+    verify(StringUtils.isNotEmpty(uploadId), "Empty uploadId");
+    verify(length >= 0, "Invalid length: " + length);
+    destinationPath();
+    verify(etags != null, "No etag list");
+    validateCollectionClass(etags, String.class);
+    for (String etag : etags) {
+      verify(StringUtils.isNotEmpty(etag), "Empty etag");
+    }
+    if (extraData != null) {
+      validateCollectionClass(extraData.keySet(), String.class);
+      validateCollectionClass(extraData.values(), String.class);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "DelayedCompleteData{");
+    sb.append("version=").append(version);
+    sb.append(", uri='").append(uri).append('\'');
+    sb.append(", destination='").append(destinationKey).append('\'');
+    sb.append(", uploadId='").append(uploadId).append('\'');
+    sb.append(", created=").append(created);
+    sb.append(", saved=").append(saved);
+    sb.append(", size=").append(length);
+    sb.append(", date='").append(date).append('\'');
+    sb.append(", jobId='").append(jobId).append('\'');
+    sb.append(", taskId='").append(taskId).append('\'');
+    sb.append(", notes='").append(text).append('\'');
+    if (etags != null) {
+      sb.append(", etags=[");
+      sb.append(join(",", etags));
+      sb.append(']');
+    } else {
+      sb.append(", etags=null");
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public byte[] toBytes() throws IOException {
+    validate();
+    return serializer().toBytes(this);
+  }
+
+  @Override
+  public void save(FileSystem fs, Path path, boolean overwrite)
+      throws IOException {
+    serializer().save(fs, path, this, overwrite);
+  }
+
+  /**
+   * Build the destination path of the object.
+   * @return the path
+   * @throws IllegalStateException if the URI is invalid
+   */
+  public Path destinationPath() {
+    Preconditions.checkState(StringUtils.isNotEmpty(uri), "Empty uri");
+    try {
+      return new Path(new URI(uri));
+    } catch (URISyntaxException e) {
+      throw new IllegalStateException("Cannot parse URI " + uri);
+    }
+  }
+
+  /**
+   * Get the number of etags.
+   * @return the size of the etag list.
+   */
+  public int getPartCount() {
+    return etags.size();
+  }
+
+  /**
+   * Iterate over the etags.
+   * @return an iterator.
+   */
+  @Override
+  public Iterator<String> iterator() {
+    return etags.iterator();
+  }
+
+  /** @return version marker. */
+  public int getVersion() {
+    return version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  /**
+   * This is the filename of the pending file itself.
+   * Used during processing; it's persistent value, if any, is ignored.
+   * @return filename
+   */
+  public String getFilename() {
+    return filename;
+  }
+
+  public void setFilename(String filename) {
+    this.filename = filename;
+  }
+
+  /** @return path URI of the destination. */
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  /** @return ID of the upload. */
+  public String getUploadId() {
+    return uploadId;
+  }
+
+  public void setUploadId(String uploadId) {
+    this.uploadId = uploadId;
+  }
+
+  /** @return destination bucket. */
+  public String getBucket() {
+    return bucket;
+  }
+
+  public void setBucket(String bucket) {
+    this.bucket = bucket;
+  }
+
+  /** @return destination key in the bucket. */
+  public String getDestinationKey() {
+    return destinationKey;
+  }
+
+  public void setDestinationKey(String destinationKey) {
+    this.destinationKey = destinationKey;
+  }
+
+  /**
+   * When was the upload created?
+   * @return timestamp
+   */
+  public long getCreated() {
+    return created;
+  }
+
+  public void setCreated(long created) {
+    this.created = created;
+  }
+
+  /**
+   * When was the upload saved?
+   * @return timestamp
+   */
+  public long getSaved() {
+    return saved;
+  }
+
+  public void setSaved(long saved) {
+    this.saved = saved;
+  }
+
+  /**
+   * Timestamp as date; no expectation of parseability.
+   * @return date string
+   */
+  public String getDate() {
+    return date;
+  }
+
+  public void setDate(String date) {
+    this.date = date;
+  }
+
+  /** @return Job ID, if known. */
+  public String getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(String jobId) {
+    this.jobId = jobId;
+  }
+
+  /** @return Task ID, if known. */
+  public String getTaskId() {
+    return taskId;
+  }
+
+  public void setTaskId(String taskId) {
+    this.taskId = taskId;
+  }
+
+  /**
+   * Arbitrary notes.
+   * @return any notes
+   */
+  public String getText() {
+    return text;
+  }
+
+  public void setText(String text) {
+    this.text = text;
+  }
+
+  /** @return ordered list of etags. */
+  public List<String> getEtags() {
+    return etags;
+  }
+
+  public void setEtags(List<String> etags) {
+    this.etags = etags;
+  }
+
+  /**
+   * Any custom extra data committer subclasses may choose to add.
+   * @return custom data
+   */
+  public Map<String, String> getExtraData() {
+    return extraData;
+  }
+
+  public void setExtraData(Map<String, String> extraData) {
+    this.extraData = extraData;
+  }
+
+  /**
+   * Destination file size.
+   * @return size of destination object
+   */
+  public long getLength() {
+    return length;
+  }
+
+  public void setLength(long length) {
+    this.length = length;
+  }
+}

+ 322 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java

@@ -0,0 +1,322 @@
+/*
+ * 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.fs.s3a.commit.files;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+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.s3a.commit.ValidationFailure;
+import org.apache.hadoop.util.JsonSerialization;
+
+/**
+ * Summary data saved into a {@code _SUCCESS} marker file.
+ *
+ * This provides an easy way to determine which committer was used
+ * to commit work.
+ * <ol>
+ *   <li>File length == 0: classic {@code FileOutputCommitter}.</li>
+ *   <li>Loadable as {@link SuccessData}:
+ *   A s3guard committer with name in in {@link #committer} field.</li>
+ *   <li>Not loadable? Something else.</li>
+ * </ol>
+ *
+ * This is an unstable structure intended for diagnostics and testing.
+ * Applications reading this data should use/check the {@link #name} field
+ * to differentiate from any other JSON-based manifest and to identify
+ * changes in the output format.
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SuccessData extends PersistentCommitData {
+  private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class);
+
+  /**
+   * Serialization ID: {@value}.
+   */
+  private static final long serialVersionUID = 507133045258460084L;
+
+  /**
+   * 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/1";
+
+  /**
+   * Name of file; includes version marker.
+   */
+  private String name;
+
+  /** Timestamp of creation. */
+  private long timestamp;
+
+  /** 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;
+
+  /**
+   * Metrics.
+   */
+  private Map<String, Long> metrics = new HashMap<>();
+
+  /**
+   * Diagnostics information.
+   */
+  private Map<String, String> diagnostics = new HashMap<>();
+
+  /**
+   * Filenames in the commit.
+   */
+  private List<String> filenames = new ArrayList<>(0);
+
+  @Override
+  public void validate() throws ValidationFailure {
+    ValidationFailure.verify(name != null,
+        "Incompatible file format: no 'name' field");
+    ValidationFailure.verify(NAME.equals(name),
+        "Incompatible file format: " + name);
+  }
+
+  @Override
+  public byte[] toBytes() throws IOException {
+    return serializer().toBytes(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(
+        "SuccessData{");
+    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
+   * @throws ValidationFailure if the data is invalid
+   */
+  public static SuccessData load(FileSystem fs, Path path)
+      throws IOException {
+    LOG.debug("Reading success data from {}", path);
+    SuccessData instance = serializer().load(fs, path);
+    instance.validate();
+    return instance;
+  }
+
+  /**
+   * Get a JSON serializer for this class.
+   * @return a serializer.
+   */
+  private static JsonSerialization<SuccessData> serializer() {
+    return new JsonSerialization<>(SuccessData.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(Map<String, Long> metrics) {
+    this.metrics = metrics;
+  }
+
+  /**
+   * @return a list of filenames in the commit.
+   */
+  public List<String> getFilenames() {
+    return filenames;
+  }
+
+  public void setFilenames(List<String> filenames) {
+    this.filenames = filenames;
+  }
+
+  public Map<String, String> getDiagnostics() {
+    return diagnostics;
+  }
+
+  public void setDiagnostics(Map<String, String> diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  /**
+   * Add a diagnostics entry.
+   * @param key name
+   * @param value value
+   */
+  public void addDiagnostic(String key, String value) {
+    diagnostics.put(key, value);
+  }
+}

+ 45 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java

@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+/**
+ * Persistent data formats for the committers.
+ *
+ * All of these formats share a base class of
+ * {@link org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData};
+ * the subclasses record
+ * <ol>
+ *   <li>The content of a single pending commit
+ *   (used by the Magic committer).</li>
+ *   <li>The list of all the files uploaded by a staging committer.</li>
+ *   <li>The summary information saved in the {@code _SUCCESS} file.</li>
+ * </ol>
+ *
+ * There are no guarantees of stability between versions; these are internal
+ * structures.
+ *
+ * The {@link org.apache.hadoop.fs.s3a.commit.files.SuccessData} file is
+ * the one visible to callers after a job completes; it is an unstable
+ * manifest intended for testing only.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.commit.files;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 161 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java

@@ -0,0 +1,161 @@
+/*
+ * 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.fs.s3a.commit.magic;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+import org.apache.hadoop.fs.s3a.commit.PutTracker;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+
+/**
+ * Put tracker for Magic commits.
+ * <p>Important</p>: must not directly or indirectly import a class which
+ * uses any datatype in hadoop-mapreduce.
+ */
+@InterfaceAudience.Private
+public class MagicCommitTracker extends PutTracker {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      MagicCommitTracker.class);
+
+  private final String originalDestKey;
+  private final String pendingPartKey;
+  private final Path path;
+  private final WriteOperationHelper writer;
+  private final String bucket;
+  private static final byte[] EMPTY = new byte[0];
+
+  /**
+   * Magic commit tracker.
+   * @param path path nominally being written to
+   * @param bucket dest bucket
+   * @param originalDestKey the original key, in the magic directory.
+   * @param destKey key for the destination
+   * @param pendingsetKey key of the pendingset file
+   * @param writer writer instance to use for operations
+   */
+  public MagicCommitTracker(Path path,
+      String bucket,
+      String originalDestKey,
+      String destKey,
+      String pendingsetKey,
+      WriteOperationHelper writer) {
+    super(destKey);
+    this.bucket = bucket;
+    this.path = path;
+    this.originalDestKey = originalDestKey;
+    this.pendingPartKey = pendingsetKey;
+    this.writer = writer;
+    LOG.info("File {} is written as magic file to path {}",
+        path, destKey);
+  }
+
+  /**
+   * Initialize the tracker.
+   * @return true, indicating that the multipart commit must start.
+   * @throws IOException any IO problem.
+   */
+  @Override
+  public boolean initialize() throws IOException {
+    return true;
+  }
+
+  /**
+   * Flag to indicate that output is not visible after the stream
+   * is closed.
+   * @return true
+   */
+  @Override
+  public boolean outputImmediatelyVisible() {
+    return false;
+  }
+
+  /**
+   * Complete operation: generate the final commit data, put it.
+   * @param uploadId Upload ID
+   * @param parts list of parts
+   * @param bytesWritten bytes written
+   * @return false, indicating that the commit must fail.
+   * @throws IOException any IO problem.
+   * @throws IllegalArgumentException bad argument
+   */
+  @Override
+  public boolean aboutToComplete(String uploadId,
+      List<PartETag> parts,
+      long bytesWritten)
+      throws IOException {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
+        "empty/null upload ID: "+ uploadId);
+    Preconditions.checkArgument(parts != null,
+        "No uploaded parts list");
+    Preconditions.checkArgument(!parts.isEmpty(),
+        "No uploaded parts to save");
+    SinglePendingCommit commitData = new SinglePendingCommit();
+    commitData.touch(System.currentTimeMillis());
+    commitData.setDestinationKey(getDestKey());
+    commitData.setBucket(bucket);
+    commitData.setUri(path.toUri().toString());
+    commitData.setUploadId(uploadId);
+    commitData.setText("");
+    commitData.setLength(bytesWritten);
+    commitData.bindCommitData(parts);
+    byte[] bytes = commitData.toBytes();
+    LOG.info("Uncommitted data pending to file {};"
+            + " commit metadata for {} parts in {}. sixe: {} byte(s)",
+        path.toUri(), parts.size(), pendingPartKey, bytesWritten);
+    LOG.debug("Closed MPU to {}, saved commit information to {}; data=:\n{}",
+        path, pendingPartKey, commitData);
+    PutObjectRequest put = writer.createPutObjectRequest(
+        pendingPartKey,
+        new ByteArrayInputStream(bytes),
+        bytes.length);
+    writer.uploadObject(put);
+
+    // now put a 0-byte file with the name of the original under-magic path
+    PutObjectRequest originalDestPut = writer.createPutObjectRequest(
+        originalDestKey,
+        new ByteArrayInputStream(EMPTY),
+        0);
+    writer.uploadObject(originalDestPut);
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "MagicCommitTracker{");
+    sb.append(", destKey=").append(getDestKey());
+    sb.append(", pendingPartKey='").append(pendingPartKey).append('\'');
+    sb.append(", path=").append(path);
+    sb.append(", writer=").append(writer);
+    sb.append('}');
+    return sb.toString();
+  }
+}

+ 288 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java

@@ -0,0 +1,288 @@
+/*
+ * 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.fs.s3a.commit.magic;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter;
+import org.apache.hadoop.fs.s3a.commit.CommitOperations;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
+import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
+
+/**
+ * This is a dedicated committer which requires the "magic" directory feature
+ * of the S3A Filesystem to be enabled; it then uses paths for task and job
+ * attempts in magic paths, so as to ensure that the final output goes direct
+ * to the destination directory.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class MagicS3GuardCommitter extends AbstractS3ACommitter {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MagicS3GuardCommitter.class);
+
+  /** Name: {@value}. */
+  public static final String NAME = CommitConstants.COMMITTER_NAME_MAGIC;
+
+  /**
+   * Create a task committer.
+   * @param outputPath the job's output path
+   * @param context the task's context
+   * @throws IOException on a failure
+   */
+  public MagicS3GuardCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    super(outputPath, context);
+    setWorkPath(getTaskAttemptPath(context));
+    verifyIsMagicCommitPath(getDestS3AFS(), getWorkPath());
+    LOG.debug("Task attempt {} has work path {}",
+        context.getTaskAttemptID(),
+        getWorkPath());
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  /**
+   * Require magic paths in the FS client.
+   * @return true, always.
+   */
+  @Override
+  protected boolean requiresDelayedCommitOutputInFileSystem() {
+    return true;
+  }
+
+  @Override
+  public void setupJob(JobContext context) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Setup Job %s", jobIdString(context))) {
+      Path jobAttemptPath = getJobAttemptPath(context);
+      getDestinationFS(jobAttemptPath,
+          context.getConfiguration()).mkdirs(jobAttemptPath);
+    }
+  }
+
+  /**
+   * Get the list of pending uploads for this job attempt, by listing
+   * all .pendingset files in the job attempt directory.
+   * @param context job context
+   * @return a list of pending commits.
+   * @throws IOException Any IO failure
+   */
+  protected List<SinglePendingCommit> listPendingUploadsToCommit(
+      JobContext context)
+      throws IOException {
+    FileSystem fs = getDestFS();
+    return loadPendingsetFiles(context, false, fs,
+        listAndFilter(fs, getJobAttemptPath(context), false,
+            CommitOperations.PENDINGSET_FILTER));
+  }
+
+  /**
+   * Delete the magic directory.
+   */
+  public void cleanupStagingDirs() {
+    Path path = magicSubdir(getOutputPath());
+    Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(),
+        () -> deleteWithWarning(getDestFS(), path, true));
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext context) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Setup Task %s", context.getTaskAttemptID())) {
+      Path taskAttemptPath = getTaskAttemptPath(context);
+      FileSystem fs = taskAttemptPath.getFileSystem(getConf());
+      fs.mkdirs(taskAttemptPath);
+    }
+  }
+
+  /**
+   * Did this task write any files in the work directory?
+   * Probes for a task existing by looking to see if the attempt dir exists.
+   * This adds more HTTP requests to the call. It may be better just to
+   * return true and rely on the commit task doing the work.
+   * @param context the task's context
+   * @return true if the attempt path exists
+   * @throws IOException failure to list the path
+   */
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context)
+      throws IOException {
+    Path taskAttemptPath = getTaskAttemptPath(context);
+    try (DurationInfo d = new DurationInfo(LOG,
+        "needsTaskCommit task %s", context.getTaskAttemptID())) {
+      return taskAttemptPath.getFileSystem(
+          context.getConfiguration())
+          .exists(taskAttemptPath);
+    }
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Commit task %s", context.getTaskAttemptID())) {
+      PendingSet commits = innerCommitTask(context);
+      LOG.info("Task {} committed {} files", context.getTaskAttemptID(),
+          commits.size());
+    } catch (IOException e) {
+      getCommitOperations().taskCompleted(false);
+      throw e;
+    } finally {
+      // delete the task attempt so there's no possibility of a second attempt
+      deleteTaskAttemptPathQuietly(context);
+    }
+    getCommitOperations().taskCompleted(true);
+  }
+
+  /**
+   * Inner routine for committing a task.
+   * The list of pending commits is loaded and then saved to the job attempt
+   * dir.
+   * Failure to load any file or save the final file triggers an abort of
+   * all known pending commits.
+   * @param context context
+   * @return the summary file
+   * @throws IOException exception
+   */
+  private PendingSet innerCommitTask(
+      TaskAttemptContext context) throws IOException {
+    Path taskAttemptPath = getTaskAttemptPath(context);
+    // load in all pending commits.
+    CommitOperations actions = getCommitOperations();
+    Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
+        loaded = actions.loadSinglePendingCommits(
+            taskAttemptPath, true);
+    PendingSet pendingSet = loaded.getKey();
+    List<Pair<LocatedFileStatus, IOException>> failures = loaded.getValue();
+    if (!failures.isEmpty()) {
+      // At least one file failed to load
+      // revert all which did; report failure with first exception
+      LOG.error("At least one commit file could not be read: failing");
+      abortPendingUploads(context, pendingSet.getCommits(), true);
+      throw failures.get(0).getValue();
+    }
+    // patch in IDs
+    String jobId = String.valueOf(context.getJobID());
+    String taskId = String.valueOf(context.getTaskAttemptID());
+    for (SinglePendingCommit commit : pendingSet.getCommits()) {
+      commit.setJobId(jobId);
+      commit.setTaskId(taskId);
+    }
+
+    Path jobAttemptPath = getJobAttemptPath(context);
+    TaskAttemptID taskAttemptID = context.getTaskAttemptID();
+    Path taskOutcomePath = new Path(jobAttemptPath,
+        taskAttemptID.getTaskID().toString() +
+        CommitConstants.PENDINGSET_SUFFIX);
+    LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath);
+    try {
+      pendingSet.save(getDestFS(), taskOutcomePath, false);
+    } catch (IOException e) {
+      LOG.warn("Failed to save task commit data to {} ",
+          taskOutcomePath, e);
+      abortPendingUploads(context, pendingSet.getCommits(), true);
+      throw e;
+    }
+    return pendingSet;
+  }
+
+  /**
+   * Abort a task. Attempt load then abort all pending files,
+   * then try to delete the task attempt path.
+   * This method may be called on the job committer, rather than the
+   * task one (such as in the MapReduce AM after a task container failure).
+   * It must extract all paths and state from the passed in context.
+   * @param context task context
+   * @throws IOException if there was some problem querying the path other
+   * than it not actually existing.
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    Path attemptPath = getTaskAttemptPath(context);
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Abort task %s", context.getTaskAttemptID())) {
+      getCommitOperations().abortAllSinglePendingCommits(attemptPath, true);
+    } finally {
+      deleteQuietly(
+          attemptPath.getFileSystem(context.getConfiguration()),
+          attemptPath, true);
+    }
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param appAttemptId the ID of the application attempt for this job.
+   * @return the path to store job attempt data.
+   */
+  protected Path getJobAttemptPath(int appAttemptId) {
+    return getMagicJobAttemptPath(appAttemptId, getOutputPath());
+  }
+
+  /**
+   * 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.
+   */
+  public Path getTaskAttemptPath(TaskAttemptContext context) {
+    return getMagicTaskAttemptPath(context, getOutputPath());
+  }
+
+  @Override
+  protected Path getBaseTaskAttemptPath(TaskAttemptContext context) {
+    return getBaseMagicTaskAttemptPath(context, getOutputPath());
+  }
+
+  /**
+   * Get a temporary directory for data. When a task is aborted/cleaned
+   * up, the contents of this directory are all deleted.
+   * @param context task context
+   * @return a path for temporary data.
+   */
+  public Path getTempTaskAttemptPath(TaskAttemptContext context) {
+    return CommitUtilsWithMR.getTempTaskAttemptPath(context, getOutputPath());
+  }
+
+}

+ 47 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitterFactory.java

@@ -0,0 +1,47 @@
+/*
+ * 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.fs.s3a.commit.magic;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+
+/**
+ * Factory for the Magic committer.
+ */
+public class MagicS3GuardCommitterFactory
+    extends AbstractS3ACommitterFactory {
+
+  /**
+   * Name of this class: {@value}.
+   */
+  public static final String CLASSNAME
+      = "org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitterFactory";
+
+  public PathOutputCommitter createTaskCommitter(S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return new MagicS3GuardCommitter(outputPath, context);
+  }
+
+}

+ 27 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/package-info.java

@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * This is the "Magic" committer and support.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.commit.magic;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 28 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/package-info.java

@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * Support for committing the output of analytics jobs directly to S3.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.commit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 33 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/ConflictResolution.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.fs.s3a.commit.staging;
+
+/**
+ * Enum of conflict resolution algorithms.
+ */
+public enum ConflictResolution {
+  /** Fail. */
+  FAIL,
+
+  /** Merge new data with existing data. */
+  APPEND,
+
+  /** Overwrite existing data. */
+  REPLACE
+}

+ 116 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java

@@ -0,0 +1,116 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathExistsException;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
+
+/**
+ * This commits to a directory.
+ * The conflict policy is
+ * <ul>
+ *   <li>FAIL: fail the commit</li>
+ *   <li>APPEND: add extra data to the destination.</li>
+ *   <li>REPLACE: delete the destination directory in the job commit
+ *   (i.e. after and only if all tasks have succeeded.</li>
+ * </ul>
+ */
+public class DirectoryStagingCommitter extends StagingCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      DirectoryStagingCommitter.class);
+
+  /** Name: {@value}. */
+  public static final String NAME = COMMITTER_NAME_DIRECTORY;
+
+  public DirectoryStagingCommitter(Path outputPath, TaskAttemptContext context)
+      throws IOException {
+    super(outputPath, context);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public void setupJob(JobContext context) throws IOException {
+    super.setupJob(context);
+    Path outputPath = getOutputPath();
+    FileSystem fs = getDestFS();
+    if (getConflictResolutionMode(context, fs.getConf())
+        == ConflictResolution.FAIL
+        && fs.exists(outputPath)) {
+      LOG.debug("Failing commit by task attempt {} to write"
+              + " to existing output path {}",
+          context.getJobID(), getOutputPath());
+      throw new PathExistsException(outputPath.toString(), E_DEST_EXISTS);
+    }
+  }
+
+  /**
+   * Pre-commit actions for a job.
+   * Here: look at the conflict resolution mode and choose
+   * an action based on the current policy.
+   * @param context job context
+   * @param pending pending commits
+   * @throws IOException any failure
+   */
+  @Override
+  protected void preCommitJob(JobContext context,
+      List<SinglePendingCommit> pending) throws IOException {
+    Path outputPath = getOutputPath();
+    FileSystem fs = getDestFS();
+    Configuration fsConf = fs.getConf();
+    switch (getConflictResolutionMode(context, fsConf)) {
+    case FAIL:
+      // this was checked in setupJob, but this avoids some cases where
+      // output was created while the job was processing
+      if (fs.exists(outputPath)) {
+        throw new PathExistsException(outputPath.toString(), E_DEST_EXISTS);
+      }
+      break;
+    case APPEND:
+      // do nothing
+      break;
+    case REPLACE:
+      if (fs.delete(outputPath, true /* recursive */)) {
+        LOG.info("{}: removed output path to be replaced: {}",
+            getRole(), outputPath);
+      }
+      break;
+    default:
+      throw new IOException(getRole() + ": unknown conflict resolution mode: "
+          + getConflictResolutionMode(context, fsConf));
+    }
+  }
+}

+ 48 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitterFactory.java

@@ -0,0 +1,48 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+
+/**
+ * Factory for the Directory committer.
+ */
+public class DirectoryStagingCommitterFactory
+    extends AbstractS3ACommitterFactory {
+
+  /**
+   * Name of this class: {@value}.
+   */
+  public static final String CLASSNAME
+      = "org.apache.hadoop.fs.s3a.commit.staging"
+      + ".DirectoryStagingCommitterFactory";
+
+  public PathOutputCommitter createTaskCommitter(S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return new DirectoryStagingCommitter(outputPath, context);
+  }
+
+}

+ 159 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java

@@ -0,0 +1,159 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Sets;
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathExistsException;
+import org.apache.hadoop.fs.s3a.commit.PathCommitException;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
+
+/**
+ * Partitioned committer.
+ * This writes data to specific "partition" subdirectories, applying
+ * conflict resolution on a partition-by-partition basis. The existence
+ * and state of any parallel partitions for which there is no are output
+ * files are not considered in the conflict resolution.
+ *
+ * The conflict policy is
+ * <ul>
+ *   <li>FAIL: fail the commit if any of the partitions have data.</li>
+ *   <li>APPEND: add extra data to the destination partitions.</li>
+ *   <li>REPLACE: delete the destination partition in the job commit
+ *   (i.e. after and only if all tasks have succeeded.</li>
+ * </ul>
+ */
+public class PartitionedStagingCommitter extends StagingCommitter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      PartitionedStagingCommitter.class);
+
+  /** Name: {@value}. */
+  public static final String NAME = COMMITTER_NAME_PARTITIONED;
+
+  public PartitionedStagingCommitter(Path outputPath,
+      TaskAttemptContext context)
+      throws IOException {
+    super(outputPath, context);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "PartitionedStagingCommitter{");
+    sb.append(super.toString());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  protected int commitTaskInternal(TaskAttemptContext context,
+      List<? extends FileStatus> taskOutput) throws IOException {
+    Path attemptPath = getTaskAttemptPath(context);
+    Set<String> partitions = Paths.getPartitions(attemptPath, taskOutput);
+
+    // enforce conflict resolution, but only if the mode is FAIL. for APPEND,
+    // it doesn't matter that the partitions are already there, and for REPLACE,
+    // deletion should be done during job commit.
+    FileSystem fs = getDestFS();
+    if (getConflictResolutionMode(context, fs.getConf())
+        == ConflictResolution.FAIL) {
+      for (String partition : partitions) {
+        // getFinalPath adds the UUID to the file name. this needs the parent.
+        Path partitionPath = getFinalPath(partition + "/file",
+            context).getParent();
+        if (fs.exists(partitionPath)) {
+          LOG.debug("Failing commit by task attempt {} to write"
+              + " to existing path {} under {}",
+              context.getTaskAttemptID(), partitionPath, getOutputPath());
+          throw new PathExistsException(partitionPath.toString(),
+              E_DEST_EXISTS);
+        }
+      }
+    }
+    return super.commitTaskInternal(context, taskOutput);
+  }
+
+  /**
+   * Job-side conflict resolution.
+   * The partition path conflict resolution actions are:
+   * <ol>
+   *   <li>FAIL: assume checking has taken place earlier; no more checks.</li>
+   *   <li>APPEND: allowed.; no need to check.</li>
+   *   <li>REPLACE deletes all existing partitions.</li>
+   * </ol>
+   * @param context job context
+   * @param pending the pending operations
+   * @throws IOException any failure
+   */
+  @Override
+  protected void preCommitJob(JobContext context,
+      List<SinglePendingCommit> pending) throws IOException {
+
+    FileSystem fs = getDestFS();
+
+    // enforce conflict resolution
+    Configuration fsConf = fs.getConf();
+    switch (getConflictResolutionMode(context, fsConf)) {
+    case FAIL:
+      // FAIL checking is done on the task side, so this does nothing
+      break;
+    case APPEND:
+      // no check is needed because the output may exist for appending
+      break;
+    case REPLACE:
+      Set<Path> partitions = pending.stream()
+          .map(SinglePendingCommit::destinationPath)
+          .map(Path::getParent)
+          .collect(Collectors.toCollection(Sets::newLinkedHashSet));
+      for (Path partitionPath : partitions) {
+        LOG.debug("{}: removing partition path to be replaced: " +
+            getRole(), partitionPath);
+        fs.delete(partitionPath, true);
+      }
+      break;
+    default:
+      throw new PathCommitException("",
+          getRole() + ": unknown conflict resolution mode: "
+          + getConflictResolutionMode(context, fsConf));
+    }
+  }
+
+}

+ 48 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitterFactory.java

@@ -0,0 +1,48 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+
+/**
+ * Factory for the {@link PartitionedStagingCommitter}.
+ */
+public class PartitionedStagingCommitterFactory
+    extends AbstractS3ACommitterFactory {
+
+  /**
+   * Name of this class: {@value}.
+   */
+  public static final String CLASSNAME
+      = "org.apache.hadoop.fs.s3a.commit.staging"
+      + ".PartitionedStagingCommitterFactory";
+
+  public PathOutputCommitter createTaskCommitter(S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return new PartitionedStagingCommitter(outputPath, context);
+  }
+
+}

+ 300 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java

@@ -0,0 +1,300 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsDirectoryException;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.*;
+
+/**
+ * Path operations for the staging committers.
+ */
+public final class Paths {
+
+  private Paths() {
+  }
+
+  /**
+   * Insert the UUID to a path if it is not there already.
+   * If there is a trailing "." in the prefix after the last slash, the
+   * UUID is inserted before it with a "-" prefix; otherwise appended.
+   *
+   * Examples:
+   * <pre>
+   *   /example/part-0000  ==&gt; /example/part-0000-0ab34
+   *   /example/part-0001.gz.csv  ==&gt; /example/part-0001-0ab34.gz.csv
+   *   /example/part-0002-0abc3.gz.csv  ==&gt; /example/part-0002-0abc3.gz.csv
+   *   /example0abc3/part-0002.gz.csv  ==&gt; /example0abc3/part-0002.gz.csv
+   * </pre>
+   *
+   *
+   * @param pathStr path as a string; must not have a trailing "/".
+   * @param uuid UUID to append; must not be empty
+   * @return new path.
+   */
+  public static String addUUID(String pathStr, String uuid) {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(pathStr), "empty path");
+    Preconditions.checkArgument(StringUtils.isNotEmpty(uuid), "empty uuid");
+    // In some cases, Spark will add the UUID to the filename itself.
+    if (pathStr.contains(uuid)) {
+      return pathStr;
+    }
+
+    int dot; // location of the first '.' in the file name
+    int lastSlash = pathStr.lastIndexOf('/');
+    if (lastSlash >= 0) {
+      Preconditions.checkState(lastSlash + 1 < pathStr.length(),
+          "Bad path: " + pathStr);
+      dot = pathStr.indexOf('.', lastSlash);
+    } else {
+      dot = pathStr.indexOf('.');
+    }
+
+    if (dot >= 0) {
+      return pathStr.substring(0, dot) + "-" + uuid + pathStr.substring(dot);
+    } else {
+      return pathStr + "-" + uuid;
+    }
+  }
+
+  /**
+   * Get the parent path of a string path: everything up to but excluding
+   * the last "/" in the path.
+   * @param pathStr path as a string
+   * @return the parent or null if there is no parent.
+   */
+  public static String getParent(String pathStr) {
+    int lastSlash = pathStr.lastIndexOf('/');
+    if (lastSlash >= 0) {
+      return pathStr.substring(0, lastSlash);
+    }
+    return null;
+  }
+
+  /**
+   * Using {@code URI#relativize()}, build the relative path from the
+   * base path to the full path.
+   * If {@code childPath} is not a child of {@code basePath} the outcome
+   * os undefined.
+   * @param basePath base path
+   * @param fullPath full path under the base path.
+   * @return the relative path
+   */
+  public static String getRelativePath(Path basePath,
+      Path fullPath) {
+    return basePath.toUri().relativize(fullPath.toUri()).getPath();
+  }
+
+  /**
+   * Varags constructor of paths. Not very efficient.
+   * @param parent parent path
+   * @param child child entries. "" elements are skipped.
+   * @return the full child path.
+   */
+  public static Path path(Path parent, String... child) {
+    Path p = parent;
+    for (String c : child) {
+      if (!c.isEmpty()) {
+        p = new Path(p, c);
+      }
+    }
+    return p;
+  }
+
+  /**
+   * A cache of temporary folders. There's a risk here that the cache
+   * gets too big
+   */
+  private static Cache<TaskAttemptID, Path> tempFolders = CacheBuilder
+      .newBuilder().build();
+
+  /**
+   * Get the task attempt temporary directory in the local filesystem.
+   * @param conf configuration
+   * @param uuid some UUID, such as a job UUID
+   * @param attemptID attempt ID
+   * @return a local task attempt directory.
+   * @throws IOException IO problem.
+   */
+  public static Path getLocalTaskAttemptTempDir(final Configuration conf,
+      final String uuid,
+      final TaskAttemptID attemptID)
+      throws IOException {
+    try {
+      final LocalDirAllocator allocator =
+          new LocalDirAllocator(Constants.BUFFER_DIR);
+      return tempFolders.get(attemptID,
+          () -> {
+            return FileSystem.getLocal(conf).makeQualified(
+                allocator.getLocalPathForWrite(uuid, conf));
+          });
+    } catch (ExecutionException e) {
+      throw new RuntimeException(e.getCause());
+    } catch (UncheckedExecutionException e) {
+      if (e.getCause() instanceof RuntimeException) {
+        throw (RuntimeException) e.getCause();
+      }
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Remove all information held about task attempts.
+   * @param attemptID attempt ID.
+   */
+  public static void clearTempFolderInfo(final TaskAttemptID attemptID) {
+    tempFolders.invalidate(attemptID);
+  }
+
+  /**
+   * Reset the temp folder cache; useful in tests.
+   */
+  @VisibleForTesting
+  public static void resetTempFolderCache() {
+    tempFolders.invalidateAll();
+  }
+
+  /**
+   * Try to come up with a good temp directory for different filesystems.
+   * @param fs filesystem
+   * @param conf configuration
+   * @return a qualified path under which temporary work can go.
+   */
+  public static Path tempDirForStaging(FileSystem fs,
+      Configuration conf) {
+    String fallbackPath = fs.getScheme().equals("file")
+        ? System.getProperty(JAVA_IO_TMPDIR)
+        : FILESYSTEM_TEMP_PATH;
+
+    return fs.makeQualified(new Path(conf.getTrimmed(
+        FS_S3A_COMMITTER_STAGING_TMP_PATH, fallbackPath)));
+  }
+
+  /**
+   * Get the Application Attempt ID for this job.
+   * @param conf the config to look in
+   * @return the Application Attempt ID for a given job.
+   */
+  private static int getAppAttemptId(Configuration conf) {
+    return conf.getInt(
+        MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
+  }
+
+  /**
+   * Build a qualified temporary path for the multipart upload commit
+   * information in the cluster filesystem.
+   * Path is built by
+   * {@link #getMultipartUploadCommitsDirectory(FileSystem, Configuration, String)}
+   * @param conf configuration defining default FS.
+   * @param uuid uuid of job
+   * @return a path which can be used for temporary work
+   * @throws IOException on an IO failure.
+   */
+  public static Path getMultipartUploadCommitsDirectory(Configuration conf,
+      String uuid) throws IOException {
+    return getMultipartUploadCommitsDirectory(FileSystem.get(conf), conf, uuid);
+  }
+
+  /**
+   * Build a qualified temporary path for the multipart upload commit
+   * information in the supplied filesystem
+   * (which is expected to be the cluster FS).
+   * Currently {code $tempDir/$user/$uuid/staging-uploads} where
+   * {@code tempDir} is from
+   * {@link #tempDirForStaging(FileSystem, Configuration)}.
+   * @param fs target FS
+   * @param conf configuration
+   * @param uuid uuid of job
+   * @return a path which can be used for temporary work
+   * @throws IOException on an IO failure.
+   */
+  @VisibleForTesting
+  static Path getMultipartUploadCommitsDirectory(FileSystem fs,
+      Configuration conf, String uuid) throws IOException {
+    return path(
+        tempDirForStaging(fs, conf),
+        UserGroupInformation.getCurrentUser().getShortUserName(),
+        uuid,
+        STAGING_UPLOADS);
+  }
+
+  /**
+   * Returns the partition of a relative file path, or null if the path is a
+   * file name with no relative directory.
+   *
+   * @param relative a relative file path
+   * @return the partition of the relative file path
+   */
+  protected static String getPartition(String relative) {
+    return getParent(relative);
+  }
+
+  /**
+   * Get the set of partitions from the list of files being staged.
+   * This is all immediate parents of those files. If a file is in the root
+   * dir, the partition is declared to be
+   * {@link StagingCommitterConstants#TABLE_ROOT}.
+   * @param attemptPath path for the attempt
+   * @param taskOutput list of output files.
+   * @return list of partitions.
+   * @throws IOException IO failure
+   */
+  public static Set<String> getPartitions(Path attemptPath,
+      List<? extends FileStatus> taskOutput)
+      throws IOException {
+    // get a list of partition directories
+    Set<String> partitions = Sets.newLinkedHashSet();
+    for (FileStatus fileStatus : taskOutput) {
+      // sanity check the output paths
+      Path outputFile = fileStatus.getPath();
+      if (!fileStatus.isFile()) {
+        throw new PathIsDirectoryException(outputFile.toString());
+      }
+      String partition = getPartition(
+          getRelativePath(attemptPath, outputFile));
+      partitions.add(partition != null ? partition : TABLE_ROOT);
+    }
+
+    return partitions;
+  }
+
+}

+ 851 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java

@@ -0,0 +1,851 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import com.google.common.base.Preconditions;
+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.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants;
+import org.apache.hadoop.fs.s3a.commit.Tasks;
+import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
+import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+
+import static com.google.common.base.Preconditions.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Invoker.*;
+import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
+
+/**
+ * Committer based on the contributed work of the
+ * <a href="https://github.com/rdblue/s3committer">Netflix multipart committers.</a>
+ * <ol>
+ *   <li>
+ *   The working directory of each task is actually under a temporary
+ *   path in the local filesystem; jobs write directly into it.
+ *   </li>
+ *   <li>
+ *     Task Commit: list all files under the task working dir, upload
+ *     each of them but do not commit the final operation.
+ *     Persist the information for each pending commit into the cluster
+ *     for enumeration and commit by the job committer.
+ *   </li>
+ *   <li>Task Abort: recursive delete of task working dir.</li>
+ *   <li>Job Commit: list all pending PUTs to commit; commit them.</li>
+ *   <li>
+ *     Job Abort: list all pending PUTs to commit; abort them.
+ *     Delete all task attempt directories.
+ *   </li>
+ * </ol>
+ *
+ * This is the base class of the Partitioned and Directory committers.
+ * It does not do any conflict resolution, and is made non-abstract
+ * primarily for test purposes. It is not expected to be used in production.
+ */
+public class StagingCommitter extends AbstractS3ACommitter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      StagingCommitter.class);
+
+  /** Name: {@value}. */
+  public static final String NAME = "staging";
+  private final Path constructorOutputPath;
+  private final long uploadPartSize;
+  private final String uuid;
+  private final boolean uniqueFilenames;
+  private final FileOutputCommitter wrappedCommitter;
+
+  private ConflictResolution conflictResolution;
+  private String s3KeyPrefix;
+
+  /** The directory in the cluster FS for commits to go to. */
+  private Path commitsDirectory;
+
+  /**
+   * Committer for a single task attempt.
+   * @param outputPath final output path
+   * @param context task context
+   * @throws IOException on a failure
+   */
+  public StagingCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    super(outputPath, context);
+    this.constructorOutputPath = checkNotNull(getOutputPath(), "output path");
+    Configuration conf = getConf();
+    this.uploadPartSize = conf.getLongBytes(
+        MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
+    this.uuid = getUploadUUID(conf, context.getJobID());
+    this.uniqueFilenames = conf.getBoolean(
+        FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES,
+        DEFAULT_STAGING_COMMITTER_UNIQUE_FILENAMES);
+    setWorkPath(buildWorkPath(context, uuid));
+    this.wrappedCommitter = createWrappedCommitter(context, conf);
+    setOutputPath(constructorOutputPath);
+    Path finalOutputPath = getOutputPath();
+    Preconditions.checkNotNull(finalOutputPath, "Output path cannot be null");
+    S3AFileSystem fs = getS3AFileSystem(finalOutputPath,
+        context.getConfiguration(), false);
+    s3KeyPrefix = fs.pathToKey(finalOutputPath);
+    LOG.debug("{}: final output path is {}", getRole(), finalOutputPath);
+    // forces evaluation and caching of the resolution mode.
+    ConflictResolution mode = getConflictResolutionMode(getJobContext(),
+        fs.getConf());
+    LOG.debug("Conflict resolution mode: {}", mode);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  /**
+   * Create the wrapped committer.
+   * This includes customizing its options, and setting up the destination
+   * directory.
+   * @param context job/task context.
+   * @param conf config
+   * @return the inner committer
+   * @throws IOException on a failure
+   */
+  protected FileOutputCommitter createWrappedCommitter(JobContext context,
+      Configuration conf) throws IOException {
+
+    // explicitly choose commit algorithm
+    initFileOutputCommitterOptions(context);
+    commitsDirectory = Paths.getMultipartUploadCommitsDirectory(conf, uuid);
+    return new FileOutputCommitter(commitsDirectory, context);
+  }
+
+  /**
+   * Init the context config with everything needed for the file output
+   * committer. In particular, this code currently only works with
+   * commit algorithm 1.
+   * @param context context to configure.
+   */
+  protected void initFileOutputCommitterOptions(JobContext context) {
+    context.getConfiguration()
+        .setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, 1);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("StagingCommitter{");
+    sb.append(super.toString());
+    sb.append(", conflictResolution=").append(conflictResolution);
+    if (wrappedCommitter != null) {
+      sb.append(", wrappedCommitter=").append(wrappedCommitter);
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Get the UUID of an upload; may be the job ID.
+   * Spark will use a fake app ID based on the current minute and job ID 0.
+   * To avoid collisions, the key policy is:
+   * <ol>
+   *   <li>Value of {@link InternalCommitterConstants#FS_S3A_COMMITTER_STAGING_UUID}.</li>
+   *   <li>Value of {@code "spark.sql.sources.writeJobUUID"}.</li>
+   *   <li>Value of {@code "spark.app.id"}.</li>
+   *   <li>JobId passed in.</li>
+   * </ol>
+   * The staging UUID is set in in {@link #setupJob(JobContext)} and so will
+   * be valid in all sequences where the job has been set up for the
+   * configuration passed in.
+   * @param conf job/task configuration
+   * @param jobId Job ID
+   * @return an ID for use in paths.
+   */
+  public static String getUploadUUID(Configuration conf, String jobId) {
+    return conf.getTrimmed(
+        InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID,
+        conf.getTrimmed(SPARK_WRITE_UUID,
+            conf.getTrimmed(SPARK_APP_ID, jobId)));
+  }
+
+  /**
+   * Get the UUID of a Job.
+   * @param conf job/task configuration
+   * @param jobId Job ID
+   * @return an ID for use in paths.
+   */
+  public static String getUploadUUID(Configuration conf, JobID jobId) {
+    return getUploadUUID(conf, jobId.toString());
+  }
+
+  /**
+   * Get the work path for a task.
+   * @param context job/task complex
+   * @param uuid UUID
+   * @return a path or null if the context is not of a task
+   * @throws IOException failure to build the path
+   */
+  private static Path buildWorkPath(JobContext context, String uuid)
+      throws IOException {
+    if (context instanceof TaskAttemptContext) {
+      return taskAttemptWorkingPath((TaskAttemptContext) context, uuid);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Is this committer using unique filenames?
+   * @return true if unique filenames are used.
+   */
+  public Boolean useUniqueFilenames() {
+    return uniqueFilenames;
+  }
+
+  /**
+   * Get the filesystem for the job attempt.
+   * @param context the context of the job.  This is used to get the
+   * application attempt ID.
+   * @return the FS to store job attempt data.
+   * @throws IOException failure to create the FS.
+   */
+  public FileSystem getJobAttemptFileSystem(JobContext context)
+      throws IOException {
+    Path p = getJobAttemptPath(context);
+    return p.getFileSystem(context.getConfiguration());
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param context the context of the job.  This is used to get the
+   * application attempt ID.
+   * @param out the output path to place these in.
+   * @return the path to store job attempt data.
+   */
+  public static Path getJobAttemptPath(JobContext context, Path out) {
+    return getJobAttemptPath(getAppAttemptId(context), out);
+  }
+
+  /**
+   * Compute the path where the output of a given job attempt will be placed.
+   * @param appAttemptId the ID of the application attempt for this job.
+   * @return the path to store job attempt data.
+   */
+  private static Path getJobAttemptPath(int appAttemptId, Path out) {
+    return new Path(getPendingJobAttemptsPath(out),
+        String.valueOf(appAttemptId));
+  }
+
+  @Override
+  protected Path getJobAttemptPath(int appAttemptId) {
+    return new Path(getPendingJobAttemptsPath(commitsDirectory),
+        String.valueOf(appAttemptId));
+  }
+
+  /**
+   * Compute the path where the output of pending task attempts are stored.
+   * @param context the context of the job with pending tasks.
+   * @return the path where the output of pending task attempts are stored.
+   */
+  private static Path getPendingTaskAttemptsPath(JobContext context, Path out) {
+    return new Path(getJobAttemptPath(context, out), TEMPORARY);
+  }
+
+  /**
+   * 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.
+   * @param out The output path to put things in.
+   * @return the path where a task attempt should be stored.
+   */
+  public static Path getTaskAttemptPath(TaskAttemptContext context, Path out) {
+    return new Path(getPendingTaskAttemptsPath(context, out),
+        String.valueOf(context.getTaskAttemptID()));
+  }
+
+  /**
+   * Get the location of pending job attempts.
+   * @param out the base output directory.
+   * @return the location of pending job attempts.
+   */
+  private static Path getPendingJobAttemptsPath(Path out) {
+    Preconditions.checkNotNull(out, "Null 'out' path");
+    return new Path(out, TEMPORARY);
+  }
+
+  /**
+   * Compute the path where the output of a committed task is stored until
+   * the entire job is committed.
+   * @param context the context of the task attempt
+   * @return the path where the output of a committed task is stored until
+   * the entire job is committed.
+   */
+  public Path getCommittedTaskPath(TaskAttemptContext context) {
+    return getCommittedTaskPath(getAppAttemptId(context), context);
+  }
+
+  /**
+   * Validate the task attempt context; makes sure
+   * that the task attempt ID data is valid.
+   * @param context task context
+   */
+  private static void validateContext(TaskAttemptContext context) {
+    Preconditions.checkNotNull(context, "null context");
+    Preconditions.checkNotNull(context.getTaskAttemptID(),
+        "null task attempt ID");
+    Preconditions.checkNotNull(context.getTaskAttemptID().getTaskID(),
+        "null task ID");
+    Preconditions.checkNotNull(context.getTaskAttemptID().getJobID(),
+        "null job ID");
+  }
+
+  /**
+   * Compute the path where the output of a committed task is stored until the
+   * entire job is committed for a specific application attempt.
+   * @param appAttemptId the ID of the application attempt to use
+   * @param context the context of any task.
+   * @return the path where the output of a committed task is stored.
+   */
+  protected Path getCommittedTaskPath(int appAttemptId,
+      TaskAttemptContext context) {
+    validateContext(context);
+    return new Path(getJobAttemptPath(appAttemptId),
+        String.valueOf(context.getTaskAttemptID().getTaskID()));
+  }
+
+  @Override
+  public Path getTempTaskAttemptPath(TaskAttemptContext context) {
+    throw new UnsupportedOperationException("Unimplemented");
+  }
+
+  /**
+   * Lists the output of a task under the task attempt path. Subclasses can
+   * override this method to change how output files are identified.
+   * <p>
+   * This implementation lists the files that are direct children of the output
+   * path and filters hidden files (file names starting with '.' or '_').
+   * <p>
+   * The task attempt path is provided by
+   * {@link #getTaskAttemptPath(TaskAttemptContext)}
+   *
+   * @param context this task's {@link TaskAttemptContext}
+   * @return the output files produced by this task in the task attempt path
+   * @throws IOException on a failure
+   */
+  protected List<LocatedFileStatus> getTaskOutput(TaskAttemptContext context)
+      throws IOException {
+
+    // get files on the local FS in the attempt path
+    Path attemptPath = getTaskAttemptPath(context);
+    Preconditions.checkNotNull(attemptPath,
+        "No attemptPath path in {}", this);
+
+    LOG.debug("Scanning {} for files to commit", attemptPath);
+
+    return listAndFilter(getTaskAttemptFilesystem(context),
+        attemptPath, true, HIDDEN_FILE_FILTER);
+  }
+
+  /**
+   * Returns the final S3 key for a relative path. Subclasses can override this
+   * method to upload files to a different S3 location.
+   * <p>
+   * This implementation concatenates the relative path with the key prefix
+   * from the output path.
+   * If {@link CommitConstants#FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES} is
+   * set, then the task UUID is also included in the calculation
+   *
+   * @param relative the path of a file relative to the task attempt path
+   * @param context the JobContext or TaskAttemptContext for this job
+   * @return the S3 key where the file will be uploaded
+   */
+  protected String getFinalKey(String relative, JobContext context) {
+    if (uniqueFilenames) {
+      return getS3KeyPrefix(context) + "/" + Paths.addUUID(relative, uuid);
+    } else {
+      return getS3KeyPrefix(context) + "/" + relative;
+    }
+  }
+
+  /**
+   * Returns the final S3 location for a relative path as a Hadoop {@link Path}.
+   * This is a final method that calls {@link #getFinalKey(String, JobContext)}
+   * to determine the final location.
+   *
+   * @param relative the path of a file relative to the task attempt path
+   * @param context the JobContext or TaskAttemptContext for this job
+   * @return the S3 Path where the file will be uploaded
+   */
+  protected final Path getFinalPath(String relative, JobContext context)
+      throws IOException {
+    return getDestS3AFS().keyToQualifiedPath(getFinalKey(relative, context));
+  }
+
+  /**
+   * Return the local work path as the destination for writing work.
+   * @param context the context of the task attempt.
+   * @return a path in the local filesystem.
+   */
+  @Override
+  public Path getBaseTaskAttemptPath(TaskAttemptContext context) {
+    // a path on the local FS for files that will be uploaded
+    return getWorkPath();
+  }
+
+  /**
+   * For a job attempt path, the staging committer returns that of the
+   * wrapped committer.
+   * @param context the context of the job.
+   * @return a path in HDFS.
+   */
+  @Override
+  public Path getJobAttemptPath(JobContext context) {
+    return wrappedCommitter.getJobAttemptPath(context);
+  }
+
+  /**
+   * Set up the job, including calling the same method on the
+   * wrapped committer.
+   * @param context job context
+   * @throws IOException IO failure.
+   */
+  @Override
+  public void setupJob(JobContext context) throws IOException {
+    LOG.debug("{}, Setting up job {}", getRole(), jobIdString(context));
+    context.getConfiguration().set(
+        InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID, uuid);
+    wrappedCommitter.setupJob(context);
+  }
+
+  /**
+   * Get the list of pending uploads for this job attempt.
+   * @param context job context
+   * @return a list of pending uploads.
+   * @throws IOException Any IO failure
+   */
+  @Override
+  protected List<SinglePendingCommit> listPendingUploadsToCommit(
+      JobContext context)
+      throws IOException {
+    return listPendingUploads(context, false);
+  }
+
+  /**
+   * Get the list of pending uploads for this job attempt, swallowing
+   * exceptions.
+   * @param context job context
+   * @return a list of pending uploads. If an exception was swallowed,
+   * then this may not match the actual set of pending operations
+   * @throws IOException shouldn't be raised, but retained for the compiler
+   */
+  protected List<SinglePendingCommit> listPendingUploadsToAbort(
+      JobContext context) throws IOException {
+    return listPendingUploads(context, true);
+  }
+
+  /**
+   * Get the list of pending uploads for this job attempt.
+   * @param context job context
+   * @param suppressExceptions should exceptions be swallowed?
+   * @return a list of pending uploads. If exceptions are being swallowed,
+   * then this may not match the actual set of pending operations
+   * @throws IOException Any IO failure which wasn't swallowed.
+   */
+  protected List<SinglePendingCommit> listPendingUploads(
+      JobContext context, boolean suppressExceptions) throws IOException {
+    try {
+      Path wrappedJobAttemptPath = wrappedCommitter.getJobAttemptPath(context);
+      final FileSystem attemptFS = wrappedJobAttemptPath.getFileSystem(
+          context.getConfiguration());
+      return loadPendingsetFiles(context, suppressExceptions, attemptFS,
+          listAndFilter(attemptFS,
+              wrappedJobAttemptPath, false,
+              HIDDEN_FILE_FILTER));
+    } catch (IOException e) {
+      // unable to work with endpoint, if suppressing errors decide our actions
+      maybeIgnore(suppressExceptions, "Listing pending uploads", e);
+    }
+    // reached iff an IOE was caught and swallowed
+    return new ArrayList<>(0);
+  }
+
+  @Override
+  public void cleanupStagingDirs() {
+    Path workPath = getWorkPath();
+    if (workPath != null) {
+      LOG.debug("Cleaning up work path {}", workPath);
+      ignoreIOExceptions(LOG, "cleaning up", workPath.toString(),
+          () -> deleteQuietly(workPath.getFileSystem(getConf()),
+              workPath, true));
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  protected void cleanup(JobContext context,
+      boolean suppressExceptions)
+      throws IOException {
+    maybeIgnore(suppressExceptions, "Cleanup wrapped committer",
+        () -> wrappedCommitter.cleanupJob(context));
+    maybeIgnore(suppressExceptions, "Delete destination paths",
+        () -> deleteDestinationPaths(context));
+    super.cleanup(context, suppressExceptions);
+  }
+
+  @Override
+  protected void abortPendingUploadsInCleanup(boolean suppressExceptions)
+      throws IOException {
+    if (getConf()
+        .getBoolean(FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS, true)) {
+      super.abortPendingUploadsInCleanup(suppressExceptions);
+    } else {
+      LOG.info("Not cleanup up pending uploads to {} as {} is false ",
+          getOutputPath(),
+          FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS);
+    }
+  }
+
+  @Override
+  protected void abortJobInternal(JobContext context,
+      boolean suppressExceptions) throws IOException {
+    String r = getRole();
+    boolean failed = false;
+    try (DurationInfo d = new DurationInfo(LOG,
+        "%s: aborting job in state %s ", r, jobIdString(context))) {
+      List<SinglePendingCommit> pending = listPendingUploadsToAbort(context);
+      abortPendingUploads(context, pending, suppressExceptions);
+    } catch (FileNotFoundException e) {
+      // nothing to list
+      LOG.debug("No job directory to read uploads from");
+    } catch (IOException e) {
+      failed = true;
+      maybeIgnore(suppressExceptions, "aborting job", e);
+    } finally {
+      super.abortJobInternal(context, failed || suppressExceptions);
+    }
+  }
+
+  /**
+   * Delete the working paths of a job. Does not attempt to clean up
+   * the work of the wrapped committer.
+   * <ol>
+   *   <li>The job attempt path</li>
+   *   <li>$dest/__temporary</li>
+   *   <li>the local working directory for staged files</li>
+   * </ol>
+   * @param context job context
+   * @throws IOException IO failure
+   */
+  protected void deleteDestinationPaths(JobContext context) throws IOException {
+    Path attemptPath = getJobAttemptPath(context);
+    ignoreIOExceptions(LOG,
+        "Deleting Job attempt Path", attemptPath.toString(),
+        () -> deleteWithWarning(
+            getJobAttemptFileSystem(context),
+            attemptPath,
+            true));
+
+    // delete the __temporary directory. This will cause problems
+    // if there is >1 task targeting the same dest dir
+    deleteWithWarning(getDestFS(),
+        new Path(getOutputPath(), TEMPORARY),
+        true);
+    // and the working path
+    deleteTaskWorkingPathQuietly(context);
+  }
+
+
+  @Override
+  public void setupTask(TaskAttemptContext context) throws IOException {
+    Path taskAttemptPath = getTaskAttemptPath(context);
+    try (DurationInfo d = new DurationInfo(LOG,
+        "%s: setup task attempt path %s ", getRole(), taskAttemptPath)) {
+      // create the local FS
+      taskAttemptPath.getFileSystem(getConf()).mkdirs(taskAttemptPath);
+      wrappedCommitter.setupTask(context);
+    }
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context)
+      throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "%s: needsTaskCommit() Task %s",
+        getRole(), context.getTaskAttemptID())) {
+      // check for files on the local FS in the attempt path
+      Path attemptPath = getTaskAttemptPath(context);
+      FileSystem fs = getTaskAttemptFilesystem(context);
+
+      // This could be made more efficient with a probe "hasChildren(Path)"
+      // which returns true if there is >1 entry under a given path.
+      FileStatus[] stats = fs.listStatus(attemptPath);
+      LOG.debug("{} files to commit under {}", stats.length, attemptPath);
+      return stats.length > 0;
+    } catch (FileNotFoundException e) {
+      // list didn't find a directory, so nothing to commit
+      // TODO: throw this up as an error?
+      LOG.info("No files to commit");
+      throw e;
+    }
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    try (DurationInfo d = new DurationInfo(LOG,
+        "%s: commit task %s", getRole(), context.getTaskAttemptID())) {
+      int count = commitTaskInternal(context, getTaskOutput(context));
+      LOG.info("{}: upload file count: {}", getRole(), count);
+    } catch (IOException e) {
+      LOG.error("{}: commit of task {} failed",
+          getRole(), context.getTaskAttemptID(), e);
+      getCommitOperations().taskCompleted(false);
+      throw e;
+    }
+    getCommitOperations().taskCompleted(true);
+  }
+
+  /**
+   * Commit the task by uploading all created files and then
+   * writing a pending entry for them.
+   * @param context task context
+   * @param taskOutput list of files from the output
+   * @return number of uploads committed.
+   * @throws IOException IO Failures.
+   */
+  protected int commitTaskInternal(final TaskAttemptContext context,
+      List<? extends FileStatus> taskOutput)
+      throws IOException {
+    LOG.debug("{}: commitTaskInternal", getRole());
+    Configuration conf = context.getConfiguration();
+
+    final Path attemptPath = getTaskAttemptPath(context);
+    FileSystem attemptFS = getTaskAttemptFilesystem(context);
+    LOG.debug("{}: attempt path is {}", getRole(), attemptPath);
+
+    // add the commits file to the wrapped committer's task attempt location.
+    // of this method.
+    Path commitsAttemptPath = wrappedCommitter.getTaskAttemptPath(context);
+    FileSystem commitsFS = commitsAttemptPath.getFileSystem(conf);
+
+    // keep track of unfinished commits in case one fails. if something fails,
+    // we will try to abort the ones that had already succeeded.
+    int commitCount = taskOutput.size();
+    final Queue<SinglePendingCommit> commits = new ConcurrentLinkedQueue<>();
+    LOG.info("{}: uploading from staging directory to S3", getRole());
+    LOG.info("{}: Saving pending data information to {}",
+        getRole(), commitsAttemptPath);
+    if (taskOutput.isEmpty()) {
+      // there is nothing to write. needsTaskCommit() should have caught
+      // this, so warn that there is some kind of problem in the protocol.
+      LOG.warn("{}: No files to commit", getRole());
+    } else {
+      boolean threw = true;
+      // before the uploads, report some progress
+      context.progress();
+
+      PendingSet pendingCommits = new PendingSet(commitCount);
+      try {
+        Tasks.foreach(taskOutput)
+            .stopOnFailure()
+            .executeWith(buildThreadPool(context))
+            .run(stat -> {
+              Path path = stat.getPath();
+              File localFile = new File(path.toUri().getPath());
+              String relative = Paths.getRelativePath(attemptPath, path);
+              String partition = Paths.getPartition(relative);
+              String key = getFinalKey(relative, context);
+              Path destPath = getDestS3AFS().keyToQualifiedPath(key);
+              SinglePendingCommit commit = getCommitOperations()
+                  .uploadFileToPendingCommit(
+                      localFile,
+                      destPath,
+                      partition,
+                      uploadPartSize);
+              LOG.debug("{}: adding pending commit {}", getRole(), commit);
+              commits.add(commit);
+            });
+
+        for (SinglePendingCommit commit : commits) {
+          pendingCommits.add(commit);
+        }
+
+        // save the data
+        // although overwrite=false, there's still a risk of > 1 entry being
+        // committed if the FS doesn't have create-no-overwrite consistency.
+
+        LOG.debug("Saving {} pending commit(s)) to file {}",
+            pendingCommits.size(),
+            commitsAttemptPath);
+        pendingCommits.save(commitsFS, commitsAttemptPath, false);
+        threw = false;
+
+      } finally {
+        if (threw) {
+          LOG.error(
+              "{}: Exception during commit process, aborting {} commit(s)",
+              getRole(), commits.size());
+          Tasks.foreach(commits)
+              .suppressExceptions()
+              .run(commit -> getCommitOperations().abortSingleCommit(commit));
+          deleteTaskAttemptPathQuietly(context);
+        }
+      }
+      // always purge attempt information at this point.
+      Paths.clearTempFolderInfo(context.getTaskAttemptID());
+    }
+
+    LOG.debug("Committing wrapped task");
+    wrappedCommitter.commitTask(context);
+
+    LOG.debug("Cleaning up attempt dir {}", attemptPath);
+    attemptFS.delete(attemptPath, true);
+    return commits.size();
+  }
+
+  /**
+   * Abort the task.
+   * The API specifies that the task has not yet been committed, so there are
+   * no uploads that need to be cancelled.
+   * Accordingly just delete files on the local FS, and call abortTask in
+   * the wrapped committer.
+   * <b>Important: this may be called in the AM after a container failure.</b>
+   * When that occurs and the failed container was on a different host in the
+   * cluster, the local files will not be deleted.
+   * @param context task context
+   * @throws IOException any failure
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // the API specifies that the task has not yet been committed, so there are
+    // no uploads that need to be cancelled. just delete files on the local FS.
+    try (DurationInfo d = new DurationInfo(LOG,
+        "Abort task %s", context.getTaskAttemptID())) {
+      deleteTaskAttemptPathQuietly(context);
+      deleteTaskWorkingPathQuietly(context);
+      wrappedCommitter.abortTask(context);
+    } catch (IOException e) {
+      LOG.error("{}: exception when aborting task {}",
+          getRole(), context.getTaskAttemptID(), e);
+      throw e;
+    }
+  }
+
+  /**
+   * Get the work path for a task.
+   * @param context job/task complex
+   * @param uuid UUID
+   * @return a path
+   * @throws IOException failure to build the path
+   */
+  private static Path taskAttemptWorkingPath(TaskAttemptContext context,
+      String uuid) throws IOException {
+    return getTaskAttemptPath(context,
+        Paths.getLocalTaskAttemptTempDir(
+            context.getConfiguration(),
+            uuid,
+            context.getTaskAttemptID()));
+  }
+
+  /**
+   * Delete the working path of a task; no-op if there is none, that
+   * is: this is a job.
+   * @param context job/task context
+   */
+  protected void deleteTaskWorkingPathQuietly(JobContext context) {
+    ignoreIOExceptions(LOG, "Delete working path", "",
+        () -> {
+          Path path = buildWorkPath(context, getUUID());
+          if (path != null) {
+            deleteQuietly(path.getFileSystem(getConf()), path, true);
+          }
+        });
+  }
+
+  /**
+   * Get the key of the destination "directory" of the job/task.
+   * @param context job context
+   * @return key to write to
+   */
+  private String getS3KeyPrefix(JobContext context) {
+    return s3KeyPrefix;
+  }
+
+  /**
+   * A UUID for this upload, as calculated with.
+   * {@link #getUploadUUID(Configuration, String)}
+   * @return the UUID for files
+   */
+  protected String getUUID() {
+    return uuid;
+  }
+
+  /**
+   * Returns the {@link ConflictResolution} mode for this commit.
+   *
+   * @param context the JobContext for this commit
+   * @param fsConf filesystem config
+   * @return the ConflictResolution mode
+   */
+  public final ConflictResolution getConflictResolutionMode(
+      JobContext context,
+      Configuration fsConf) {
+    if (conflictResolution == null) {
+      this.conflictResolution = ConflictResolution.valueOf(
+          getConfictModeOption(context, fsConf));
+    }
+    return conflictResolution;
+  }
+
+  /**
+   * Get the conflict mode option string.
+   * @param context context with the config
+   * @param fsConf filesystem config
+   * @return the trimmed configuration option, upper case.
+   */
+  public static String getConfictModeOption(JobContext context,
+      Configuration fsConf) {
+    return getConfigurationOption(context,
+        fsConf,
+        FS_S3A_COMMITTER_STAGING_CONFLICT_MODE,
+        DEFAULT_CONFLICT_MODE).toUpperCase(Locale.ENGLISH);
+  }
+
+}

+ 64 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java

@@ -0,0 +1,64 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Internal staging committer constants.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class StagingCommitterConstants {
+
+  private StagingCommitterConstants() {
+  }
+
+  /**
+   * The temporary path for staging data, if not explicitly set.
+   * By using an unqualified path, this will be qualified to be relative
+   * to the users' home directory, so protectec from access for others.
+   */
+  public static final String FILESYSTEM_TEMP_PATH = "tmp/staging";
+
+  /** Name of the root partition :{@value}. */
+  public static final String TABLE_ROOT = "table_root";
+
+  /**
+   * Filename used under {@code ~/${UUID}} for the staging files.
+   */
+  public static final String STAGING_UPLOADS = "staging-uploads";
+
+  // Spark configuration keys
+
+  /**
+   * The UUID for jobs: {@value}.
+   */
+  public static final String SPARK_WRITE_UUID =
+      "spark.sql.sources.writeJobUUID";
+
+  /**
+   * The App ID for jobs.
+   */
+
+  public static final String SPARK_APP_ID = "spark.app.id";
+
+  public static final String JAVA_IO_TMPDIR = "java.io.tmpdir";
+}

+ 49 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterFactory.java

@@ -0,0 +1,49 @@
+/*
+ * 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.fs.s3a.commit.staging;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+
+/**
+ * Factory for the staging committer.
+ * This is for internal test use, rather than the public directory and
+ * partitioned committers.
+ */
+public class StagingCommitterFactory
+    extends AbstractS3ACommitterFactory {
+
+  /**
+   * Name of this class: {@value}.
+   */
+  public static final String CLASSNAME
+      = "org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterFactory";
+
+  public PathOutputCommitter createTaskCommitter(S3AFileSystem fileSystem,
+      Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    return new StagingCommitter(outputPath, context);
+  }
+
+}

+ 27 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/package-info.java

@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * The staging committers.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.commit.staging;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 253 - 115
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java

@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
@@ -67,8 +68,12 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.Tristate;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
@@ -211,6 +216,28 @@ public class DynamoDBMetadataStore implements MetadataStore {
   private RetryPolicy dataAccessRetryPolicy;
   private S3AInstrumentation.S3GuardInstrumentation instrumentation;
 
+  /** Owner FS: only valid if configured with an owner FS. */
+  private S3AFileSystem owner;
+
+  /** Invoker for IO. Until configured properly, use try-once. */
+  private Invoker invoker = new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL,
+      Invoker.NO_OP
+  );
+
+  /** Data access can have its own policies. */
+  private Invoker dataAccess;
+
+  /**
+   * Total limit on the number of throttle events after which
+   * we stop warning in the log. Keeps the noise down.
+   */
+  private static final int THROTTLE_EVENT_LOG_LIMIT = 100;
+
+  /**
+   * Count of the total number of throttle events; used to crank back logging.
+   */
+  private AtomicInteger throttleEventCount = new AtomicInteger(0);
+
   /**
    * A utility function to create DynamoDB instance.
    * @param conf the file system configuration
@@ -232,28 +259,34 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceRaw
   public void initialize(FileSystem fs) throws IOException {
     Preconditions.checkArgument(fs instanceof S3AFileSystem,
         "DynamoDBMetadataStore only supports S3A filesystem.");
-    final S3AFileSystem s3afs = (S3AFileSystem) fs;
-    instrumentation = s3afs.getInstrumentation().getS3GuardInstrumentation();
-    final String bucket = s3afs.getBucket();
-    String confRegion = s3afs.getConf().getTrimmed(S3GUARD_DDB_REGION_KEY);
+    owner = (S3AFileSystem) fs;
+    instrumentation = owner.getInstrumentation().getS3GuardInstrumentation();
+    final String bucket = owner.getBucket();
+    conf = owner.getConf();
+    String confRegion = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
     if (!StringUtils.isEmpty(confRegion)) {
       region = confRegion;
       LOG.debug("Overriding S3 region with configured DynamoDB region: {}",
           region);
     } else {
-      region = s3afs.getBucketLocation();
+      region = owner.getBucketLocation();
       LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
     }
-    username = s3afs.getUsername();
-    conf = s3afs.getConf();
+    username = owner.getUsername();
     dynamoDB = createDynamoDB(conf, region);
 
     // use the bucket as the DynamoDB table name if not specified in config
     tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket);
-    setMaxRetries(conf);
+    initDataAccessRetries(conf);
+
+    // set up a full retry policy
+    invoker = new Invoker(new S3ARetryPolicy(conf),
+        this::retryEvent
+    );
 
     initTable();
 
@@ -283,6 +316,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @throws IllegalArgumentException if the configuration is incomplete
    */
   @Override
+  @Retries.OnceRaw
   public void initialize(Configuration config) throws IOException {
     conf = config;
     // use the bucket as the DynamoDB table name if not specified in config
@@ -295,7 +329,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
     dynamoDB = createDynamoDB(conf, region);
 
     username = UserGroupInformation.getCurrentUser().getShortUserName();
-    setMaxRetries(conf);
+    initDataAccessRetries(conf);
 
     initTable();
   }
@@ -304,22 +338,25 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * Set retry policy. This is driven by the value of
    * {@link Constants#S3GUARD_DDB_MAX_RETRIES} with an exponential backoff
    * between each attempt of {@link #MIN_RETRY_SLEEP_MSEC} milliseconds.
-   * @param config
+   * @param config configuration for data access
    */
-  private void setMaxRetries(Configuration config) {
+  private void initDataAccessRetries(Configuration config) {
     int maxRetries = config.getInt(S3GUARD_DDB_MAX_RETRIES,
         S3GUARD_DDB_MAX_RETRIES_DEFAULT);
     dataAccessRetryPolicy = RetryPolicies
         .exponentialBackoffRetry(maxRetries, MIN_RETRY_SLEEP_MSEC,
             TimeUnit.MILLISECONDS);
+    dataAccess = new Invoker(dataAccessRetryPolicy, this::retryEvent);
   }
 
   @Override
+  @Retries.RetryTranslated
   public void delete(Path path) throws IOException {
     innerDelete(path, true);
   }
 
   @Override
+  @Retries.RetryTranslated
   public void forgetMetadata(Path path) throws IOException {
     innerDelete(path, false);
   }
@@ -332,9 +369,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @param tombstone flag to create a tombstone marker
    * @throws IOException I/O error.
    */
-  private void innerDelete(Path path, boolean tombstone)
+  @Retries.RetryTranslated
+  private void innerDelete(final Path path, boolean tombstone)
       throws IOException {
-    path = checkPath(path);
+    checkPath(path);
     LOG.debug("Deleting from table {} in region {}: {}",
         tableName, region, path);
 
@@ -343,23 +381,25 @@ public class DynamoDBMetadataStore implements MetadataStore {
       LOG.debug("Skip deleting root directory as it does not exist in table");
       return;
     }
-
-    try {
-      if (tombstone) {
-        Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
-            PathMetadata.tombstone(path));
-        table.putItem(item);
-      } else {
-        table.deleteItem(pathToKey(path));
-      }
-    } catch (AmazonClientException e) {
-      throw translateException("delete", path, e);
+    // the policy on whether repeating delete operations is based
+    // on that of S3A itself
+    boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT;
+    if (tombstone) {
+      Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
+          PathMetadata.tombstone(path));
+      invoker.retry("Put tombstone", path.toString(), idempotent,
+          () -> table.putItem(item));
+    } else {
+      PrimaryKey key = pathToKey(path);
+      invoker.retry("Delete key", path.toString(), idempotent,
+          () -> table.deleteItem(key));
     }
   }
 
   @Override
+  @Retries.RetryTranslated
   public void deleteSubtree(Path path) throws IOException {
-    path = checkPath(path);
+    checkPath(path);
     LOG.debug("Deleting subtree from table {} in region {}: {}",
         tableName, region, path);
 
@@ -375,6 +415,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
     }
   }
 
+  @Retries.OnceRaw
   private Item getConsistentItem(PrimaryKey key) {
     final GetItemSpec spec = new GetItemSpec()
         .withPrimaryKey(key)
@@ -383,52 +424,65 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceTranslated
   public PathMetadata get(Path path) throws IOException {
     return get(path, false);
   }
 
   @Override
+  @Retries.OnceTranslated
   public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
       throws IOException {
-    path = checkPath(path);
+    checkPath(path);
     LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
+    return Invoker.once("get", path.toString(),
+        () -> innerGet(path, wantEmptyDirectoryFlag));
+  }
 
-    try {
-      final PathMetadata meta;
-      if (path.isRoot()) {
-        // Root does not persist in the table
-        meta = new PathMetadata(makeDirStatus(username, path));
-      } else {
-        final Item item = getConsistentItem(pathToKey(path));
-        meta = itemToPathMetadata(item, username);
-        LOG.debug("Get from table {} in region {} returning for {}: {}",
-            tableName, region, path, meta);
-      }
+  /**
+   * Inner get operation, as invoked in the retry logic.
+   * @param path the path to get
+   * @param wantEmptyDirectoryFlag Set to true to give a hint to the
+   *   MetadataStore that it should try to compute the empty directory flag.
+   * @return metadata for {@code path}, {@code null} if not found
+   * @throws IOException IO problem
+   * @throws AmazonClientException dynamo DB level problem
+   */
+  @Retries.OnceRaw
+  private PathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    final PathMetadata meta;
+    if (path.isRoot()) {
+      // Root does not persist in the table
+      meta = new PathMetadata(makeDirStatus(username, path));
+    } else {
+      final Item item = getConsistentItem(pathToKey(path));
+      meta = itemToPathMetadata(item, username);
+      LOG.debug("Get from table {} in region {} returning for {}: {}",
+          tableName, region, path, meta);
+    }
 
-      if (wantEmptyDirectoryFlag && meta != null) {
-        final FileStatus status = meta.getFileStatus();
-        // for directory, we query its direct children to determine isEmpty bit
-        if (status.isDirectory()) {
-          final QuerySpec spec = new QuerySpec()
-              .withHashKey(pathToParentKeyAttribute(path))
-              .withConsistentRead(true)
-              .withFilterExpression(IS_DELETED + " = :false")
-              .withValueMap(deleteTrackingValueMap);
-          final ItemCollection<QueryOutcome> items = table.query(spec);
-          boolean hasChildren = items.iterator().hasNext();
-          // When this class has support for authoritative
-          // (fully-cached) directory listings, we may also be able to answer
-          // TRUE here.  Until then, we don't know if we have full listing or
-          // not, thus the UNKNOWN here:
-          meta.setIsEmptyDirectory(
-              hasChildren ? Tristate.FALSE : Tristate.UNKNOWN);
-        }
+    if (wantEmptyDirectoryFlag && meta != null) {
+      final FileStatus status = meta.getFileStatus();
+      // for directory, we query its direct children to determine isEmpty bit
+      if (status.isDirectory()) {
+        final QuerySpec spec = new QuerySpec()
+            .withHashKey(pathToParentKeyAttribute(path))
+            .withConsistentRead(true)
+            .withFilterExpression(IS_DELETED + " = :false")
+            .withValueMap(deleteTrackingValueMap);
+        final ItemCollection<QueryOutcome> items = table.query(spec);
+        boolean hasChildren = items.iterator().hasNext();
+        // When this class has support for authoritative
+        // (fully-cached) directory listings, we may also be able to answer
+        // TRUE here.  Until then, we don't know if we have full listing or
+        // not, thus the UNKNOWN here:
+        meta.setIsEmptyDirectory(
+            hasChildren ? Tristate.FALSE : Tristate.UNKNOWN);
       }
-
-      return meta;
-    } catch (AmazonClientException e) {
-      throw translateException("get", path, e);
     }
+
+    return meta;
   }
 
   /**
@@ -445,35 +499,38 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
-  public DirListingMetadata listChildren(Path path) throws IOException {
-    path = checkPath(path);
+  @Retries.OnceTranslated
+  public DirListingMetadata listChildren(final Path path) throws IOException {
+    checkPath(path);
     LOG.debug("Listing table {} in region {}: {}", tableName, region, path);
 
     // find the children in the table
-    try {
-      final QuerySpec spec = new QuerySpec()
-          .withHashKey(pathToParentKeyAttribute(path))
-          .withConsistentRead(true); // strictly consistent read
-      final ItemCollection<QueryOutcome> items = table.query(spec);
-
-      final List<PathMetadata> metas = new ArrayList<>();
-      for (Item item : items) {
-        PathMetadata meta = itemToPathMetadata(item, username);
-        metas.add(meta);
-      }
-      LOG.trace("Listing table {} in region {} for {} returning {}",
-          tableName, region, path, metas);
+    return Invoker.once("listChildren", path.toString(),
+        () -> {
+          final QuerySpec spec = new QuerySpec()
+              .withHashKey(pathToParentKeyAttribute(path))
+              .withConsistentRead(true); // strictly consistent read
+          final ItemCollection<QueryOutcome> items = table.query(spec);
 
-      return (metas.isEmpty() && get(path) == null)
-          ? null
-          : new DirListingMetadata(path, metas, false);
-    } catch (AmazonClientException e) {
-      // failure, including the path not being present
-      throw translateException("listChildren", path, e);
-    }
+          final List<PathMetadata> metas = new ArrayList<>();
+          for (Item item : items) {
+            PathMetadata meta = itemToPathMetadata(item, username);
+            metas.add(meta);
+          }
+          LOG.trace("Listing table {} in region {} for {} returning {}",
+              tableName, region, path, metas);
+
+          return (metas.isEmpty() && get(path) == null)
+              ? null
+              : new DirListingMetadata(path, metas, false);
+        });
   }
 
-  // build the list of all parent entries.
+  /**
+   * build the list of all parent entries.
+   * @param pathsToCreate paths to create
+   * @return the full ancestry paths
+   */
   Collection<PathMetadata> completeAncestry(
       Collection<PathMetadata> pathsToCreate) {
     // Key on path to allow fast lookup
@@ -499,6 +556,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceTranslated
   public void move(Collection<Path> pathsToDelete,
       Collection<PathMetadata> pathsToCreate) throws IOException {
     if (pathsToDelete == null && pathsToCreate == null) {
@@ -527,21 +585,20 @@ public class DynamoDBMetadataStore implements MetadataStore {
       }
     }
 
-    try {
-      processBatchWriteRequest(null, pathMetadataToItem(newItems));
-    } catch (AmazonClientException e) {
-      throw translateException("move", (String) null, e);
-    }
+    Invoker.once("move", tableName,
+        () -> processBatchWriteRequest(null, pathMetadataToItem(newItems)));
   }
 
   /**
    * Helper method to issue a batch write request to DynamoDB.
    *
-   * Callers of this method should catch the {@link AmazonClientException} and
-   * translate it for better error report and easier debugging.
+   * The retry logic here is limited to repeating the write operations
+   * until all items have been written; there is no other attempt
+   * at recovery/retry. Throttling is handled internally.
    * @param keysToDelete primary keys to be deleted; can be null
    * @param itemsToPut new items to be put; can be null
    */
+  @Retries.OnceRaw("Outstanding batch items are updated with backoff")
   private void processBatchWriteRequest(PrimaryKey[] keysToDelete,
       Item[] itemsToPut) throws IOException {
     final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
@@ -575,7 +632,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
       // Check for unprocessed keys in case of exceeding provisioned throughput
       Map<String, List<WriteRequest>> unprocessed = res.getUnprocessedItems();
       int retryCount = 0;
-      while (unprocessed.size() > 0) {
+      while (!unprocessed.isEmpty()) {
         retryBackoff(retryCount++);
         res = dynamoDB.batchWriteItemUnprocessed(unprocessed);
         unprocessed = res.getUnprocessedItems();
@@ -603,12 +660,17 @@ public class DynamoDBMetadataStore implements MetadataStore {
         LOG.debug("Sleeping {} msec before next retry", action.delayMillis);
         Thread.sleep(action.delayMillis);
       }
+    } catch (InterruptedException e) {
+      throw (IOException)new InterruptedIOException(e.toString()).initCause(e);
+    } catch (IOException e) {
+      throw e;
     } catch (Exception e) {
       throw new IOException("Unexpected exception", e);
     }
   }
 
   @Override
+  @Retries.OnceRaw
   public void put(PathMetadata meta) throws IOException {
     // For a deeply nested path, this method will automatically create the full
     // ancestry and save respective item in DynamoDB table.
@@ -624,6 +686,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceRaw
   public void put(Collection<PathMetadata> metas) throws IOException {
     LOG.debug("Saving batch to table {} in region {}", tableName, region);
 
@@ -633,6 +696,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   /**
    * Helper method to get full path of ancestors that are nonexistent in table.
    */
+  @Retries.OnceRaw
   private Collection<PathMetadata> fullPathsToPut(PathMetadata meta)
       throws IOException {
     checkPathMetadata(meta);
@@ -675,25 +739,34 @@ public class DynamoDBMetadataStore implements MetadataStore {
         null, owner, owner, f);
   }
 
+  /**
+   * {@inheritDoc}.
+   * There is retry around building the list of paths to update, but
+   * the call to {@link #processBatchWriteRequest(PrimaryKey[], Item[])}
+   * is only tried once.
+   * @param meta Directory listing metadata.
+   * @throws IOException
+   */
   @Override
+  @Retries.OnceTranslated("retry(listFullPaths); once(batchWrite)")
   public void put(DirListingMetadata meta) throws IOException {
     LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
 
     // directory path
-    PathMetadata p = new PathMetadata(makeDirStatus(meta.getPath(), username),
+    Path path = meta.getPath();
+    PathMetadata p = new PathMetadata(makeDirStatus(path, username),
         meta.isEmpty(), false);
 
     // First add any missing ancestors...
-    final Collection<PathMetadata> metasToPut = fullPathsToPut(p);
+    final Collection<PathMetadata> metasToPut = invoker.retry(
+        "paths to put", path.toString(), true,
+        () -> fullPathsToPut(p));
 
     // next add all children of the directory
     metasToPut.addAll(meta.getListing());
 
-    try {
-      processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
-    } catch (AmazonClientException e) {
-      throw translateException("put", (String) null, e);
-    }
+    Invoker.once("put", path.toString(),
+        () -> processBatchWriteRequest(null, pathMetadataToItem(metasToPut)));
   }
 
   @Override
@@ -709,6 +782,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceTranslated
   public void destroy() throws IOException {
     if (table == null) {
       LOG.info("In destroy(): no table to delete");
@@ -731,10 +805,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
       throw new InterruptedIOException("Table " + tableName
           + " in region " + region + " has not been deleted");
     } catch (AmazonClientException e) {
-      throw translateException("destroy", (String) null, e);
+      throw translateException("destroy", tableName, e);
     }
   }
 
+  @Retries.OnceRaw
   private ItemCollection<ScanOutcome> expiredFiles(long modTime) {
     String filterExpression = "mod_time < :mod_time";
     String projectionExpression = "parent,child";
@@ -743,6 +818,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceRaw("once(batchWrite)")
   public void prune(long modTime) throws IOException {
     int itemCount = 0;
     try {
@@ -797,6 +873,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * or table is being deleted, or any other I/O exception occurred.
    */
   @VisibleForTesting
+  @Retries.OnceRaw
   void initTable() throws IOException {
     table = dynamoDB.getTable(tableName);
     try {
@@ -848,7 +925,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
       }
 
     } catch (AmazonClientException e) {
-      throw translateException("initTable", (String) null, e);
+      throw translateException("initTable", tableName, e);
     }
   }
 
@@ -856,8 +933,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * Get the version mark item in the existing DynamoDB table.
    *
    * As the version marker item may be created by another concurrent thread or
-   * process, we retry a limited times before we fail to get it.
+   * process, we sleep and retry a limited times before we fail to get it.
+   * This does not include handling any failure other than "item not found",
+   * so this method is tagged as "OnceRaw"
    */
+  @Retries.OnceRaw
   private Item getVersionMarkerItem() throws IOException {
     final PrimaryKey versionMarkerKey =
         createVersionMarkerPrimaryKey(VERSION_MARKER);
@@ -913,16 +993,20 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @param t table to block on.
    * @throws IOException IO problems
    * @throws InterruptedIOException if the wait was interrupted
+   * @throws IllegalArgumentException if an exception was raised in the waiter
    */
-  private void waitForTableActive(Table t) throws IOException {
+  @Retries.OnceRaw
+  private void waitForTableActive(Table t) throws InterruptedIOException {
     try {
       t.waitForActive();
     } catch (InterruptedException e) {
       LOG.warn("Interrupted while waiting for table {} in region {} active",
           tableName, region, e);
       Thread.currentThread().interrupt();
-      throw (IOException) new InterruptedIOException("DynamoDB table '"
-          + tableName + "' is not active yet in region " + region).initCause(e);
+      throw (InterruptedIOException)
+          new InterruptedIOException("DynamoDB table '"
+          + tableName + "' is not active yet in region " + region)
+              .initCause(e);
     }
   }
 
@@ -933,6 +1017,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @throws IOException on any failure.
    * @throws InterruptedIOException if the wait was interrupted
    */
+  @Retries.OnceRaw
   private void createTable(ProvisionedThroughput capacity) throws IOException {
     try {
       LOG.info("Creating non-existent DynamoDB table {} in region {}",
@@ -960,6 +1045,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @param item item to put
    * @return the outcome.
    */
+  @Retries.OnceRaw
   PutItemOutcome putItem(Item item) {
     LOG.debug("Putting item {}", item);
     return table.putItem(item);
@@ -967,22 +1053,27 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
   /**
    * Provision the table with given read and write capacity units.
+   * Call will fail if the table is busy, or the new values match the current
+   * ones.
+   * @param readCapacity read units
+   * @param writeCapacity write units
+   * @throws IOException on a failure
    */
+  @Retries.RetryTranslated
   void provisionTable(Long readCapacity, Long writeCapacity)
       throws IOException {
     final ProvisionedThroughput toProvision = new ProvisionedThroughput()
         .withReadCapacityUnits(readCapacity)
         .withWriteCapacityUnits(writeCapacity);
-    try {
-      final ProvisionedThroughputDescription p =
-          table.updateTable(toProvision).getProvisionedThroughput();
-      LOG.info("Provision table {} in region {}: readCapacityUnits={}, "
-              + "writeCapacityUnits={}",
-          tableName, region, p.getReadCapacityUnits(),
-          p.getWriteCapacityUnits());
-    } catch (AmazonClientException e) {
-      throw translateException("provisionTable", (String) null, e);
-    }
+    invoker.retry("ProvisionTable", tableName, true,
+        () -> {
+          final ProvisionedThroughputDescription p =
+              table.updateTable(toProvision).getProvisionedThroughput();
+          LOG.info("Provision table {} in region {}: readCapacityUnits={}, "
+                  + "writeCapacityUnits={}",
+              tableName, region, p.getReadCapacityUnits(),
+              p.getWriteCapacityUnits());
+        });
   }
 
   Table getTable() {
@@ -999,8 +1090,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   /**
-   * Validates a path object; it must be absolute, and contain a host
-   * (bucket) component.
+   * Validates a path object; it must be absolute, have an s3a:/// scheme
+   * and contain a host (bucket) component.
+   * @param path path to check
+   * @return the path passed in
    */
   private Path checkPath(Path path) {
     Preconditions.checkNotNull(path);
@@ -1025,6 +1118,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceRaw
   public Map<String, String> getDiagnostics() throws IOException {
     Map<String, String> map = new TreeMap<>();
     if (table != null) {
@@ -1052,6 +1146,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
     return map;
   }
 
+  @Retries.OnceRaw
   private TableDescription getTableDescription(boolean forceUpdate) {
     TableDescription desc = table.getDescription();
     if (desc == null || forceUpdate) {
@@ -1061,6 +1156,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   }
 
   @Override
+  @Retries.OnceRaw
   public void updateParameters(Map<String, String> parameters)
       throws IOException {
     Preconditions.checkNotNull(table, "Not initialized");
@@ -1103,4 +1199,46 @@ public class DynamoDBMetadataStore implements MetadataStore {
     }
   }
 
+  /**
+   * Callback from {@link Invoker} when an operation is retried.
+   * @param text text of the operation
+   * @param ex exception
+   * @param attempts number of attempts
+   * @param idempotent is the method idempotent
+   */
+  void retryEvent(
+      String text,
+      IOException ex,
+      int attempts,
+      boolean idempotent) {
+    if (S3AUtils.isThrottleException(ex)) {
+      // throttled
+      if (instrumentation != null) {
+        instrumentation.throttled();
+      }
+      int eventCount = throttleEventCount.addAndGet(1);
+      if (attempts == 1 && eventCount < THROTTLE_EVENT_LOG_LIMIT) {
+        LOG.warn("DynamoDB IO limits reached in {};"
+                + " consider increasing capacity: {}", text, ex.toString());
+        LOG.debug("Throttled", ex);
+      } else {
+        // user has been warned already, log at debug only.
+        LOG.debug("DynamoDB IO limits reached in {};"
+                + " consider increasing capacity: {}", text, ex.toString());
+      }
+    } else if (attempts == 1) {
+      // not throttled. Log on the first attempt only
+      LOG.info("Retrying {}: {}", text, ex.toString());
+      LOG.debug("Retrying {}", text, ex);
+    }
+
+    if (instrumentation != null) {
+      // note a retry
+      instrumentation.retrying();
+    }
+    if (owner != null) {
+      owner.metastoreOperationRetried(ex, attempts, idempotent);
+    }
+  }
+
 }

+ 2 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java

@@ -37,6 +37,7 @@ 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.s3a.Retries;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.Tristate;
@@ -83,6 +84,7 @@ public final class S3Guard {
    * @return Reference to new MetadataStore.
    * @throws IOException if the metadata store cannot be instantiated
    */
+  @Retries.OnceTranslated
   public static MetadataStore getMetadataStore(FileSystem fs)
       throws IOException {
     Preconditions.checkNotNull(fs);

+ 15 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
@@ -960,6 +961,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     public static final String AUTH_FLAG = "auth";
     public static final String NONAUTH_FLAG = "nonauth";
     public static final String ENCRYPTION_FLAG = "encryption";
+    public static final String MAGIC_FLAG = "magic";
 
     public static final String PURPOSE = "provide/check S3Guard information"
         + " about a specific bucket";
@@ -967,11 +969,15 @@ public abstract class S3GuardTool extends Configured implements Tool {
         + "\t" + PURPOSE + "\n\n"
         + "Common options:\n"
         + "  -" + GUARDED_FLAG + " - Require S3Guard\n"
+        + "  -" + UNGUARDED_FLAG + " - Require S3Guard to be disabled\n"
+        + "  -" + AUTH_FLAG + " - Require the S3Guard mode to be \"authoritative\"\n"
+        + "  -" + NONAUTH_FLAG + " - Require the S3Guard mode to be \"non-authoritative\"\n"
+        + "  -" + MAGIC_FLAG + " - Require the S3 filesystem to be support the \"magic\" committer\n"
         + "  -" + ENCRYPTION_FLAG
         + " -require {none, sse-s3, sse-kms} - Require encryption policy";
 
     BucketInfo(Configuration conf) {
-      super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG);
+      super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG, MAGIC_FLAG);
       CommandFormat format = getCommandFormat();
       format.addOptionWithValue(ENCRYPTION_FLAG);
     }
@@ -1014,6 +1020,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
       } else {
         println(out, "Filesystem %s is not using S3Guard", fsUri);
       }
+      boolean magic = fs.hasCapability(
+          CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER);
+      println(out, "The \"magic\" committer %s supported",
+          magic ? "is" : "is not");
+
       println(out, "%nS3A Client");
 
       String endpoint = conf.getTrimmed(ENDPOINT, "");
@@ -1043,6 +1054,9 @@ public abstract class S3GuardTool extends Configured implements Tool {
           throw badState("S3Guard is not enabled for %s", fsUri);
         }
       }
+      if (commands.getOpt(MAGIC_FLAG) && !magic) {
+        throw badState("The magic committer is not enabled for %s", fsUri);
+      }
 
       String desiredEncryption = getCommandFormat()
           .getOptValue(ENCRYPTION_FLAG);

+ 1951 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md

@@ -0,0 +1,1951 @@
+<!---
+  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.
+-->
+
+# S3A Committers: Architecture and Implementation
+
+<!-- DISABLEDMACRO{toc|fromDepth=0|toDepth=5} -->
+
+This document covers the architecture and implementation details of the S3A committers.
+
+For information on using the committers, see [the S3A Committers](./committer.html).
+
+
+## Problem: Efficient, reliable commits of work to consistent S3 buckets
+
+
+The standard commit algorithms (the `FileOutputCommitter` and its v1 and v2 algorithms)
+rely on directory rename being an `O(1)` atomic operation: callers output their
+work to temporary directories in the destination filesystem, then
+rename these directories to the final destination as way of committing work.
+This is the perfect solution for commiting work against any filesystem with
+consistent listing operations and where the `FileSystem.rename()` command
+is an atomic `O(1)` operation.
+
+Using rename allows individual tasks to work in temporary directories, with the
+rename as the atomic operation can be used to explicitly commit tasks and
+ultimately the entire job. Because the cost of the rename is low, it can be
+performed during task and job commits with minimal delays. Note that HDFS
+will lock the namenode metadata during the rename operation, so all rename() calls
+will be serialized. However, as they only update the metadata of two directory
+entries, the duration of the lock is low.
+
+In contrast to a "real" filesystem, Amazon's S3A object store, similar to
+most others, does not support `rename()` at all. A hash operation on the filename
+determines the location of of the data —there is no separate metadata to change.
+To mimic renaming, the Hadoop S3A client has to copy the data to a new object
+with the destination filename, then delete the original entry. This copy
+can be executed server-side, but as it does not complete until the in-cluster
+copy has completed, it takes time proportional to the amount of data.
+
+The rename overhead is the most visible issue, but it is not the most dangerous.
+That is the fact that path listings have no consistency guarantees, and may
+lag the addition or deletion of files.
+If files are not listed, the commit operation will *not* copy them, and
+so they will not appear in the final output.
+
+The solution to this problem is closely coupled to the S3 protocol itself:
+delayed completion of multi-part PUT operations
+
+That is: tasks write all data as multipart uploads, *but delay the final
+commit action until until the final, single job commit action.* Only that
+data committed in the job commit action will be made visible; work from speculative
+and failed tasks will not be instiantiated. As there is no rename, there is no
+delay while data is copied from a temporary directory to the final directory.
+The duration of the commit will be the time needed to determine which commit operations
+to construct, and to execute them.
+
+
+## Terminology
+
+* *Job*: a potentially parallelized query/operation to execute. The execution
+of a job: the division of work into tasks and the management of their completion,
+is generally executed in a single process.
+
+The output of a Job is made visible to other stages in a larger operation
+sequence or other applications if the job *completes successfully*.
+
+* *Job Driver*. Not sure quite what term to use here. Whatever process schedules
+task execution, tracks success/failures and, determines when all the work has been
+processed and then commits the output. It may also determine that a job
+has failed and cannot be recovered, in which case the job is aborted.
+In MR and Tez, this is inside the YARN application master.
+In Spark it is the driver, which can run in the AM, the YARN client, or other
+places (e.g Livy?).
+
+* *Final directory*: the directory into which the output of a job is placed
+so as to be visible.
+
+* *Task* a single operation within a job, on a single process, one which generates
+one or more files.
+After a successful job completion, the data MUST be visible in the final directory.
+A task completes successfully if it generates all the output it expects to without
+failing in some way (error in processing; network/process failure).
+
+* *Job Context* an instance of the class `org.apache.hadoop.mapreduce.JobContext`,
+which provides a read-only view of the Job for the Job Driver and tasks.
+
+* *Task Attempt Context* an instance of the class
+`org.apache.hadoop.mapreduce.TaskAttemptContext extends JobContext, Progressable`,
+which provides operations for tasks, such as getting and setting status,
+progress and counter values.
+
+* *Task Working Directory*: a directory for exclusive access by a single task,
+into which uncommitted work may be placed.
+
+* *Task Commit* The act of taking the output of a task, as found in the
+Task Working Directory, and making it visible in the final directory.
+This is traditionally implemented via a `FileSystem.rename()` call.
+
+  It is useful to differentiate between a *task-side commit*: an operation performed
+  in the task process after its work, and a *driver-side task commit*, in which
+  the Job driver perfoms the commit operation. Any task-side commit work will
+  be performed across the cluster, and may take place off the critical part for
+  job execution. However, unless the commit protocol requires all tasks to await
+  a signal from the job driver, task-side commits cannot instantiate their output
+  in the final directory. They may be used to promote the output of a successful
+  task into a state ready for the job commit, addressing speculative execution
+  and failures.
+
+* *Job Commit* The act of taking all successfully completed tasks of a job,
+and committing them. This process is generally non-atomic; as it is often
+a serialized operation at the end of a job, its performance can be a bottleneck.
+
+* *Task Abort* To cancel a task such that its data is not committed.
+
+* *Job Abort* To cancel all work in a job: no task's work is committed.
+
+* *Speculative Task Execution/ "Speculation"* Running multiple tasks against the same
+input dataset in parallel, with the first task which completes being the one
+which is considered successful. Its output SHALL be committed; the other task
+SHALL be aborted. There's a requirement that a task can be executed in parallel,
+and that the output of a task MUST NOT BE visible until the job is committed,
+at the behest of the Job driver. There is the expectation that the output
+SHOULD BE the same on each task, though that MAY NOT be the case. What matters
+is if any instance of a speculative task is committed, the output MUST BE
+considered valid.
+
+There is an expectation that the Job Driver and tasks can communicate: if a task
+performs any operations itself during the task commit phase, it shall only do
+this when instructed by the Job Driver. Similarly, if a task is unable to
+communicate its final status to the Job Driver, it MUST NOT commit is work.
+This is very important when working with S3, as some network partitions could
+isolate a task from the Job Driver, while the task retains access to S3.
+
+## The execution workflow
+
+
+**setup**:
+
+* A job is created, assigned a Job ID (YARN?).
+* For each attempt, an attempt ID is created, to build the job attempt ID.
+* `Driver`: a `JobContext` is created/configured
+* A committer instance is instantiated with the `JobContext`; `setupJob()` invoked.
+
+
+## The `FileOutputCommitter`
+
+The standard commit protocols are implemented in
+ `org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter`.
+
+There are two algorithms, the "v1" designed to address failures and restarts
+of the MapReduce application master. The V2 algorithm cannot recover from failure
+except by re-executing the entire job. It does, however, propagate all its work
+to the output directory in the task commit. When working with object stores which
+mimic `rename()` by copy and delete, it is more efficient due to the reduced
+number of listings, copies and deletes, and, because these are executed in
+task commits, eliminates the final `O(data)` pause at the end of all work.
+It is still highly inefficient, but the inefficiencies are less visible as
+large pauses in execution.
+
+
+Notes
+
+* The v1 algorithm was implemented to handle MapReduce AM restarts, it was
+ not used in Hadoop 1.x, whose JobTracker could not recover from failures.
+ Historically then, it is the second version of a file commit algorithm.
+* Because the renames are considered to be fast, there is no logging
+of renames being in progress, or their duration.
+* Speculative execution is supported by having every task attempt write its
+uncommitted data to a task attempt directory. When a task is ready to commit,
+it must notify that job driver that it is ready to commit -the job driver
+will then commit or abort the task.
+
+## Hadoop MR Commit algorithm "1"
+
+
+The "v1" commit algorithm is the default commit algorithm in Hadoop 2.x;
+it was implemented as part of [MAPREDUCE-2702](https://issues.apache.org/jira/browse/MAPREDUCE-2702).
+
+This algorithm is designed to handle a failure and restart of the Job driver,
+with the restarted job driver only rerunning the incomplete tasks; the
+output of the completed tasks is recovered for commitment when the restarted
+job completes.
+
+There is a cost: the time to commit by recursive listing all files in all
+committed task directories, and renaming this.
+
+As this is performed sequentially, time to commit is `O(files)`, which is
+generally `O(tasks)`.
+
+
+
+```python
+#Job Attempt Path is `$dest/_temporary/$appAttemptId/`
+
+jobAttemptPath = '$dest/_temporary/$appAttemptId/'
+
+# Task Attempt Path is `$dest/_temporary/$appAttemptId/_temporary/$taskAttemptID`
+taskAttemptPath = '$dest/_temporary/$appAttemptId/_temporary/$taskAttemptID'
+
+#Task committed path is `$dest/_temporary/$appAttemptId/$taskAttemptID`
+taskCommittedPath = '$dest/_temporary/$appAttemptId/$taskAttemptID'
+```
+
+Tasks write in/under the task attempt path.
+
+### Job Setup
+
+```python
+fs.mkdir(jobAttemptPath)
+```
+
+### Task Setup
+
+None: directories are created on demand.
+
+
+### Task Commit
+
+Rename task attempt path to task committed path.
+
+```python
+
+def needsTaskCommit(fs, jobAttemptPath, taskAttemptPath, dest):
+  return fs.exists(taskAttemptPath)
+
+
+def commitTask(fs, jobAttemptPath, taskAttemptPath, dest):
+  if fs.exists(taskAttemptPath) :
+    fs.delete(taskCommittedPath, recursive=True)
+    fs.rename(taskAttemptPath, taskCommittedPath)
+```
+
+On a genuine fileystem this is an `O(1)` directory rename.
+
+On an object store with a mimiced rename, it is `O(data)` for the copy,
+along with overhead for listing and deleting all files (For S3, that's
+`(1 + files/500)` lists, and the same number of delete calls.
+
+
+### Task Abort
+
+Delete task attempt path.
+
+```python
+def abortTask(fs, jobAttemptPath, taskAttemptPath, dest):
+  fs.delete(taskAttemptPath, recursive=True)
+```
+
+On a genuine fileystem this is an `O(1)` operation. On an object store,
+proportional to the time to list and delete files, usually in batches.
+
+
+### Job Commit
+
+Merge all files/directories in all task commited paths into final destination path.
+Optionally; create 0-byte `_SUCCESS` file in destination path.
+
+```python
+def commitJob(fs, jobAttemptDir, dest):
+  for committedTask in fs.listFiles(jobAttemptDir):
+    mergePathsV1(fs, committedTask, dest)
+  fs.touch("$dest/_SUCCESS")
+```
+
+(See below for details on `mergePaths()`)
+
+
+A failure during job abort cannot be recovered from except by re-executing
+the entire query:
+
+```python
+def isCommitJobRepeatable() :
+  return True
+```
+
+Accordingly, it is a failure point in the protocol. With a low number of files
+and fast rename/list algorithms, the window of vulnerability is low. At
+scale, the vulnerability increases. It could actually be reduced through
+parallel execution of the renaming of of committed tasks.
+
+
+### Job Abort
+
+Delete all data under job attempt path.
+
+```python
+def abortJob(fs, jobAttemptDir, dest):
+  fs.delete(jobAttemptDir, recursive = True)
+```
+
+### Job Cleanup
+
+```python
+def cleanupJob(fs, dest):
+  fs.delete('$dest/_temporary', recursive = True)
+```
+
+
+### Job Recovery
+
+1. Data under task committed paths is retained
+1. All directories under `$dest/_temporary/$appAttemptId/_temporary/` are deleted.
+
+Uncommitted/unexecuted tasks are (re)executed.
+
+This significantly improves time to recover from Job driver (here MR AM) failure.
+The only lost work is that of all tasks in progress -those which had generated
+data but were not yet committed.
+
+Only the failure of the job driver requires a job restart, not an individual
+task. Therefore the probability of this happening is independent of the number
+of tasks executed in parallel, instead simply due to the duration of the query.
+
+The longer the task, the higher the risk of failure, the more value there is
+in recovering the work in progress.
+
+Fast queries not only have a lower risk of failure, they can recover from
+failure simply by rerunning the entire job. This is implicitly the strategy
+in Spark, which does not attempt to recover any in-progress jobs. The faster
+your queries, the simpler your recovery strategy needs to be.
+
+### `mergePaths(FileSystem fs, FileStatus src, Path dest)` Algorithm
+
+`mergePaths()` is the core algorithm to merge data; it is somewhat confusing
+as the implementation mixes the strategies for both algorithms across
+two co-recursive routines, `mergePaths()` and `renameOrMerge()`.
+
+
+Here the two algorithms have been split, and one of the co-recursive methods
+inlined.
+
+```python
+def mergePathsV1(fs, src, dest) :
+  if fs.exists(dest) :
+    toStat = fs.getFileStatus(dest)
+  else:
+    toStat = None
+
+  if src.isFile :
+    if not toStat is None :
+      fs.delete(dest, recursive = True)
+    fs.rename(src.getPath, dest)
+  else :
+    # destination is directory, choose action on source type
+    if src.isDirectory :
+      if not toStat is None :
+        if not toStat.isDirectory :
+          # Destination exists and is not a directory
+          fs.delete(dest)
+          fs.rename(src.getPath(), dest)
+        else :
+          # Destination exists and is a directory
+          # merge all children under destination directory
+          for child in fs.listStatus(src.getPath) :
+            mergePathsV1(fs, child, dest + child.getName)
+      else :
+        # destination does not exist
+        fs.rename(src.getPath(), dest)
+```
+
+## v2 commit algorithm
+
+
+The v2 algorithm directly commits task output into the destination directory.
+It is essentially a re-implementation of the Hadoop 1.x commit algorithm.
+
+1. During execution, intermediate data becomes visible.
+1. On a failure, all output must be deleted and the job restarted.
+
+It implements `mergePaths` differently, as shown below.
+
+```python
+def mergePathsV2(fs, src, dest) :
+  if fs.exists(dest) :
+    toStat = fs.getFileStatus(dest)
+  else:
+    toStat = None
+
+  if src.isFile :
+    if not toStat is None :
+      fs.delete(dest, recursive = True)
+    fs.rename(src.getPath, dest)
+  else :
+    # destination is directory, choose action on source type
+    if src.isDirectory :
+      if not toStat is None :
+        if not toStat.isDirectory :
+          # Destination exists and is not a directory
+          fs.delete(dest)
+          fs.mkdirs(dest)                                    #
+          for child in fs.listStatus(src.getPath) :          # HERE
+            mergePathsV2(fs, child, dest + child.getName)    #
+
+        else :
+          # Destination exists and is a directory
+          # merge all children under destination directory
+          for child in fs.listStatus(src.getPath) :
+            mergePathsV2(fs, child, dest + child.getName)
+      else :
+        # destination does not exist
+        fs.mkdirs(dest)                                     #
+        for child in fs.listStatus(src.getPath) :           # HERE
+          mergePathsV2(fs, child, dest + child.getName)     #
+```
+
+Both recurse down any source directory tree, and commit single files
+by renaming the files.
+
+A a key difference is that the v1 algorithm commits a source directory to
+via a directory rename, which is traditionally an `O(1)` operation.
+
+In constrast, the v2 algorithm lists all direct children of a source directory
+and recursively calls `mergePath()` on them, ultimately renaming the individual
+files. As such, the number of renames it performa equals the number of source
+*files*, rather than the number of source *directories*; the number of directory
+listings being `O(depth(src))` , where `depth(path)` is a function returning the
+depth of directories under the given path.
+
+On a normal filesystem, the v2 merge algorithm is potentially more expensive
+than the v1 algorithm. However, as the merging only takes place in task commit,
+it is potentially less of a bottleneck in the entire execution process.
+
+On an objcct store, it is suboptimal not just from its expectation that `rename()`
+is an `O(1)` operation, but from its expectation that a recursive tree walk is
+an efficient way to enumerate and act on a tree of data. If the algorithm was
+switched to using `FileSystem.listFiles(path, recursive)` for a single call to
+enumerate all children under a path, then the listing operation would be significantly
+faster, at least on a deep or wide tree. However, for any realistic dataset,
+the size of the output files is likely to be the main cause of delays. That
+is, if the cost of `mergePathsV2` is `O(depth(src)) + O(data))`, then
+generally the `O(data)` value will be more significant than the `depth(src)`.
+
+There is one key exception: tests which work on small amounts of data yet try
+to generate realistic output directory structures. In these tests the cost
+of listing directories and calling `getFileStatus()` could exceed that of the copy
+calls. This is why small-scale tests of the commit algorithms against object stores
+must be considered significantly misleading.
+
+### v2 Task Commit
+
+Rename task attempt path to task committed path.
+
+```python
+def needsTaskCommit(fs, jobAttemptPath, taskAttemptPath, dest):
+  return fs.exists(taskAttemptPath)
+
+def commitTask(fs, jobAttemptPath, taskAttemptPath, dest):
+  if fs.exists(taskAttemptPath) :
+    mergePathsV2(fs. taskAttemptPath, dest)
+```
+
+### v2 Task Abort
+
+Delete task attempt path.
+
+```python
+def abortTask(fs, jobAttemptPath, taskAttemptPath, dest):
+  fs.delete(taskAttemptPath, recursive=True)
+```
+
+Cost: `O(1)` for normal filesystems, `O(files)` for object stores.
+
+
+### v2 Job Commit
+
+As all the task output is already completed, all that is needed is
+to touch the `_SUCCESS` marker.
+
+```python
+def commitJob(fs, jobAttemptDir, dest):
+  fs.touch("$dest/_SUCCESS")
+```
+Cost: `O(1)`
+
+A failure during job abort is implicitly repeatable
+
+```python
+def isCommitJobRepeatable() :
+  return True
+```
+
+### v2 Job Abort
+
+Delete all data under job attempt path.
+
+```python
+def abortJob(fs, jobAttemptDir, dest):
+  fs.delete(jobAttemptDir, recursive=True)
+```
+
+Cost: `O(1)` for normal filesystems, `O(files)` for object stores.
+
+### v2 Task Recovery
+
+As no data is written to the destination directory, a task can be cleaned up
+by deleting the task attempt directory.
+
+### v2 Job Recovery
+
+Because the data has been renamed into the destination directory, it is nominally
+recoverable. However, this assumes that the number and name of generated
+files are constant on retried tasks.
+
+
+## How MapReduce uses the committer in a task
+
+MapReduce runs each Mapper or Reducer in its own container; it
+gets its own process. Implicitly, this also means that each task gets
+its own instance of every filesystem.
+
+The work is choreographed in `org.apache.hadoop.mapred.Task`, for which there
+are specific subclassing for the Map (`MapTask`) and reduce (`ReduceTask`).
+There are also cleanup tasks whose role is simpler: clean things up,
+and a Job setup task which runs `OutputCommittre.setupJob`. That is: even
+the job setup phase is run in a Task.
+
+MapTask uses the committer to write the output of all the mappers into
+the filesystem, ready for the reducers. Each partition writes its data
+as a `MapFile`, which is actually two `SequenceFile` files in a directory,:
+the `data` file of all Key-Value
+output, and `index` which contains an index of some of the keys in the file.
+
+This is all written to the local filesystem.
+
+The `ReduceTask` does the final write to the destination filesystem.
+
+Because the Map phase uses a committer to commits intermediate work,
+any plug in committer supplied to a process
+through any extension mechanism *must* work with the output generated by a mapper.
+The staging committers do only if unique filenames are disabled, but as they
+and the magic committers are only meant to be used for the final output of work,
+it is somewhat moot. What is important is to be able to use different committers
+for the map phase as the final reduce. This is implicit if separate committers
+are defined for different filesystems: the new committer can be defined for
+the final destination FS, while `file://` can retain the default
+`FileOutputCommitter`.
+
+### Task Setup
+
+`Task.initialize()`: read in the configuration, instantate the `JobContextImpl`
+and `TaskAttemptContextImpl` instances bonded to the current job & task.
+
+### Task Ccommit
+
+After the work is completed, `Task.done()` is invoked. Which is essentially the following
+codepath:
+
+```java
+if (committer.needsTaskCommit(taskContext)) {
+  // get permission to commit from AM
+  int retries = MAX_RETRIES;
+  while(true) {
+    try {
+      umbilical.commitPending(taskId, taskStatus);
+      break;
+    } catch(IOException ie) {
+      if (--retries == 0) {
+        // FAIL WITHOUT CALLING ABORT
+        System.exit(67);
+      }
+    }
+  }
+  // commit the work
+  try {
+    committer.commitTask(taskContext);
+  } catch (IOException iee) {
+    // failure: abort
+    try {
+      committer.abortTask(taskContext);
+    } catch (IOException ioe)  {
+      LOG.warn("Failure cleaning up: " +
+               StringUtils.stringifyException(ioe));
+    }
+    throw iee;
+  }
+}
+```
+
+That is: if and only if there is data to write, the Task requests clearance
+to do so from the AM. This ensures that speculative work is not committed if
+another task has written it, *and that even non-speculative work is not committed
+if the task has lost contact with the AM*.
+
+That is: task output is not committed if the Job's AM has failed or
+is in a separate network partition from the Task.
+
+If permission is granted, the commit call is invoked.
+If this doesn't work, `abortTask()` is called; all
+failures there are logged and swallowed.
+
+
+This method actually appears to be limited in that it does not guarantee
+that `committer.abortTask()` is always called: if the `umbilical.commitPending()` calls
+fail repeatedly and the Task process aborts, `committer.abortTask()` is not called.
+If this is due to a network partition, we can hope that the AM invokes
+`committer.abortTask()`, and that if it is an AM failure then a restarted AM can
+clean up its previous attempts. For the classic FileOutputCommitter, listing and
+deleting the previous attempt's data is straightforward. However, for S3 committers
+using Multipart Upload as the means of uploading uncommitted data, it is critical
+to ensure that pending uploads are always aborted. This can be done by
+
+* Making sure that all task-side failure branvches in `Task.done()` call `committer.abortTask()`.
+* Having job commit & abort cleaning up all pending multipart writes to the same directory
+tree. That is: require that no other jobs are writing to the same tree, and so
+list all pending operations and cancel them.
+* Add a CLI command to verify that there are no commits under a path; returning
+a non-zero exit code if there are. This can be used in testing.
+* Add a CLI command to purge commits under a path. This can be used in workflows,
+if felt necessary, and in test runs to isolate the output of different tests.
+* Document a recommended purge-pending-commit timeout for the system
+* Add probes in the integration tests to verify that pending uploads are never
+present after a write.
+
+
+
+### How MapReduce uses the committer in the Application Master
+
+The AM uses a committer to set up and commit the job. To support failure
+and recovery of the AM, `OutputCommitter.isRecoverySupported()`  is used
+to declare whether all the output of successful tasks can be used in the final
+job, or the entire job needs to be reset and repeated.
+`OutputCommitter.isCommitJobRepeatable()` addresses the other question:
+can a committer recover from a failure of the commit process itself.
+
+A committer is created in the Application Master, and handed to an instance
+of `org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler`,
+which then manages the job-level lifecycle.
+
+If the MRv1 API is used, the committer is chosen from the value of
+`"mapred.output.committer.class"`; in the MRv2 API the output format
+is instantiated, then asked for a committer using a task and task attempt ID of
+0. A committer is obtained from the output format via a call to
+`Committer.getOutputCommitter(taskContext)`, again using the task attempt context
+with the (job, 0, 0) task and attempt IDs. That is: even for the job committers,
+a task context is always passed in to the `OutputFormat` when requesting a committer.
+*It is critical for all implementations of `OutputCommitter.abortTask()` to
+be able to execute from the AM, rather than the container and host running
+the task. Furthermore, all information needed for the abort (paths, filesystem instances
+&c) *must* be retrieved from the `TaskAttemptContext` passed to the method,
+rather than relying on fields initiated from the context passed to the constructor.
+
+
+#### AM: Job setup: `OutputCommitter.setupJob()`
+
+This is initated in `org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.StartTransition`.
+It is queued for asynchronous execution in `org.apache.hadoop.mapreduce.v2.app.MRAppMaster.startJobs()`,
+which is invoked when the service is started. Thus: the job is set up when the
+AM is started.
+
+
+#### AM: Job Commit: `OutputCommitter.commitJob()`
+
+In the "default"/cluster filesystem, the `CommitterEventHandler` uses data in
+the staging area defined in `yarn.app.mapreduce.am.staging-dir`
+(default `/tmp/hadoop-yarn/staging/$user/.staging`), in a subdirectory
+named from the job ID.
+
+Three paths are built up for using the filesystem for tracking the state
+of the commit process, with a goal of making the commit operation recoverable,
+where supported.
+
+
+| name |  role |
+|------|--------|
+| `COMMIT_STARTED` | mark the commencement of the job commit |
+| `COMMIT_SUCCESS` | mark the successful completion of the job commit |
+| `COMMIT_FAIL` | mark the failure of the job commit |
+
+These markers are used to manage job restart/failure of they happen during
+the job commit itself.
+
+When an AM starts up, it looks in its staging area for these files. as a way
+to determine the previous state of the job. If there are no `COMMIT_` marker files,
+the job is considered not to have attempted to commit itself yet.
+
+
+The presence of `COMMIT_SUCCESS` or `COMMIT_FAIL` are taken as evidence
+that the previous job completed successfully or unsucessfully; the AM
+then completes with a success/failure error code, without attempting to rerun
+the job.
+
+If `COMMIT_STARTED` exists but not either of the completion markers, then,
+if the committer declares that its job commit operation is repeatable
+(`Committer.isCommitJobRepeatable(jobContext) == true`), then an attempt
+is made to recommit the job, deleting the `COMMIT_STARTED` and commencing
+the commit process again.
+
+These `COMMIT_STARTED` files are simply 0-byte files, but are created
+with the overwrite bit only set to true if the job commit is considered
+repeatable:
+
+```java
+private void touchz(Path p, boolean overwrite) throws IOException {
+  fs.create(p, overwrite).close();
+}
+```
+That is: the atomicity of the `create(path, overwrite=false)` on the cluster
+filesystem is used to guarantee that only one process will attempt to commit
+a specific job.
+
+```java
+boolean commitJobIsRepeatable = committer.isCommitJobRepeatable(
+      event.getJobContext());
+try {
+  touchz(startCommitFile, commitJobIsRepeatable);
+  waitForValidCommitWindow();
+  committer.commitJob(event.getJobContext());
+  touchz(endCommitSuccessFile, commitJobIsRepeatable);
+} catch (Exception e) {
+  touchz(endCommitFailureFile, commitJobIsRepeatable);
+}
+```
+
+The `waitForValidCommitWindow()` operation is important: it declares that
+the committer must not commit unless there has been communication with the YARN
+Resource Manager with in `yarn.app.mapreduce.am.job.committer.commit-window` milliseconds
+(default: 10,000). It does this by waiting until the next heartbeat it received.
+There's a possible bug here: if the interval is set too small the thread may
+permanently spin waiting a callback within the window. Ignoring that, this algorithm
+guarantees that
+
+1. As only one call can create a file with `overwrite=false`,
+   only one process's attempt to commit a non-repeatable job will proceed
+
+1. Only a process with contact with the YARN within the configured window
+   may commit a job.
+
+1. If the AM is partitioned from the rest of the network, provided that its clock
+is monotonically increasing at the same rate as the rest of the cluster, then
+the rest of the cluster can be confident that
+`yarn.app.mapreduce.am.ob.committer.commit-window` milliseconds after the AM
+successfully heartbeated to the YARN RM, then the output of this job attempt
+will *never* be committed. This permits 1 job to run simultaneously, but
+helps ensure that only one of them will attempt to commit.
+
+1. Provided YARN heartbeats are only sent to the AM which successfully created
+the `COMMIT_STARTED` file, it will initiate the commit operation.
+
+Possible issues with this algorithm:
+
+* The `COMMIT_STARTED` file is created before waiting to get a heartbeat. It
+may be that this AM has lost contact with YARN, but doesn't know it yet. When the
+YARN liveness protocols eventually time out, the AM will correctly terminate,
+but as the `COMMIT_STARTED` file has been created at this point, no other launched
+AM will be able to commit.
+
+* If two committers attempt to create `COMMIT_STARTED` files on a no-repeatable
+commit, one will succeed, wait for a heartbeat then attempt a (possibly slow) commit.
+The second committer will fail, and will *immediately* create a `COMMIT_FAILED` file.
+As a result, the state of the staging area will imply that the commit has failed,
+when really it is in progress, and that only the second process failed.
+
+It would seem safer to address this through
+1. Waiting for a heartbeat before creating the `COMMIT_STARTED` file.
+1. Maybe: not creating the `COMMIT_FAILED` file if the failure happens when
+trying to create the `COMMIT_STARTED` file. That is: only a process which
+successfully created the `COMMIT_STARTED` file may indicate that a commit has failed.
+
+
+### AM: Cancelling job commit
+
+The thread performing the commit is interrupted; the `CommitterEventHandler`
+awaits for it to finish. (set in `yarn.app.mapreduce.am.job.committer.cancel-timeout`
+as milliseconds).
+
+### AM: Task Abort
+
+The AM may call the `OutputCommitter.taskAbort()` with a task attempt context,
+when handling the failure/loss of a container. That is: on container failure,
+the task abort operation is executed in the AM, using the AM's committer.
+This avoids the need to create a new container, and means that the "best-effort"
+task abort does cope with container failures.
+
+A partition between the AM and the task container means that this AM-executed
+task abort may take place while a task in the partitioned container is still
+executing. Unless output writing operations will fail after the abort operation,
+the partitioned task may not become aware of the partition until it's own task
+commit sequence in `Task.done()`, when `talkToAMTGetPermissionToCommit()`
+
+# Requirements of an S3A Committer
+
+1. Support an eventually consistent S3 object store as a reliable direct
+destination of work through the S3A filesystem client.
+1. Efficient: implies no rename, and a minimal amount of delay in the job driver's
+task and job commit phases,
+1. Support task failure and speculation.
+1. Can be used by existing code: Hadoop MapReduce, Spark, Hive.
+1. Retrofittable to existing subclasses of FileOutputFormat and/or compatible
+with committers which expect a specific FileOutputFormat.
+1. Clean up uncommitted data from all task attempts, all previous attempts of
+the job, and any previous incompleted jobs.
+1. Security: not to permit privilege escalation from other users with
+write access to the same file system(s).
+
+
+## Features of S3 and the S3A Client
+
+
+A core problem is that
+[object stores are not filesystems](../../../hadoop-project-dist/hadoop-common/filesystem/introduction.html);
+how `rename()` has been emulated in the S3A client means that both the existing
+MR committer algorithms have significant performance problems.
+
+1. Single-object renames are implemented as a copy and delete sequence.
+1. COPY is atomic, but overwrites cannot be prevented.
+1. Amazon S3 is eventually consistent on listings, deletes and updates.
+1. Amazon S3 has create consistency, however, the negative response of a HEAD/GET
+performed on a path before an object was created can be cached, unintentionally
+creating a create inconsistency. The S3A client library does perform such a check,
+on `create()` and `rename()` to check the state of the destination path, and
+so, whether the operation is permitted.
+1. Multi-object renames are sequential or parallel single object COPY+DELETE operations:
+non atomic, `O(data)` and, on failure, can leave the filesystem in an unknown
+state.
+1. There is a PUT operation, capable of uploading 5GB of data in one HTTP request.
+1. The PUT operation is atomic, but there is no PUT-no-overwrite option.
+1. There is a multipart POST/PUT sequence for uploading larger amounts of data
+in a sequence of PUT requests.
+
+
+The Hadoop S3A Filesystem client supports PUT and multipart PUT for uploading
+data, with the `S3ABlockOutputStream` of HADOOP-13560 uploading written data
+as parts of a multipart PUT once the threshold set in the configuration
+parameter `fs.s3a.multipart.size` (default: 100MB).
+
+[S3Guard](./s3guard.html) adds an option of consistent view of the filesystem
+to all processes using the shared DynamoDB table as the authoritative store of
+metadata. Some S3-compatible object stores are fully consistent; the
+proposed algorithm is designed to work with such object stores without the
+need for any DynamoDB tables.
+
+## Related work: Spark's `DirectOutputCommitter`
+
+One implementation to look at is the
+[`DirectOutputCommitter` of Spark 1.6](https://github.com/apache/spark/blob/branch-1.6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/DirectParquetOutputCommitter.scala).
+
+This implements a zero rename commit by subclassing the `ParquetOutputCommitter`
+and then
+
+1. Returning the destination directory as the task working directory.
+1. Subclassing all the task commit/abort operations to be no-ops.
+
+With the working directory as the destination directory, there is no need
+to move/rename the task output on a successful commit. However, it is flawed.
+There is no notion of "committing" or "aborting" a task, hence no ability to
+handle speculative execution or failures. This is why the committer
+was removed from Spark 2 [SPARK-10063](https://issues.apache.org/jira/browse/SPARK-10063)
+
+There is also the issue that work-in-progress data is visible; this may or may
+not be a problem.
+
+## Related work: IBM's "Stocator" committer
+
+IBM's [Stocator](https://github.com/SparkTC/stocator) can transform indirect
+writes of V1/V2 committers into direct writes to the destination directory.
+
+Hpw does it do this? It's a special Hadoop `FileSystem` implementation which
+recognizes writes to `_temporary` paths and translate them to writes to the
+base directory. As well as translating the write operation, it also supports
+a `getFileStatus()` call on the original path, returning details on the file
+at the final destination. This allows for committing applications to verify
+the creation/existence/size of the written files (in contrast to the magic
+committer covdered below).
+
+The FS targets Openstack Swift, though other object stores are supportable through
+different backends.
+
+This solution is innovative in that it appears to deliver the same semantics
+(and hence failure modes) as the Spark Direct OutputCommitter, but which
+does not need any changs in either Spark *or* the Hadoop committers. In contrast,
+the committers proposed here combines changing the Hadoop MR committers for
+ease of pluggability, and offers a new committer exclusivley for S3, one
+strongly dependent upon and tightly integrated with the S3A Filesystem.
+
+The simplicity of the Stocator committer is something to appreciate.
+
+## Background: The S3 multi-part PUT mechanism
+
+In the [S3 REST API](http://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html),
+multipart uploads allow clients to upload a series of "Parts" of a file,
+then commit the upload with a final call.
+
+1. Caller initiates a multipart request, including the destination bucket, key
+and metadata.
+
+        POST bucket.s3.aws.com/path?uploads
+
+    An UploadId is returned
+
+1. Caller uploads one or more parts.
+
+        PUT bucket.s3.aws.com/path?partNumber=PartNumber&uploadId=UploadId
+
+    The part number is used to declare the ordering of the PUTs; they
+    can be uploaded in parallel and out of order.
+    All parts *excluding the final part* must be 5MB or larger.
+    Every upload completes with an etag returned
+
+1. Caller completes the operation
+
+        POST /ObjectName?uploadId=UploadId
+        <CompleteMultipartUpload>
+          <Part><PartNumber>(number)<PartNumber><ETag>(Tag)</ETag></Part>
+          ...
+        </CompleteMultipartUpload>
+
+    This final call lists the etags of all uploaded parts and the actual ordering
+    of the parts within the object.
+
+The completion operation is apparently `O(1)`; presumably the PUT requests
+have already uploaded the data to the server(s) which will eventually be
+serving up the data for the final path. All that is needed to complete
+the upload is to construct an object by linking together the files in
+the server's local filesystem and udate an entry the index table of the
+object store.
+
+In the S3A client, all PUT calls in the sequence and the final commit are
+initiated by the same process. *This does not have to be the case*.
+It is that fact, that a different process may perform different parts
+of the upload, which make this algorithm viable.
+
+
+## The Netfix "Staging" committer
+
+Ryan Blue, of Netflix, has submitted an alternate committer, one which has a
+number of appealing features
+
+* Doesn't have any requirements of the destination object store,
+* Known to work.
+
+The final point is not to be underestimated, es not even
+a need for a consistency layer.
+* Overall a simpler design.pecially given the need to
+be resilient to the various failure modes which may arise.
+
+
+The commiter writes task outputs to a temporary directory on the local FS.
+Task outputs are directed to the local FS by `getTaskAttemptPath` and `getWorkPath`.
+On task commit, the committer enumerates files in the task attempt directory (ignoring hidden files).
+Each file is uploaded to S3 using the [multi-part upload API](http://docs.aws.amazon.com/AmazonS3/latest/dev/mpuoverview.html),
+
+The information needed to commit the upload is saved into HDFS and committed
+via that protocol: when the job commits, the pending uploads of the successful
+tasks are all committed.
+
+
+### Commit logic
+
+The core algorithm is as follows:
+
+1. The destination directory for output (e.g. `FileOutputFormat` and subclasses)
+is a local `file://` reference.
+1. Task commit initiates the multipart PUT to the destination object store.
+1. A list of every pending PUT for task is persisted to a single file
+within a consistent, cluster-wide filesystem. For Netflix, that is HDFS.
+1. The Standard `FileOutputCommitter` (algorithm 1) is used to manage the commit/abort of these
+files. That is: it copies only those lists of files to commit from successful tasks
+into a (transient) job commmit directory.
+1. The S3 job committer reads the pending file list for every task committed
+in HDFS, and completes those put requests.
+
+By using `FileOutputCommmitter` to manage the propagation of the lists of files
+to commit, the existing commit algorithm implicitly becomes that defining which
+files will be committed at the end of the job.
+
+
+The Netflix contribution has Hadoop `OutputCommitter` implementations for S3.
+
+There are 3 main classes:
+* `S3MultipartOutputCommitter` is a base committer class that handles commit logic. This should not be used directly.
+* `S3DirectoryOutputCommitter` for writing unpartitioned data to S3 with conflict resolution.
+* `S3PartitionedOutputCommitter` for writing partitioned data to S3 with conflict resolution.
+
+Callers should use `S3DirectoryOutputCommitter` for single-directory outputs,
+or `S3PartitionedOutputCommitter` for partitioned data.
+
+
+These S3 committers work by writing task outputs to a temporary directory on the local FS.
+Task outputs are directed to the local FS by `getTaskAttemptPath` and `getWorkPath`.
+
+
+### Conflict Resolution
+
+The single-directory and partitioned committers handle conflict resolution by
+checking whether target paths exist in S3 before uploading any data.
+There are 3 conflict resolution modes, controlled by setting `fs.s3a.committer.staging.conflict-mode`:
+
+* `fail`: Fail a task if an output directory or partition already exists. (Default)
+* `append`: Upload data files without checking whether directories or partitions already exist.
+* `replace`: If an output directory exists, delete it so the new data replaces the current content.
+
+The partitioned committer enforces the conflict mode when a conflict is detected with output data, not before the job runs.
+Conflict resolution differs from an output mode because it does not enforce the mode when there is no conflict.
+For example, overwriting a partition should remove all sub-partitions and data it contains, whether or not new output is created.
+Conflict resolution will only replace partitions that have output data.
+
+When the conflict mode is `replace`, conflicting directories are removed during
+job commit. Data is only deleted if all tasks have completed successfully.
+
+A UUID that identifies a write is added to filenames that are uploaded to S3.
+This allows rolling back data from jobs that fail during job commit (see failure cases below) and avoids
+file-level conflicts when appending data to existing directories.
+
+
+*Note* the checks for existence are made via `S3AFileSystem.getFileStatus()` requests of the destination paths.
+Unless the view of the S3 store is consistent, it may be that a newly-deleted object
+is still discovered in the probe, so a commit fail, even when there is no longer any actual conflict.
+
+### Performance
+
+Compared to the previous proposal, henceforth the "magic" committer, this
+committer, the "staging committer", adds the extra overhead of uploading
+each file at the end of every task. This is an `O(data)` operation; it can be
+parallelized, but is bounded by the bandwidth from compute node to S3, as
+well as the write/IOP capacity of the destination shard of S3. If many tasks
+complete at or near the same time, there may be a peak of bandwidth load
+slowing down the upload.
+
+Time to commit will be the same, and, given the Netflix committer has already
+implemented the paralellization logic here, a time of `O(files/threads)`.
+
+### Resilience
+
+There's already a lot of code in the task and job commits to handle failure.
+
+Any failure in a commit triggers a best-effort abort/revert of the commit
+actions for a task/job.
+
+Task commits delegate to the `FileOutputCommitter` to ensure that only one task's
+output reaches the job commit.
+
+Similarly, if a task is aborted, temporary output on the local FS is removed.
+
+If a task dies while the committer is running, it is possible for data to be
+eft on the local FS or as unfinished parts in S3.
+Unfinished upload parts in S3 are not visible to table readers and are cleaned
+up following the rules in the target bucket's life-cycle policy.
+
+Failures during job commit are handled by deleting any files that have already
+been completed and aborting the remaining uploads.
+Because uploads are completed individually, the files that are deleted were visible to readers.
+
+If the process dies while the job committer is running, there are two possible failures:
+
+1. Some directories that would be replaced have been deleted, but no new data is visible.
+2. Some new data is visible but is not complete, and all replaced directories have been removed.
+ Only complete files are visible.
+
+If the process dies during job commit, cleaning up is a manual process.
+File names include a UUID for each write so that files can be identified and removed.
+
+
+**Failure during task execution**
+
+All data is written to local temporary files; these need to be cleaned up.
+
+The job must ensure that the local (pending) data is purged.
+
+**Failure during task commit**
+
+
+A process failure during the upload process will result in the
+list of pending multipart PUTs to *not* be persisted to the cluster filesystem.
+This window is smaller than the entire task execution, but still potentially
+significant, at least for large uploads.
+
+Per-file persistence, or incremental overwrites of the upload list may
+reduce the problems here, but there would still be a small risk of
+an outstanding multipart upload not being recorded
+
+**Explicit Task abort before task commit**
+
+Task will delete all local data; no uploads will be initiated.
+
+**Failure to communicate with S3 during data upload**
+
+If an upload fails, tasks will:
+
+* Retry using the retry policies implemented in the S3AFileSystem classes
+and the AWS libraries.
+* Eventually: attempt to abort outstanding multipart uploads already initiated.
+* Remove temporary files on the local FS.
+
+
+**Explicit Job Abort**
+
+All in-progress tasks are aborted and cleaned up. The pending commit data
+of all completed tasks can be loaded, outstanding multipart PUT requests aborted.
+
+
+This is done by
+
+1. Listing all local files, with a best effort read attempt followed by
+an abort of all successfully read files.
+1. List and abort all pending multipart uploads.
+
+Because of action #2, action #1 is superflous. It is retained so as to leave
+open the option of making action #2 a configurable option -which would be
+required to handle the use case of >1 partitioned commit running simultaneously/
+
+**Job Driver failure before Job Commit**
+
+Because the local data is managed with the v1 commit algorithm, the
+second attempt of the job will recover all the outstanding commit data
+of the first attempt; those tasks will not be rerun.
+
+This also ensures that on a job abort, the invidual tasks' .pendingset
+files can be read and used to initiate the abort of those uploads.
+That is: a recovered job can clean up the pending writes of the previous job
+
+If the query engine does not support multiple job attempts, then the
+pending commit data will not be recovered; an explicit abort operation will
+need to be initiated (we will add a CLI command for this), or the S3 bucket
+must be configured to automatically delete the pending request.
+
+**Job Driver failure during Job Commit**
+
+Those uploads already executed by a failed job commit will persist; those
+yet to execute will remain outstanding.
+
+The committer currently declares itself as non-recoverble, but that
+may not actually hold, as the recovery process could be one of:
+
+1. Enumerate all job commits from the .pendingset files (*:= Commits*).
+1. List all outstanding uploads under the destination path (*:= Outstandings*)..
+1. List all written files (for better consistency, via a GET call on the known
+filenames)
+1. Identify all files which are yet to be completed (*Commits - Written*)/
+1. Verify that the set of pending uploads matches (*Outstanding = (Commits - Written)*)
+
+The main problem here is the progress of the job-commit-time conflict resolution
+process: how to determine if it completed, as the only way to be confident
+that all files in the destination directory are to be retained is by knowing
+that the pre-commit phase completed. This could be implicitly determined
+based on the rule "no uploads are committed until precommit is completed".
+If it can be determined that 1+ upload has completed, then it could be inferred
+that precommit had completed and so the job could be repeated.
+
+This is dangerous territory to delve into. For now, the committer declares
+itself as unrecoverable.
+
+**Entire application failure before any task commit**
+
+Data is left on local systems, in the temporary directories. This may
+not be cleaned up.
+
+**Entire application failure after one or more task commits, before job commit**
+
+* A multipart PUT request will be outstanding for every pending write.
+* A temporary directory in HDFS will list all known pending requests.
+
+**Job complete/abort after >1 task failure**
+
+1. All pending put data listed in the job completion directory needs to be loaded
+and then cancelled.
+1. Any other pending writes to the dest dir need to be enumerated and aborted.
+This catches the situation of a task failure before the output is written.
+1. All pending data in local dirs need to be deleted.
+
+Issue: what about the destination directory: overwrite or not? It could well
+depend upon the merge policy.
+
+
+#### Overall Resilience
+
+1. The only time that incomplete work will appear in the destination directory
+is if the job commit operation fails partway through.
+1. There's a risk of leakage of local filesystem data; this will need to
+be managed in the response to a task failure.
+1. There's a risk of uncommitted multipart PUT operations remaining outstanding,
+operations which will run up bills until cancelled. (as indeed, so does the Magic Committer).
+
+
+For cleaning up PUT commits, as well as scheduled GC of uncommitted writes, we
+may want to consider having job setup list and cancel all pending commits
+to the destination directory, on the assumption that these are from a previous
+incomplete operation.
+
+We should add a "commit" command to the S3guard CLI to probe for, list and abort pending requests under
+a path, e.g. `--has-pending <path>`, `--list-pending <path>`, `--abort-pending <path>`.
+
+
+
+## The "Magic" Committer
+
+Development on this committer began before Netflix donated their committer.
+
+By making changes to the `S3AFileSystem` and the `S3ABlockOutputStream`, this committer
+manages to postpone the completion of writes of all files written to special
+("magic") directories; the final destination of the write being altered to
+that of the final job destination. When the job is committed, the pending
+writes are instantiated.
+
+With the addition of the Netflix Staging committer, the actual committer
+code now shares common formats for the persistent metadadata and shared routines
+for parallel committing of work, including all the error handling based on
+the Netflix experience.
+
+It differs in that it directly streams data to S3 (there is no staging),
+and it also stores the lists of pending commits in S3 too. That mandates
+consistent metadata on S3, which S3Guard provides.
+
+
+### Core concept: A new/modified output stream for delayed PUT commits
+
+
+This algorithm uses a modified `S3ABlockOutputStream`  Output stream, which, rather
+than commit any active multipart upload in the final `close()` operation,
+it instead saves enough information into the S3 repository for an independent
+process to be able to complete or abort the upload.
+
+Originally, in `OutputStream.close()`, it chose whether to perform a single PUT or to
+complete an ongoing multipart write.
+
+If a multipart PUT is in progress, then the stream waits for the ongoing uploads
+to complete (including any final block submitted), and then builds and PUTs
+the final multipart commit operation. The list of parts (and their ordering)
+has been built up during the opt
+
+In contrast, when writing to a delayed-commit file:
+
+1. A multipart write MUST always be initiated, even for small writes. This write
+MAY be initiated during the creation of the stream.
+
+1. Instead of committing the write in the `close()` call, perform a PUT to
+a path in the S3A repository with all the information needed to commit the operation.
+That is: the final path, the multipart upload ID, and the ordered list of etags
+for the uploaded parts.
+
+
+Recognising when a file is "special" is problematic; the normal `create(Path, Boolean)`
+call must recognize when the file being created is to be a delayed-commit file,
+so returning the special new stream.
+
+
+
+This is done with a "magic" temporary directory name, `__magic`, to indicate that all files
+created under this path are not to be completed during the stream write process.
+Directories created under the path will still be created —this allows job- and
+task-specific directories to be created for individual job and task attempts.
+
+For example, the pattern `__magic/${jobID}/${taskId}` could be used to
+store pending commits to the final directory for that specific task. If that
+task is committed, all pending commit files stored in that path will be loaded
+and used to commit the final uploads.
+
+Consider a job with the final directory `/results/latest`
+
+ The intermediate directory for the task 01 attempt 01 of job `job_400_1` would be
+
+    /results/latest/__magic/job_400_1/_task_01_01
+
+This would be returned as the temp directory.
+
+When a client attempted to create the file
+`/results/latest/__magic/job_400_1/task_01_01/latest.orc.lzo` , the S3A FS would initiate
+a multipart request with the final destination of `/results/latest/latest.orc.lzo`.
+
+As data was written to the output stream, it would be incrementally uploaded as
+individual multipart PUT operations
+
+On `close()`, summary data would be written to the file
+`/results/latest/__magic/job400_1/task_01_01/latest.orc.lzo.pending`.
+This would contain the upload ID and all the parts and etags of uploaded data.
+
+
+#### Task commit
+
+The information needed to commit a task is moved from the task attempt
+to the job attempt.
+
+1. The task commit operation lists all `.pending` files in its attempt directory.
+1. The contents are loaded into a list of single pending uploads.
+1. These are merged into to a single `Pendingset` structure.
+1. Which is saved to a `.pendingset` file in the job attempt directory.
+1. Finally, the task attempt directory is deleted. In the example, this
+would be to `/results/latest/__magic/job400_1/task_01_01.pendingset`;
+
+
+A failure to load any of the single pending upload files (i.e. the file
+could not load or was considered invalid, the task is considered to
+have failed. All successfully loaded pending commits will be aborted, then
+the failure reported.
+
+Similarly, a failure to save the `.pendingset` file will trigger an
+abort of all its pending uploads.
+
+
+#### Job Commit
+
+The job committer loads all `.pendingset` files in its job attempt directory.
+
+A failure to load any of these files is considered a job failure; all
+pendingsets which could be loaded will be aborted.
+
+If all pendingsets were loaded, then every
+pending commit in the job will be committed. If any one of these commits
+failed, then all successful commits will be reverted by deleting the destination
+file.
+
+#### Supporting directory trees
+
+To allow tasks to generate data in subdirectories, a special filename `__base`
+will be used to provide an extra cue as to the final path. When mapping an output
+path  `/results/latest/__magic/job_400/task_01_01/__base/2017/2017-01-01.orc.lzo.pending`
+to a final destination path, the path will become `/results/latest/2017/2017-01-01.orc.lzo`.
+That is: all directories between `__magic` and `__base` inclusive will be ignored.
+
+
+**Issues**
+
+Q. What if there are some non-`.pending` files in the task attempt directory?
+
+A. This can only happen if the magic committer is being used in an S3A client
+which does not have the "magic path" feature enabled. This will be checked for
+during job and task committer initialization.
+
+
+### Failure cases
+
+#### Network Partitioning
+
+The job/task commit protocol is expected to handle this with the task
+only committing work when the job driver tells it to. A network partition
+should trigger the task committer's cancellation of the work (this is a protcol
+above the committers).
+
+#### Job Driver failure
+
+The job will be restarted. When it completes it will delete all
+outstanding requests to the destination directory which it has not
+committed itself.
+
+#### Task failure
+
+The task will be restarted. Pending work of the task will not be committed;
+when the job driver cleans up it will cancel pending writes under the directory.
+
+#### Multiple jobs targeting the same destination directory
+
+This leaves things in an inderminate state.
+
+
+#### Failure during task commit
+
+Pending uploads will remain, but no changes will be visible.
+
+If the `.pendingset` file has been saved to the job attempt directory, the
+task has effectively committed, it has just failed to report to the
+controller. This will cause complications during job commit, as there
+may be two task PendingSet committing the same files, or committing
+files with
+
+*Proposed*: track task ID in pendingsets, recognise duplicates on load
+and then respond by cancelling one set and committing the other. (or fail?)
+
+#### Failure during job commit
+
+The destination will be left in an unknown state.
+
+#### Failure during task/job abort
+
+Failures in the abort process are not well handled in either the committers
+or indeed in the applications which use these committers. If an abort
+operation fails, what can be done?
+
+While somewhat hypothetical for the use case of a task being aborted due
+to the protocol (e.g. speculative jobs being aborted), the abort task/abort job
+calls may be made as part of the exception handling logic on a failure to commit.
+As such, the caller may assume that the abort does not fail: if it does,
+the newly thrown exception may hide the original problem.
+
+Two options present themselves
+
+1. Catch, log and swallow failures in the `abort()`
+1. Throw the exceptions, and expect the callers to handle them: review, fix
+and test that code as appropriate.
+
+Fixing the calling code does seem to be the best strategy, as it allows the
+failure to be explictly handled in the commit protocol, rather than hidden
+in the committer.::OpenFile
+
+#### Preemption
+
+Preemption is the explicit termination of work at the behest of the cluster
+scheduler. It's a failure, but a special one: pre-empted tasks must not be counted
+as a failure in any code which only allows a limited number of trackers, and the
+Job driver can assume that the task was successfully terminated.
+
+Job drivers themselves may be preempted.
+
+
+
+#### Cleaning up after complete job failure
+
+One failure case is that the entire execution framework failed; a new process
+must identify outstanding jobs with pending work, and abort them, then delete
+the appropriate `__magic` directories.
+
+This can be done either by scanning the directory tree for `__magic` directories
+and scanning underneath them, or by using the `listMultipartUploads()` call to
+list multipart uploads under a path, then cancel them. The most efficient solution
+may be to use `listMultipartUploads` to identify all outstanding request, and use that
+to identify which requests to cancel, and where to scan for `__magic` directories.
+This strategy should address scalability problems when working with repositories
+with many millions of objects —rather than list all keys searching for those
+with `/__magic/**/*.pending` in their name, work backwards from the active uploads to
+the directories with the data.
+
+We may also want to consider having a cleanup operationn in the S3 CLI to
+do the full tree scan and purge of pending items; give some statistics on
+what was found. This will keep costs down and help us identify problems
+related to cleanup.
+
+### Performance
+
+The time to upload is that of today's block upload (`s3a.fast.upload=true`)
+output stream; ongoing through the write, and in the `close()` operation,
+a delay to upload any pending data and await all outstanding uploads to complete.
+There wouldn't be any overhead of the final completion request. If no
+data had yet been uploaded, the `close()` time would be that of the initiate
+multipart request and the final put. This could perhaps be simplified by always
+requesting a multipart ID on stream creation.
+
+The time to commit each task is `O(files)`: all `.pending` files in and under the task attempt
+directory will be listed, their contents read and then an aggregate `.pendingset`
+file PUT to the job attempt directory. The `.pending` files are then deleted.
+
+The time to commit a job will be `O(files/threads)`
+
+Every `.pendingset` file in the job attempt directory must be loaded, and a PUT
+request issued for every incomplete upload listed in the files.
+
+Note that it is the bulk listing of all children which is where full consistency
+is required. If instead, the list of files to commit could be returned from
+tasks to the job committer, as the Spark commit protocol allows, it would be
+possible to commit data to an inconsistent object store.
+
+### Cost
+
+Uncommitted data in an incomplete multipart upload is billed at the storage
+cost of the S3 bucket. To keep costs down, outstanding data from
+failed jobs must be deleted. This can be done through S3 bucket lifecycle policies,
+or some command tools which we would need to write.
+
+### Limitations of this algorithm
+
+1. Files will not be visible after the `close()` call, as they will not exist.
+Any code which expected pending-commit files to be visible will fail.
+
+1. Failures of tasks and jobs will leave outstanding multipart uploads. These
+will need to be garbage collected. S3 now supports automated cleanup; S3A has
+the option to do it on startup, and we plan for the `hadoop s3` command to
+allow callers to explicitly do it. If tasks were to explicitly write the upload
+ID of writes as a write commenced, cleanup by the job committer may be possible.
+
+1. The time to write very small files may be higher than that of PUT and COPY.
+We are ignoring this problem as not relevant in production; any attempt at optimizing
+small file operations will only complicate development, maintenance and testing.
+
+1. The files containing temporary information could be mistaken for actual
+data.
+
+1. It could potentially be harder to diagnose what is causing problems. Lots of
+logging can help, especially with debug-level listing of the directory structure
+of the temporary directories.
+
+1. To reliably list all PUT requests outstanding, we need list consistency
+In the absence of a means to reliably identify when an S3 endpoint is consistent, people
+may still use eventually consistent stores, with the consequent loss of data.
+
+1. If there is more than one job simultaneously writing to the same destination
+directories, the output may get confused. This appears to hold today with the current
+commit algorithms.
+
+1. It is possible to create more than one client writing to the
+same destination file within the same S3A client/task, either sequentially or in parallel.
+
+1. Even with a consistent metadata store, if a job overwrites existing
+files, then old data may still be visible to clients reading the data, until
+the update has propagated to all replicas of the data.
+
+1. If the operation is attempting to completely overwrite the contents of
+a directory, then it is not going to work: the existing data will not be cleaned
+up. A cleanup operation would need to be included in the job commit, deleting
+all files in the destination directory which where not being overwritten.
+
+1. It requires a path element, such as `__magic` which cannot be used
+for any purpose other than for the storage of pending commit data.
+
+1. Unless extra code is added to every FS operation, it will still be possible
+to manipulate files under the `__magic` tree. That's not bad, it just potentially
+confusing.
+
+1. As written data is not materialized until the commit, it will not be possible
+for any process to read or manipulated a file which it has just created.
+
+
+
+
+### Changes to `S3ABlockOutputStream`
+
+To avoid having to copy and paste the `S3ABlockOutputStream` it has
+been modified to be constructed with a `PutTracker` class to
+managed the immediate/delayed completion of uploads.
+It will be called at appropriate points.
+
+* Initialization, returning a marker to indicate whether or not multipart
+upload is commence immediately.
+* Multipart PUT init.
+* Single put init (not used in this algorithm, but useful for completeness).
+* Block upload init, failure and completion (from the relevant thread).
+* `close()` entered; all blocks completed —returning a marker to indicate
+whether any outstanding multipart should be committed.
+* Multipart abort in `abort()` call (maybe: move core logic elsewhere).
+
+The base implementation would do nothing
+except declare that the MPU must be executed in the `close()` call.
+
+The S3A Committer version, would
+1. Always initiate a during initialization
+1. In `close()` operation save all the data required to commit later.
+
+
+## Integrating the Committers with Hadoop MapReduce
+
+
+In order to support the ubiquitous `FileOutputFormat` and subclasses,
+S3A Committers will need somehow be accepted as a valid committer by the class,
+a class which explicity expects the output committer to be `FileOutputCommitter`
+
+```java
+public Path getDefaultWorkFile(TaskAttemptContext context,
+                               String extension) throws IOException{
+  PathOutputCommitter committer =
+    (PathOutputCommitter) getOutputCommitter(context);
+  return new Path(committer.getWorkPath(), getUniqueFile(context,
+    getOutputName(context), extension));
+}
+
+```
+
+Here are some options which have been considered, explored and discarded
+
+1. Adding more of a factory mechanism to create `FileOutputCommitter` instances;
+subclass this for S3A output and return it. The complexity of `FileOutputCommitter`
+and of supporting more dynamic consturction makes this dangerous from an implementation
+and maintenance perspective.
+
+1. Add a new commit algorithmm "3", which actually reads in the configured
+classname of a committer which it then instantiates and then relays the commit
+operations, passing in context information. Ths new committer interface would
+add methods for methods and attributes. This is viable, but does still change
+the existing Committer code in a way which may be high-maintenance.
+
+1. Allow the `FileOutputFormat` class to take any task/job context committer
+which implemented the `getWorkPath()` method —that being the sole
+specific feature which it needs from the `FileOutputCommitter`.
+
+
+Option 3, make `FileOutputFormat` support more generic committers, is the
+current design. It relies on the fact that the sole specific method of
+`FileOutputCommitter` which `FileOutputFormat` uses is `getWorkPath()`.
+
+This can be pulled up into a new abstract class, `PathOutputCommitter`, which
+`FileOutputCommitter` and `S3ACommitter` can implement:
+
+```java
+public abstract class PathOutputCommitter extends OutputCommitter {
+
+  /**
+   * Get the directory that the task should write results into.
+   * @return the work directory
+   */
+  public abstract Path getWorkPath() throws IOException;
+}
+```
+
+The sole change needed for `FileOutputFormat`  is to change what it casts
+the context committer to:
+
+```java
+PathOutputCommitter committer =
+  (PathOutputCommitter) getOutputCommitter(context);
+```
+
+Provided that `getWorkPath()` remains the sole method which `FileOutputFormat`
+uses, these changes will allow an S3A committer to replace the `FileOutputCommitter`,
+with minimal changes to the codebase.
+
+
+Update: There is a cost to this: MRv1 API support is lost.
+
+
+### MRv1 support via `org.apache.hadoop.mapred.FileOutputFormat`
+
+A price of not subclassing `FileOutputCommitter` is that the code used
+to wrap and relay the MRv1 API calls protocol to the `FileOutputCommitter`
+will not work: the new committer will not be picked up.
+
+This is visible in Spark, where the V1 API is exported from the `RDD` class
+(`RDD.saveAsHadoopFile()`)). The successor code, `PairRDDFunctions.saveAsNewAPIHadoopFile()`
+does work: *To get high performance commits in Object Stores, the MRv2 commit protocol
+must be used, which means: the V2 classes.
+
+
+
+#### Resolved issues
+
+
+**Magic Committer: Name of directory**
+
+The design proposes the name `__magic` for the directory. HDFS and
+the various scanning routines always treat files and directories starting with `_`
+as temporary/excluded data.
+
+
+**Magic Committer: Subdirectories**
+
+It is legal to create subdirectories in a task work directory, which
+will then be moved into the destination directory, retaining that directory
+tree.
+
+That is, a if the task working dir is `dest/__magic/app1/task1/`, all files
+under `dest/__magic/app1/task1/part-0000/` must end up under the path
+`dest/part-0000/`.
+
+This behavior is relied upon for the writing of intermediate map data in an MR
+job.
+
+This means it is not simply enough to strip off all elements of under `__magic`,
+it is critical to determine the base path.
+
+Proposed: use the special name `__base` as a marker of the base element for
+committing. Under task attempts a `__base` dir is created and turned into the
+working dir. All files created under this path will be committed to the destination
+with a path relative to the base dir.
+
+More formally: the last parent element of a path which is `__base` sets the
+base for relative paths created underneath it.
+
+
+## Testing
+
+The committers can only be tested against an S3-compatible object store.
+
+Although a consistent object store is a requirement for a production deployment
+of the magic committer an inconsistent one has appeared to work during testing, simply by
+adding some delays to the operations: a task commit does not succeed until
+all the objects which it has PUT are visible in the LIST operation. Assuming
+that further listings from the same process also show the objects, the job
+committer will be able to list and commit the uploads.
+
+
+The committers have some unit tests, and integration tests based on
+the protocol integration test lifted from `org.apache.hadoop.mapreduce.lib.output.TestFileOutputCommitter`
+to test various state transitions of the commit mechanism has been extended
+to support the variants of the staging committer.
+
+There is an abstract integration test, `AbstractITCommitMRJob` which creates
+a MiniYARN cluster bonded to a MiniHDFS cluster, then submits a simple
+MR job using the relevant committer. This verifies that the committer actually
+works, rather than just "appears to follow the protocol"
+
+One feature added during this testing is that the `_SUCCESS` marker file saved is
+no-longer a 0-byte file, it is a JSON manifest file, as implemented in
+`org.apache.hadoop.fs.s3a.commit.files.SuccessData`. This file includes
+the committer used, the hostname performing the commit, timestamp data and
+a list of paths committed.
+
+```
+SuccessData{
+  committer='PartitionedStagingCommitter',
+  hostname='devbox.local',
+  description='Task committer attempt_1493832493956_0001_m_000000_0',
+  date='Wed May 03 18:28:41 BST 2017',
+  filenames=[/test/testMRJob/part-m-00000, /test/testMRJob/part-m-00002, /test/testMRJob/part-m-00001]
+}
+```
+
+This was useful a means of verifying that the correct
+committer had in fact been invoked in those forked processes: a 0-byte `_SUCCESS`
+marker implied the classic `FileOutputCommitter` had been used; if it could be read
+then it provides some details on the commit operation which are then used
+in assertions in the test suite.
+
+It has since been extended to collet metrics and other values, and has proven
+equally useful in Spark integration testing.
+
+## Integrating the Committers with Apache Spark
+
+
+Spark defines a commit protocol `org.apache.spark.internal.io.FileCommitProtocol`,
+implementing it in `HadoopMapReduceCommitProtocol` a subclass `SQLHadoopMapReduceCommitProtocol`
+which supports the configurable declaration of the underlying Hadoop committer class,
+and the `ManifestFileCommitProtocol` for Structured Streaming. The latter
+is best defined as "a complication" —but without support for it, S3 cannot be used
+as a reliable destination of stream checkpoints.
+
+One aspect of the Spark commit protocol is that alongside the Hadoop file committer,
+there's an API to request an absolute path as a target for a commit operation,
+`newTaskTempFileAbsPath(taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String`;
+each task's mapping of temp-> absolute files is passed to the Spark driver
+in the `TaskCommitMessage` returned after a task performs its local
+commit operations (which includes requesting permission to commit from the executor).
+These temporary paths are renamed to the final absolute paths are renamed
+in `FileCommitProtocol.commitJob()`. This is currently a serialized rename sequence
+at the end of all other work. This use of absolute paths is used in writing
+data into a destination directory tree whose directory names is driven by
+partition names (year, month, etc).
+
+Supporting that feature is going to be challenging; either we allow each directory in the partition tree to
+have its own staging directory documenting pending PUT operations, or (better) a staging directory
+tree is built off the base path, with all pending commits tracked in a matching directory
+tree.
+
+Alternatively, the fact that Spark tasks provide data to the job committer on their
+completion means that a list of pending PUT commands could be built up, with the commit
+operations being excuted by an S3A-specific implementation of the `FileCommitProtocol`.
+As noted earlier, this may permit the reqirement for a consistent list operation
+to be bypassed. It would still be important to list what was being written, as
+it is needed to aid aborting work in failed tasks, but the list of files
+created by successful tasks could be passed directly from the task to committer,
+avoid that potentially-inconsistent list.
+
+
+#### Spark, Parquet and the Spark SQL Commit mechanism
+
+Spark's `org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat`
+Parquet output format wants a subclass of `org.apache.parquet.hadoop.ParquetOutputCommitter`,
+the option being defined by the classname in the configuration
+key `spark.sql.parquet.output.committer.class`;
+this is then patched in to the value `spark.sql.sources.outputCommitterClass`
+where it is picked up by `SQLHadoopMapReduceCommitProtocol` and instantiated
+as the committer for the work.
+
+This is presumably done so the user has the option of requesting a metadata
+summary file by setting the option `"parquet.enable.summary-metadata"`.
+Creating the summary file requires scanning every single file in the destination
+directory on the job commit, so is *very* expensive, and not something which
+we recommend when working with S3.
+
+
+To use a S3Guard committer, it must also be identified as the Parquet committer.
+The fact that instances are dynamically instantiated somewhat complicates the process.
+
+In early tests; we can switch committers for ORC output without making any changes
+to the Spark code or configuration other than configuring the factory
+for Path output committers.  For Parquet support, it may be sufficient to also declare
+the classname of the specific committer (i.e not the factory).
+
+This is unfortunate as it complicates dynamically selecting a committer protocol
+based on the destination filesystem type or any per-bucket configuration.
+
+
+
+The solution as implemented in the [initial prototype](https://github.com/hortonworks-spark/cloud-integration)
+consists of two things
+
+1. A class `PathOutputCommitProtocol extends HadoopMapReduceCommitProtocol`
+which always creates the committer using the `PathOutputCommitterFactory`
+mechanism. This ensures that output format's own committers are replaced
+with an output factory mechanism.
+
+1. A class `org.apache.hadoop.mapreduce.lib.output.BindingPathOutputCommitter`
+which is a directly instantiable output committer that then creates a
+committer through the factory mechanism and delegates all operations to
+that committer. This allows a committer to be declared in any configuration
+option which takes a committer class, but still use the factory mechanism
+underneath.
+
+1. Add a patch to Spark 2.3 [SPARK-21762], which allows any output committer
+to be used for `ParquetFileOutput`.
+
+Overall, its a bit convoluted to implement, document and use. Users must
+declare two spark SQL options as well as three spark.hadoop ones
+
+```properties
+spark.sql.sources.commitProtocolClass=com.hortonworks.spark.cloud.commit.PathOutputCommitProtocol
+spark.sql.parquet.output.committer.class=org.apache.hadoop.mapreduce.lib.output.BindingPathOutputCommitter
+spark.hadoop.mapreduce.outputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory
+spark.hadoop.fs.s3a.committer.name=magic
+spark.hadoop.fs.s3a.committer.magic.enabled=true
+```
+
+We could actually simplify this by adding a new algorithm "3" to the existing
+`FileOutputFormat`, telling `FileOutputFormat` itself to use the factory
+to create committers. This would then automatically pick up the factory
+Avoiding loops in this situation would be "challenging": If instantiated
+via a factory, the file committer must not attempt to use the factory
+itself.
+
+
+## Security
+
+What are the obvious possible security risks which need to be covered
+and which code reviews should check for?
+
+* Leakage of AWS credentials in jobs/logging.
+* Exposure of data to unauthorized users.
+* Exposure of workflow to unauthorized users (e.g. paths & filenames, sequence
+of queries).
+* Silent tampering of data by unauthed users.
+* Attacks on jobs which run up (large) bills by leaking pending datasets.
+* DoS attacks with malicious users sabotaging/hindering progress of other
+users' work.
+
+#### Security Risks of the Staging Committers
+
+Staged data is on local FS, in directories listed by `fs.s3a.buffer.dir`,
+falling back to the directory in `hadoop.tmp.dir`.
+These are the same directories already
+used to buffer blocks of data being written in output streams, prior to
+PUT/POST to S3. Therefore: there is more risk than before. We should
+clearly document the security aspects of the temp dirs to ensure this.
+
+As all files written by a task are not uploaded until task commit, more
+local storage is needed. A malicious user executing work on one system
+could potentially fill up the temp disk space. Mitigation: storage
+quotas in local FS, keeping temp dirs on different mounted FS from root.
+
+The intermediate `.pendingset` files are saved in HDFS under the directory in
+`fs.s3a.committer.staging.tmp.path`; defaulting to  `/tmp`. This data can
+disclose the workflow (it contains the destination paths & amount of data
+generated), and if deleted, breaks the job. If malicous code were to edit
+the file, by, for example, reordering the ordered etag list, the generated
+data would be committed out of order, creating invalid files. As this is
+the (usually transient) cluster FS, any user in the cluster has the potential
+to do this.
+
+*Locking down the temporary directories is critical to prevent malicious
+cluster users damaging the workflow and output.*
+
+#### Security Risks of the Magic Committer
+
+The directory defined by `fs.s3a.buffer.dir` is used to buffer blocks
+before upload, unless the job is configured to buffer the blocks in memory.
+This is as before: no incremental risk. As blocks are deleted from the filesystem
+after upload, the amount of storage needed is determined by the data generation
+bandwidth and the data upload bandwdith.
+
+No use is made of the cluster filesystem; there are no risks there.
+
+A consistent store is required, which, for Amazon's infrastructure, means S3Guard.
+This is covered below.
+
+A malicious user with write access to the `__magic` directory could manipulate
+or delete the metadata of pending uploads, or potentially inject new work int
+the commit. Having access to the `__magic` directory implies write access
+to the parent destination directory: a malicious user could just as easily
+manipulate the final output, without needing to attack the committer's intermediate
+files.
+
+
+### Security Risks of all committers
+
+
+#### Visibility
+
+* If S3Guard is used for storing metadata, then the metadata is visible to
+all users with read access. A malicious user with write access could delete
+entries of newly generated files, so they would not be visible.
+
+
+#### Malicious Serialized Data
+
+The internal metadata summary files (`.pending` and `.pendingset`)
+could be tampered by someone malicious, and, when parsed or interpreted by
+the trusted account, used to execute untrusted code, fail, etc.
+The formats are all JSON, parsed with Jackson; we expect invalid JSON
+to result in parse errors, and fail the job. Aborting the job triggers a
+best-effort attempt to load the pending files, ignoring those which cannot
+be loaded or parsed, and aborts all pending uploads identified by the loaded
+files.
+
+* None of the strings in the parsed dataset are passed through any interpreter
+(e.g used in SQL queries, shell commands or similar). Therefore (SQL) injection
+attacks are not possible.
+
+* Some of the data *may* be printed in the log files generated during
+process execution. For example, commit Id, destination Key, etc. These
+are all treated as plain text, and should be served up as such in any browser-
+hosted view of the logs.
+
+* Some of the data is returned in the `toString()` values of the loaded
+classes. This may also be logged, observed in IDEs.
+
+* None of the content in the serialized data is displayed in any web UI.
+The vulnerability there is what if this happened in the future in some
+downstream application: would it be possible to inject Javascript into
+any of the text fields, script which could then be executed in some XSS
+attack. We may wish to consider sanitizing this data on load.
+
+* Paths in tampered data could be modified in an attempt to commit an upload across
+an existing file, or the MPU ID alterated to prematurely commit a different upload.
+These attempts will not going to succeed, because the destination
+path of the upload is declared on the initial POST to initiate the MPU, and
+operations associated with the MPU must also declare the path: if the path and
+ID are inconsistent, the operation will fail. If a valid (path, MPU) tuple
+were inserted in a tampered-with file, the commit or abort could complete or abort
+the upload prematurely. As the malicious party must already have enough access to the
+target S3 store to obtain this information, they are unlikely to need to
+tamper with the JSON files to perform such actions.
+
+
+#### Outstanding uncommitted data and its cost
+
+* Killed jobs will leak uncommitted uploads, which will run up bills.
+A restarted job will automatically purge
+all pending uploads under the destination path on job commit, so if the job
+is rerun it will cancel the pending writes of the previous job attempt.
+
+We shall also provide a CLI tool to list and delete pending uploads under a path.
+
+* configuring a store to automatically clean pending uploads after a
+time period such as 24h will guarantee that pending upload data will always
+be deleted, even without a rerun of the job or use of the CLI tool.
+
+AWS document [the permissions for MPU](http://docs.aws.amazon.com/AmazonS3/latest/dev/mpuAndPermissions.html).
+There are special permissions `s3:ListBucketMultipartUploads`,
+`s3:ListMultipartUploadParts` and `s3:AbortMultipartUpload`. By default
+bucket owner has all these permissions, MPU initiator will be granted
+the permissions to list the upload, list hte parts and abort their own uploads.
+Bucket owner may grant/deny perms to either (this may permit a user to be
+able to initiate & complete MPU, but not delete and abort).
+
+#### Proposed security settings & recommendations
+
+* Bucket access restricted to specific IAM roles.
+* `fs.s3a.buffer.dir` set to location under `/tmp` with read & write access
+restricted to the active user.
+* `fs.s3a.committer.staging.tmp.path` should be isolated to the active
+each user. Proposed: make the default an unqualified path, `tmp/staging`,
+which will made absolute relative to the current user. In filesystems in
+which access under user's home directories are restricted, this final, absolute
+path, will not be visible to untrusted accounts.
+
+* Maybe: define the for valid characters in a text strings, and a regext for
+ validating, e,g, `[a-zA-Z0-9 \.\,\(\) \-\+]+` and then validate any free text
+ JSON fields on load and save.

+ 819 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md

@@ -0,0 +1,819 @@
+<!---
+  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.
+-->
+
+# Committing work to S3 with the "S3A Committers"
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+This page covers the S3A Committers, which can commit work directly
+to an S3 object store.
+
+These committers are designed to solve a fundamental problem which
+the standard committers of work cannot do to S3: consistent, high performance,
+and reliable commitment of output to S3.
+
+For details on their internal design, see
+[S3A Committers: Architecture and Implementation](./committer_architecture.html).
+
+
+## Introduction: The Commit Problem
+
+
+Apache Hadoop MapReduce (and behind the scenes, Apache Spark) often write
+the output of their work to filesystems
+
+Normally, Hadoop uses the `FileOutputFormatCommitter` to manage the
+promotion of files created in a single task attempt to the final output of
+a query. This is done in a way to handle failures of tasks and jobs, and to
+support speculative execution. It does that by listing directories and renaming
+their content into the final destination when tasks and then jobs are committed.
+
+This has some key requirement of the underlying filesystem:
+
+1. When you list a directory, you see all the files which have been created in it,
+and no files which are not in it (i.e. have been deleted).
+1. When you rename a directory, it is an `O(1)` atomic transaction. No other
+process across the cluster may rename a file or directory to the same path.
+If the rename fails for any reason, either the data is at the original location,
+or it is at the destination, -in which case the rename actually succeeded.
+
+**The S3 object store and the `s3a://` filesystem client cannot meet these requirements.*
+
+1. Amazon S3 has inconsistent directory listings unless S3Guard is enabled.
+1. The S3A mimics `rename()` by copying files and then deleting the originals.
+This can fail partway through, and there is nothing to prevent any other process
+in the cluster attempting a rename at the same time.
+
+As a result,
+
+* Files my not be listed, hence not renamed into place.
+* Deleted files may still be discovered, confusing the rename process to the point
+of failure.
+* If a rename fails, the data is left in an unknown state.
+* If more than one process attempts to commit work simultaneously, the output
+directory may contain the results of both processes: it is no longer an exclusive
+operation.
+*. While S3Guard may deliver the listing consistency, commit time is still
+proportional to the amount of data created. It still can't handle task failure.
+
+**Using the "classic" `FileOutputCommmitter` to commit work to Amazon S3 risks
+loss or corruption of generated data**
+
+
+To address these problems there is now explicit support in the `hadop-aws`
+module for committing work to Amazon S3 via the S3A filesystem client,
+*the S3A Committers*
+
+
+For safe, as well as high-performance output of work to S3,
+we need use "a committer" explicitly written to work with S3, treating it as
+an object store with special features.
+
+
+### Background : Hadoop's "Commit Protocol"
+
+How exactly is work written to its final destination? That is accomplished by
+a "commit protocol" between the workers and the job manager.
+
+This protocol is implemented in Hadoop MapReduce, with a similar but extended
+version in Apache Spark:
+
+1. A "Job" is the entire query, with inputs to outputs
+1. The "Job Manager" is the process in charge of choreographing the execution
+of the job. It may perform some of the actual computation too.
+1. The job has "workers", which are processes which work the actual data
+and write the results.
+1. Workers execute "Tasks", which are fractions of the job, a job whose
+input has been *partitioned* into units of work which can be executed independently.
+1. The Job Manager directs workers to execute "tasks", usually trying to schedule
+the work close to the data (if the filesystem provides locality information).
+1. Workers can fail: the Job manager needs to detect this and reschedule their active tasks.
+1. Workers can also become separated from the Job Manager, a "network partition".
+It is (provably) impossible for the Job Manager to distinguish a running-but-unreachable
+worker from a failed one.
+1. The output of a failed task must not be visible; this is to avoid its
+data getting into the final output.
+1. Multiple workers can be instructed to evaluate the same partition of the work;
+this "speculation" delivers speedup as it can address the "straggler problem".
+When multiple workers are working on the same data, only one worker is allowed
+to write the final output.
+1. The entire job may fail (often from the failure of the Job Manager (MR Master, Spark Driver, ...)).
+1, The network may partition, with workers isolated from each other or
+the process managing the entire commit.
+1. Restarted jobs may recover from a failure by reusing the output of all
+completed tasks (MapReduce with the "v1" algorithm), or just by rerunning everything
+(The "v2" algorithm and Spark).
+
+
+What is "the commit protocol" then? It is the requirements on workers as to
+when their data is made visible, where, for a filesystem, "visible" means "can
+be seen in the destination directory of the query."
+
+* There is a destination directory of work, "the output directory."
+* The final output of tasks must be in this directory *or paths underneath it*.
+* The intermediate output of a task must not be visible in the destination directory.
+That is: they must not write directly to the destination.
+* The final output of a task *may* be visible under the destination.
+* The Job Manager makes the decision as to whether a task's data is to be "committed",
+be it directly to the final directory or to some intermediate store..
+* Individual workers communicate with the Job manager to manage the commit process:
+whether the output is to be *committed* or *aborted*
+* When a worker commits the output of a task, it somehow promotes its intermediate work to becoming
+final.
+* When a worker aborts a task's output, that output must not become visible
+(i.e. it is not committed).
+* Jobs themselves may be committed/aborted (the nature of "when" is not covered here).
+* After a Job is committed, all its work must be visible.
+* And a file `_SUCCESS` may be written to the output directory.
+* After a Job is aborted, all its intermediate data is lost.
+* Jobs may also fail. When restarted, the successor job must be able to clean up
+all the intermediate and committed work of its predecessor(s).
+* Task and Job processes measure the intervals between communications with their
+Application Master and YARN respectively.
+When the interval has grown too large they must conclude
+that the network has partitioned and that they must abort their work.
+
+
+That's "essentially" it. When working with HDFS and similar filesystems,
+directory `rename()` is the mechanism used to commit the work of tasks and
+jobs.
+* Tasks write data to task attempt directories under the directory `_temporary`
+underneath the final destination directory.
+* When a task is committed, these files are renamed to the destination directory
+(v2 algorithm) or a job attempt directory under `_temporary` (v1 algorithm).
+* When a job is committed, for the v2 algorithm the `_SUCCESS` file is created,
+and the `_temporary` deleted.
+* For the v1 algorithm, when a job is committed, all the tasks committed under
+the job attempt directory will have their output renamed into the destination
+directory.
+* The v2 algorithm recovers from failure by deleting the destination directory
+and restarting the job.
+* The v1 algorithm recovers from failure by discovering all committed tasks
+whose output is in the job attempt directory, *and only rerunning all uncommitted tasks*.
+
+
+None of this algorithm works safely or swiftly when working with "raw" AWS S3 storage:
+* Directory listing can be inconsistent: the tasks and jobs may not list all work to
+be committed.
+* Renames go from being fast, atomic operations to slow operations which can fail partway through.
+
+This then is the problem which the S3A committers address:
+
+*How to safely and reliably commit work to Amazon S3 or compatible object store*
+
+
+## Meet the S3A Commmitters
+
+Since Hadoop 3.1, the S3A FileSystem has been accompanied by classes
+designed to integrate with the Hadoop and Spark job commit protocols, classes
+which interact with the S3A filesystem to reliably commit work work to S3:
+*The S3A Committers*
+
+The underlying architecture of this process is very complex, and
+covered in [the committer architecture documentation](./committer_architecture.html).
+
+The key concept to know of is S3's "Multipart Upload" mechanism. This allows
+an S3 client to write data to S3 in multiple HTTP POST requests, only completing
+the write operation with a final POST to complete the upload; this final POST
+consisting of a short list of the etags of the uploaded blocks.
+This multipart upload mechanism is already automatically used when writing large
+amounts of data to S3; an implementation detail of the S3A output stream.
+
+The S3A committers make explicit use of this multipart upload ("MPU") mechanism:
+
+1. The individual *tasks* in a job write their data to S3 as POST operations
+within multipart uploads, yet do not issue the final POST to complete the upload.
+1. The multipart uploads are committed in the job commit process.
+
+There are two different S3A committer types, *staging*
+and *magic*. The committers primarily vary in how data is written during task execution,
+how  the pending commit information is passed to the job manager, and in how
+conflict with existing files is resolved.
+
+
+| feature | staging | magic |
+|--------|---------|---|
+| task output destination | local disk | S3A *without completing the write* |
+| task commit process | upload data from disk to S3 | list all pending uploads on s3 and write details to job attempt directory |
+| task abort process | delete local disk data | list all pending uploads and abort them |
+| job commit | list & complete pending uploads | list & complete pending uploads |
+
+The other metric is "maturity". There, the fact that the Staging committers
+are based on Netflix's production code counts in its favor.
+
+
+### The Staging Committer
+
+This is based on work from Netflix. It "stages" data into the local filesystem.
+It also requires the cluster to have HDFS, so that
+
+Tasks write to URLs with `file://` schemas. When a task is committed,
+its files are listed, uploaded to S3 as incompleted Multipart Uploads.
+The information needed to complete the uploads is saved to HDFS where
+it is committed through the standard "v1" commit algorithm.
+
+When the Job is committed, the Job Manager reads the lists of pending writes from its
+HDFS Job destination directory and completes those uploads.
+
+Cancelling a task is straighforward: the local directory is deleted with
+its staged data. Cancelling a job is achieved by reading in the lists of
+pending writes from the HDFS job attempt directory, and aborting those
+uploads. For extra safety, all outstanding multipart writes to the destination directory
+are aborted.
+
+The staging committer comes in two slightly different forms, with slightly
+diffrent conflict resolution policies:
+
+
+* **Directory**: the entire directory tree of data is written or overwritten,
+as normal.
+
+* **Partitioned**: special handling of partitioned directory trees of the form
+`YEAR=2017/MONTH=09/DAY=19`: conflict resolution is limited to the partitions
+being updated.
+
+
+The Partitioned Committer is intended to allow jobs updating a partitioned
+directory tree to restrict the conflict resolution to only those partition
+directories containing new data. It is intended for use with Apache Spark
+only.
+
+
+## Conflict Resolution in the Staging Committers
+
+The Staging committers offer the ability to replace the conflict policy
+of the execution engine with policy designed to work with the tree of data.
+This is based on the experience and needs of Netflix, where efficiently adding
+new data to an existing partitioned directory tree is a common operation.
+
+```xml
+<property>
+  <name>fs.s3a.committer.staging.conflict-mode</name>
+  <value>fail</value>
+  <description>
+    Staging committer conflict resolution policy: {@value}.
+    Supported: fail, append, replace.
+  </description>
+</property>
+```
+
+**replace** : when the job is committed (and not before), delete files in
+directories into which new data will be written.
+
+**fail**: when there are existing files in the destination, fail the job.
+
+**append**: Add new data to the directories at the destination; overwriting
+any with the same name. Reliable use requires unique names for generated files,
+which the committers generate
+by default.
+
+The difference between the two staging ommitters are as follows:
+
+The Directory Committer uses the entire directory tree for conflict resolution.
+If any file exists at the destination it will fail in job setup; if the resolution
+mechanism is "replace" then all existing files will be deleted.
+
+The partitioned committer calculates the partitions into which files are added,
+the final directories in the tree, and uses that in its conflict resolution
+process:
+
+
+**replace** : delete all data in the destination partition before committing
+the new files.
+
+**fail**: fail if there is data in the destination partition, ignoring the state
+of any parallel partitions.
+
+**append**: add the new data.
+
+It's intended for use in Apache Spark Dataset operations, rather
+than Hadoop's original MapReduce engine, and only in jobs
+where adding new data to an existing dataset is the desired goal.
+
+Preequisites for successful work
+
+1. The output is written into partitions via `PARTITIONED BY` or `partitionedBy()`
+instructions.
+2. There is no data written directly to the root path (all files there are
+ignored; it's implicitly "append").
+
+Here's an example in Spark, assuming that `sourceDataset` is a dataset
+whose columns include "year" and "month":
+
+```scala
+sourceDataset
+  .write
+  .partitionBy("year", "month")
+  .mode(SaveMode.Append)
+  .opt("fs.s3a.committer.name", "partitioned")
+  .opt("fs.s3a.committer.staging.conflict-mode", "replace")
+  .format("orc")
+  .save("s3a://examples/statistics")
+```
+
+
+### The Magic Committer
+
+The "Magic" committer does its work through "magic" in the filesystem:
+attempts to write to specific "magic" paths are interpreted as writes
+to a parent directory *which are not to be completed*. When the output stream
+is closed, the information needed to complete the write is saved in the magic
+directory. The task committer saves the list of these to a directory for the
+job committers use, or, if aborting, lists the pending writes and aborts them.
+
+The job committer reads in the list of pending commits, and commits them as
+the Staging Committer does.
+
+Compared to the Staging Committer, the Magic Committer offers faster write
+times: output is uploaded to S3 as it is written, rather than in the
+task commit.
+
+However, it has extra requirements of the filesystem
+
+1. It requires a consistent object store, which for Amazon S3,
+means that [S3Guard](./s3guard.html) must be enabled. For third-party stores,
+consult the documentation.
+1. The S3A client must be configured to recognize interactions
+with the magic directories and treat them specially.
+
+
+It's also not been field tested to the extent of Netflix's committer; consider
+it the least mature of the committers.
+
+
+#### Which Committer to Use?
+
+1. If you want to create or update existing partitioned data trees in Spark, use thee
+Partitioned Committer. Make sure you have enough hard disk capacity for all staged data.
+Do not use it in other situations.
+
+1. If you know that your object store is consistent, or that the processes
+writing data use S3Guard, use the Magic Committer for higher performance
+writing of large amounts of data.
+
+1. Otherwise: use the directory committer, making sure you have enough
+hard disk capacity for all staged data.
+
+Put differently: start with the Directory Committer.
+
+## Switching to an S3A Committer
+
+To use an S3A committer, the property `mapreduce.outputcommitter.factory.scheme.s3a`
+must be set to the S3A committer factory, `org.apache.hadoop.fs.s3a.commit.staging.S3ACommitterFactory`.
+This is done in `core-default.xml`
+
+```xml
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.s3a</name>
+  <value>org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory</value>
+  <description>
+    The committer factory to use when writing data to S3A filesystems.
+  </description>
+</property>
+```
+
+What is missing is an explicit choice of committer to use in the property
+`fs.s3a.committer.name`; so the classic (and unsafe) file committer is used.
+
+| `fs.s3a.committer.name` |  Committer |
+|--------|---------|
+| `directory` | directory staging committer |
+| `partitioned` | partition staging committer (for use in Spark only) |
+| `magic` | the "magic" committer |
+| `file` | the original and unsafe File committer; (default) |
+
+
+
+## Using the Directory and Partitioned Staging Committers
+
+Generated files are initially written to a local directory underneath one of the temporary
+directories listed in `fs.s3a.buffer.dir`.
+
+
+The staging commmitter needs a path in the cluster filesystem
+(e.g. HDFS). This must be declared in `fs.s3a.committer.tmp.path`.
+
+Temporary files are saved in HDFS (or other cluster filesystem) under the path
+`${fs.s3a.committer.tmp.path}/${user}` where `user` is the name of the user running the job.
+The default value of `fs.s3a.committer.tmp.path` is `tmp/staging`,
+Which will be converted at run time to a path under the current user's home directory,
+essentially `~/tmp/staging`
+ so the temporary directory
+
+The application attempt ID is used to create a unique path under this directory,
+resulting in a path `~/tmp/staging/${user}/${application-attempt-id}/` under which
+summary data of each task's pending commits are managed using the standard
+`FileOutputFormat` committer.
+
+When a task is committed the data is uploaded under the destination directory.
+The policy of how to react if the destination exists is defined by
+the `fs.s3a.committer.staging.conflict-mode` setting.
+
+| `fs.s3a.committer.staging.conflict-mode` | Meaning |
+| -----------------------------------------|---------|
+| `fail` | Fail if the destination directory exists |
+| `replace` | Delete all existing files before committing the new data |
+| `append` | Add the new files to the existing directory tree |
+
+
+## The "Partitioned" Staging Committer
+
+This committer an extension of the "Directory" committer which has a special conflict resolution
+policy designed to support operations which insert new data into a directory tree structured
+using Hive's partitioning strategy: different levels of the tree represent different columns.
+
+For example, log data could be partitioned by `YEAR` and then by `MONTH`, with different
+entries underneath.
+
+```
+logs/YEAR=2017/MONTH=01/
+  log-20170101.avro
+  log-20170102.avro
+  ...
+  log-20170131.avro
+
+logs/YEAR=2017/MONTH=02/
+  log-20170201.avro
+  log-20170202.avro
+  ...
+  log-20170227.avro
+
+logs/YEAR=2017/MONTH=03/
+logs/YEAR=2017/MONTH=04/
+```
+
+A partitioned structure like this allows for queries using Hive or Spark to filter out
+files which do not contain relevant data.
+
+What the partitioned committer does is, where the tooling permits, allows callers
+to add data to an existing partitioned layout*.
+
+More specifically, it does this by having a conflict resolution options which
+only act on invididual partitions, rather than across the entire output tree.
+
+| `fs.s3a.committer.staging.conflict-mode` | Meaning |
+| -----------------------------------------|---------|
+| `fail` | Fail if the destination partition(s) exist |
+| `replace` | Delete the existing data partitions before committing the new data |
+| `append` | Add the new data to the existing partitions |
+
+
+As an example, if a job was writing the file
+`logs/YEAR=2017/MONTH=02/log-20170228.avro`, then with a policy of `fail`,
+the job would fail. With a policy of `replace`, then entire directory
+`logs/YEAR=2017/MONTH=02/` would be deleted before the new file `log-20170228.avro`
+was written. With the policy of `append`, the new file would be added to
+the existing set of files.
+
+
+### Notes
+
+1. A deep partition tree can itself be a performance problem in S3 and the s3a client,
+or, more specifically. a problem with applications which use recursive directory tree
+walks to work with data.
+
+1. The outcome if you have more than one job trying simultaneously to write data
+to the same destination with any policy other than "append" is undefined.
+
+1. In the `append` operation, there is no check for conflict with file names.
+If, in the example above, the file `log-20170228.avro` already existed,
+it would be overridden. Set `fs.s3a.committer.staging.unique-filenames` to `true`
+to ensure that a UUID is included in every filename to avoid this.
+
+
+## Using the Magic committer
+
+This is less mature than the Staging Committer, but promises higher
+performance.
+
+### FileSystem client setup
+
+1. Use a *consistent* S3 object store. For Amazon S3, this means enabling
+[S3Guard](./s3guard.html). For S3-compatible filesystems, consult the filesystem
+documentation to see if it is consistent, hence compatible "out of the box".
+1. Turn the magic on by `fs.s3a.committer.magic.enabled"`
+
+```xml
+<property>
+  <name>fs.s3a.committer.magic.enabled</name>
+  <description>
+  Enable support in the filesystem for the S3 "Magic" committter.
+  </description>
+  <value>true</value>
+</property>
+```
+
+*Do not use the Magic Committer on an inconsistent S3 object store. For
+Amazon S3, that means S3Guard must *always* be enabled.
+
+
+### Enabling the committer
+
+```xml
+<property>
+  <name>fs.s3a.committer.name</name>
+  <value>magic</value>
+</property>
+
+```
+
+Conflict management is left to the execution engine itself.
+
+## Committer Configuration Options
+
+
+| Option | Magic | Directory | Partitioned | Meaning | Default |
+|--------|-------|-----------|-------------|---------|---------|
+| `mapreduce.fileoutputcommitter.marksuccessfuljobs` | X | X | X | Write a `_SUCCESS` file  at the end of each job | `true` |
+| `fs.s3a.committer.threads` | X | X | X | Number of threads in committers for parallel operations on files. | 8 |
+| `fs.s3a.committer.staging.conflict-mode` |  | X | X | Conflict resolution: `fail`, `abort` or `overwrite`| `fail` |
+| `fs.s3a.committer.staging.unique-filenames` |  | X | X | Generate unique filenames | `true` |
+
+| `fs.s3a.committer.magic.enabled` | X |  | | Enable "magic committer" support in the filesystem | `false` |
+
+
+
+
+| Option | Magic | Directory | Partitioned | Meaning | Default |
+|--------|-------|-----------|-------------|---------|---------|
+| `fs.s3a.buffer.dir` | X | X | X | Local filesystem directory for data being written and/or staged. | |
+| `fs.s3a.committer.staging.tmp.path` |  | X | X | Path in the cluster filesystem for temporary data | `tmp/staging` |
+
+
+```xml
+<property>
+  <name>fs.s3a.committer.name</name>
+  <value>file</value>
+  <description>
+    Committer to create for output to S3A, one of:
+    "file", "directory", "partitioned", "magic".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.magic.enabled</name>
+  <value>false</value>
+  <description>
+    Enable support in the filesystem for the S3 "Magic" committer.
+    When working with AWS S3, S3Guard must be enabled for the destination
+    bucket, as consistent metadata listings are required.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.threads</name>
+  <value>8</value>
+  <description>
+    Number of threads in committers for parallel operations on files
+    (upload, commit, abort, delete...)
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.tmp.path</name>
+  <value>tmp/staging</value>
+  <description>
+    Path in the cluster filesystem for temporary data.
+    This is for HDFS, not the local filesystem.
+    It is only for the summary data of each file, not the actual
+    data being committed.
+    Using an unqualified path guarantees that the full path will be
+    generated relative to the home directory of the user creating the job,
+    hence private (assuming home directory permissions are secure).
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.unique-filenames</name>
+  <value>true</value>
+  <description>
+    Option for final files to have a unique name through job attempt info,
+    or the value of fs.s3a.committer.staging.uuid
+    When writing data with the "append" conflict option, this guarantees
+    that new data will not overwrite any existing data.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.staging.conflict-mode</name>
+  <value>fail</value>
+  <description>
+    Staging committer conflict resolution policy.
+    Supported: "fail", "append", "replace".
+  </description>
+</property>
+
+<property>
+  <name>s.s3a.committer.staging.abort.pending.uploads</name>
+  <value>true</value>
+  <description>
+    Should the staging committers abort all pending uploads to the destination
+    directory?
+
+    Changing this if more than one partitioned committer is
+    writing to the same destination tree simultaneously; otherwise
+    the first job to complete will cancel all outstanding uploads from the
+    others. However, it may lead to leaked outstanding uploads from failed
+    tasks. If disabled, configure the bucket lifecycle to remove uploads
+    after a time period, and/or set up a workflow to explicitly delete
+    entries. Otherwise there is a risk that uncommitted uploads may run up
+    bills.
+  </description>
+</property>
+
+<property>
+  <name>mapreduce.outputcommitter.factory.scheme.s3a</name>
+  <value>org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory</value>
+  <description>
+    The committer factory to use when writing data to S3A filesystems.
+    If mapreduce.outputcommitter.factory.class is set, it will
+    override this property.
+
+    (This property is set in mapred-default.xml)
+  </description>
+</property>
+
+```
+
+
+## Troubleshooting
+
+### `Filesystem does not have support for 'magic' committer`
+
+```
+org.apache.hadoop.fs.s3a.commit.PathCommitException: `s3a://landsat-pds': Filesystem does not have support for 'magic' committer enabled
+in configuration option fs.s3a.committer.magic.enabled
+```
+
+The Job is configured to use the magic committer, but the S3A bucket has not been explicitly
+declared as supporting it.
+
+The destination bucket **must** be declared as supporting the magic committer.
+
+This can be done for those buckets which are known to be consistent, either
+because [S3Guard](s3guard.html) is used to provide consistency,
+or because the S3-compatible filesystem is known to be strongly consistent.
+
+```xml
+<property>
+  <name>fs.s3a.bucket.landsat-pds.committer.magic.enabled</name>
+  <value>true</value>
+</property>
+```
+
+*IMPORTANT*: only enable the magic committer against object stores which
+offer consistent listings. By default, Amazon S3 does not do this -which is
+why the option `fs.s3a.committer.magic.enabled` is disabled by default.
+
+
+Tip: you can verify that a bucket supports the magic committer through the
+`hadoop s3guard bucket-info` command:
+
+
+```
+> hadoop s3guard bucket-info -magic s3a://landsat-pds/
+
+Filesystem s3a://landsat-pds
+Location: us-west-2
+Filesystem s3a://landsat-pds is not using S3Guard
+The "magic" committer is not supported
+
+S3A Client
+  Endpoint: fs.s3a.endpoint=(unset)
+  Encryption: fs.s3a.server-side-encryption-algorithm=none
+  Input seek policy: fs.s3a.experimental.input.fadvise=normal
+2017-09-27 19:18:57,917 INFO util.ExitUtil: Exiting with status 46: 46: The magic committer is not enabled for s3a://landsat-pds
+```
+
+## Error message: "File being created has a magic path, but the filesystem has magic file support disabled:
+
+A file is being written to a path which is used for "magic" files,
+files which are actually written to a different destination than their stated path
+*but the filesystem doesn't support "magic" files*
+
+This message should not appear through the committer itself &mdash;it will
+fail with the error message in the previous section, but may arise
+if other applications are attempting to create files under the path `/__magic/`.
+
+Make sure the filesytem meets the requirements of the magic committer
+(a consistent S3A filesystem through S3Guard or the S3 service itself),
+and set the `fs.s3a.committer.magic.enabled` flag to indicate that magic file
+writes are supported.
+
+
+### `FileOutputCommitter` appears to be still used (from logs or delays in commits)
+
+The Staging committers use the original `FileOutputCommitter` to manage
+the propagation of commit information: do not worry if it the logs show `FileOutputCommitter`
+work with data in the cluster filesystem (e.g. HDFS).
+
+One way to make sure that the `FileOutputCommitter` is not being used to write
+the data to S3 is to set the option `mapreduce.fileoutputcommitter.algorithm.version`
+to a value such as "10". Because the only supported algorithms are "1" and "2",
+any erroneously created `FileOutputCommitter` will raise an exception in its constructor
+when instantiated:
+
+```
+java.io.IOException: Only 1 or 2 algorithm version is supported
+at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(FileOutputCommitter.java:130)
+at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(FileOutputCommitter.java:104)
+at org.apache.parquet.hadoop.ParquetOutputCommitter.<init>(ParquetOutputCommitter.java:42)
+at org.apache.parquet.hadoop.ParquetOutputFormat.getOutputCommitter(ParquetOutputFormat.java:395)
+at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.setupCommitter(HadoopMapReduceCommitProtocol.scala:67)
+at com.hortonworks.spark.cloud.commit.PathOutputCommitProtocol.setupCommitter(PathOutputCommitProtocol.scala:62)
+at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.setupJob(HadoopMapReduceCommitProtocol.scala:124)
+at com.hortonworks.spark.cloud.commit.PathOutputCommitProtocol.setupJob(PathOutputCommitProtocol.scala:152)
+at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:175)
+at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:145)
+```
+
+While that will not make the problem go away, it will at least make
+the failure happen at the start of a job.
+
+(Setting this option will not interfer with the Staging Committers' use of HDFS,
+as it explicitly sets the algorithm to "2" for that part of its work).
+
+The other way to check which committer to use is to examine the `_SUCCESS` file.
+If it is 0-bytes long, the classic `FileOutputCommitter` committed the job.
+The S3A committers all write a non-empty JSON file; the `committer` field lists
+the committer used.
+
+
+*Common causes*
+
+1. The property `fs.s3a.committer.name` is set to "file". Fix: change.
+1. The job has overridden the property `mapreduce.outputcommitter.factory.class`
+with a new factory class for all committers. This takes priority over
+all committers registered for the s3a:// schema.
+1. The property `mapreduce.outputcommitter.factory.scheme.s3a` is unset.
+1. The output format has overridden `FileOutputFormat.getOutputCommitter()`
+and is returning its own committer -one which is a subclass of `FileOutputCommitter`.
+
+That final cause. *the output format is returning its own committer*, is not
+easily fixed; it may be that the custom committer performs critical work
+during its lifecycle, and contains assumptions about the state of the written
+data during task and job commit (i.e. it is in the destination filesystem).
+Consult with the authors/maintainers of the output format
+to see whether it would be possible to integrate with the new committer factory
+mechanism and object-store-specific commit algorithms.
+
+Parquet is a special case here: its committer does no extra work
+other than add the option to read all newly-created files then write a schema
+summary. The Spark integration has explicit handling for Parquet to enable it
+to support the new committers, removing this (slow on S3) option.
+
+If you have subclassed `FileOutputCommitter` and want to move to the
+factory model, please get in touch.
+
+
+## Job/Task fails with PathExistsException: Destination path exists and committer conflict resolution mode is "fail"
+
+This surfaces when either of two conditions are met.
+
+1. The Directory committer is used with `fs.s3a.committer.staging.conflict-mode` set to
+`fail` and the output/destination directory exists.
+The job will fail in the driver during job setup.
+1. The Partitioned Committer is used with `fs.s3a.committer.staging.conflict-mode` set to
+`fail`  and one of the partitions. The specific task(s) generating conflicting data will fail
+during task commit, which will cause the entire job to fail.
+
+If you are trying to write data and want write conflicts to be rejected, this is the correct
+behavior: there was data at the destination so the job was aborted.
+
+## Staging committer task fails with IOException: No space left on device
+
+There's not enough space on the local hard disk (real or virtual)
+to store all the uncommitted data of the active tasks on that host.
+Because the staging committers write all output to the local disk
+and only upload the data on task commits, enough local temporary
+storage is needed to store all output generated by all uncommitted
+tasks running on the single host. Small EC2 VMs may run out of disk.
+
+1. Make sure that `fs.s3a.buffer.dir` includes a temporary directory on
+every available hard disk; this spreads load better.
+
+1. Add more disk space. In EC2: request instances with more local storage.
+There is no need for EMR storage; this is just for temporary data.
+
+1. Purge the directories listed in `fs.s3a.buffer.dir` of old data.
+Failed tasks may not clean up all old files.
+
+1. Reduce the number of worker threads/process in the host.
+
+1. Consider partitioning the job into more tasks. This *may* result in more tasks
+generating less data each.
+
+1. Use the magic committer. This only needs enough disk storage to buffer
+blocks of the currently being written file during their upload process,
+so can use a lot less disk space.

+ 172 - 16
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -28,6 +28,8 @@ See also:
 * [Encryption](./encryption.html)
 * [S3Guard](./s3guard.html)
 * [Troubleshooting](./troubleshooting_s3a.html)
+* [Committing work to S3 with the "S3A Committers"](./committers.html)
+* [S3A Committers Architecture](./committer_architecture.html)
 * [Testing](./testing.html)
 
 ##<a name="overview"></a> Overview
@@ -82,7 +84,7 @@ the Hadoop project itself.
 1. Amazon EMR's `s3://` client. This is from the Amazon EMR team, who actively
 maintain it.
 1. Apache's Hadoop's [`s3n:` filesystem client](./s3n.html).
-   This connectore is no longer available: users must migrate to the newer `s3a:` client.
+   This connector is no longer available: users must migrate to the newer `s3a:` client.
 
 
 ##<a name="getting_started"></a> Getting Started
@@ -177,6 +179,7 @@ Parts of Hadoop relying on this can have unexpected behaviour. E.g. the
 `AggregatedLogDeletionService` of YARN will not remove the appropriate logfiles.
 * Directory listing can be slow. Use `listFiles(path, recursive)` for high
 performance recursive listings whenever possible.
+* It is possible to create files under files if the caller tries hard.
 * The time to rename a directory is proportional to the number of files
 underneath it (directory or indirectly) and the size of the files. (The copyis
 executed inside the S3 storage, so the time is independent of the bandwidth
@@ -184,8 +187,13 @@ from client to S3).
 * Directory renames are not atomic: they can fail partway through, and callers
 cannot safely rely on atomic renames as part of a commit algorithm.
 * Directory deletion is not atomic and can fail partway through.
-* It is possible to create files under files if the caller tries hard.
 
+The final three issues surface when using S3 as the immediate destination
+of work, as opposed to HDFS or other "real" filesystem.
+
+The [S3A committers](./committers.html) are the sole mechanism available
+to safely save the output of queries directly into S3 object stores
+through the S3A filesystem.
 
 
 ### Warning #3: Object stores have differerent authorization models
@@ -223,18 +231,6 @@ Do not inadvertently share these credentials through means such as
 
 If you do any of these: change your credentials immediately!
 
-### Warning #5: The S3A client cannot be used on Amazon EMR
-
-On Amazon EMR `s3a://` URLs are not supported; Amazon provide
-their own filesystem client, `s3://`.
-If you are using Amazon EMR, follow their instructions for use —and be aware
-that all issues related to S3 integration in EMR can only be addressed by Amazon
-themselves: please raise your issues with them.
-
-Equally importantly: much of this document does not apply to the EMR `s3://` client.
-Pleae consult
-[the EMR storage documentation](http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-plan-file-systems.html)
-instead.
 
 ## <a name="authenticating"></a> Authenticating with S3
 
@@ -616,7 +612,7 @@ Because the provider path is not itself a sensitive secret, there is no risk
 from placing its declaration on the command line.
 
 
-## <a name="general_configuration"></a>Genaral S3A Client configuration
+## <a name="general_configuration"></a>General S3A Client configuration
 
 All S3A client options are configured with options with the prefix `fs.s3a.`.
 
@@ -875,6 +871,166 @@ options are covered in [Testing](./testing.md).
 </property>
 ```
 
+## <a name="retry_and_recovery"></a>Retry and Recovery
+
+The S3A client makes a best-effort attempt at recovering from network failures;
+this section covers the details of what it does.
+
+The S3A divides exceptions returned by the AWS SDK into different categories,
+and chooses a differnt retry policy based on their type and whether or
+not the failing operation is idempotent.
+
+
+### Unrecoverable Problems: Fail Fast
+
+* No object/bucket store: `FileNotFoundException`
+* No access permissions: `AccessDeniedException`
+* Network errors considered unrecoverable (`UnknownHostException`,
+ `NoRouteToHostException`, `AWSRedirectException`).
+* Interruptions: `InterruptedIOException`, `InterruptedException`.
+* Rejected HTTP requests: `InvalidRequestException`
+
+These are all considered unrecoverable: S3A will make no attempt to recover
+from them.
+
+### Possibly Recoverable Problems: Retry
+
+* Connection timeout: `ConnectTimeoutException`. Timeout before
+setting up a connection to the S3 endpoint (or proxy).
+* HTTP response status code 400, "Bad Request"
+
+The status code 400, Bad Request usually means that the request
+is unrecoverable; it's the generic "No" response. Very rarely it
+does recover, which is why it is in this category, rather than that
+of unrecoverable failures.
+
+These failures will be retried with a fixed sleep interval set in
+`fs.s3a.retry.interval`, up to the limit set in `fs.s3a.retry.limit`.
+
+
+### Only retrible on idempotent operations
+
+Some network failures are considered to be retriable if they occur on
+idempotent operations; there's no way to know if they happened
+after the request was processed by S3.
+
+* `SocketTimeoutException`: general network failure.
+* `EOFException` : the connection was broken while reading data
+* "No response from Server" (443, 444) HTTP responses.
+* Any other AWS client, service or S3 exception.
+
+These failures will be retried with a fixed sleep interval set in
+`fs.s3a.retry.interval`, up to the limit set in `fs.s3a.retry.limit`.
+
+*Important*: DELETE is considered idempotent, hence: `FileSystem.delete()`
+and `FileSystem.rename()` will retry their delete requests on any
+of these failures.
+
+The issue of whether delete should be idempotent has been a source
+of historical controversy in Hadoop.
+
+1. In the absence of any other changes to the object store, a repeated
+DELETE request will eventually result in the named object being deleted;
+it's a no-op if reprocessed. As indeed, is `Filesystem.delete()`.
+1. If another client creates a file under the path, it will be deleted.
+1. Any filesystem supporting an atomic `FileSystem.create(path, overwrite=false)`
+operation to reject file creation if the path exists MUST NOT consider
+delete to be idempotent, because a `create(path, false)` operation will
+only succeed if the first `delete()` call has already succeded.
+1. And a second, retried `delete()` call could delete the new data.
+
+Because S3 is eventially consistent *and* doesn't support an
+atomic create-no-overwrite operation, the choice is more ambigious.
+
+Currently S3A considers delete to be
+idempotent because it is convenient for many workflows, including the
+commit protocols. Just be aware that in the presence of transient failures,
+more things may be deleted than expected. (For anyone who considers this to
+be the wrong decision: rebuild the `hadoop-aws` module with the constant
+`S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT` set to `false`).
+
+
+
+
+
+
+### Throttled requests from S3 and Dynamo DB
+
+
+When S3A or Dynamo DB returns a response indicating that requests
+from the caller are being throttled, an exponential back-off with
+an initial interval and a maximum number of requests.
+
+```xml
+<property>
+  <name>fs.s3a.retry.throttle.limit</name>
+  <value>${fs.s3a.attempts.maximum}</value>
+  <description>
+    Number of times to retry any throttled request.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.throttle.interval</name>
+  <value>1000ms</value>
+  <description>
+    Interval between retry attempts on throttled requests.
+  </description>
+</property>
+```
+
+Notes
+
+1. There is also throttling taking place inside the AWS SDK; this is managed
+by the value `fs.s3a.attempts.maximum`.
+1. Throttling events are tracked in the S3A filesystem metrics and statistics.
+1. Amazon KMS may thottle a customer based on the total rate of uses of
+KMS *across all user accounts and applications*.
+
+Throttling of S3 requests is all too common; it is caused by too many clients
+trying to access the same shard of S3 Storage. This generatlly
+happen if there are too many reads, those being the most common in Hadoop
+applications. This problem is exacerbated by Hive's partitioning
+strategy used when storing data, such as partitioning by year and then month.
+This results in paths with little or no variation at their start, which ends
+up in all the data being stored in the same shard(s).
+
+Here are some expensive operations; the more of these taking place
+against part of an S3 bucket, the more load it experiences.
+* Many clients trying to list directories or calling `getFileStatus` on
+paths (LIST and HEAD requests respectively)
+* The GET requests issued when reading data.
+* Random IO used when reading columnar data (ORC, Parquet) means that many
+more GET requests than a simple one-per-file read.
+* The number of active writes to that part of the S3 bucket.
+
+A special case is when enough data has been written into part of an S3 bucket
+that S3 decides to split the data across more than one shard: this
+is believed to be one by some copy operation which can take some time.
+While this is under way, S3 clients access data under these paths will
+be throttled more than usual.
+
+
+Mitigation strategies
+
+1. Use separate buckets for intermediate data/different applications/roles.
+1. Use significantly different paths for different datasets in the same bucket.
+1. Increase the value of `fs.s3a.retry.throttle.interval` to provide
+longer delays between attempts.
+1. Reduce the parallelism of the queries. The more tasks trying to access
+data in parallel, the more load.
+1. Reduce `fs.s3a.threads.max` to reduce the amount of parallel operations
+performed by clients.
+!. Maybe: increase `fs.s3a.readahead.range` to increase the minimum amount
+of data asked for in every GET request, as well as how much data is
+skipped in the existing stream before aborting it and creating a new stream.
+1. If the DynamoDB tables used by S3Guard are being throttled, increase
+the capacity through `hadoop s3guard set-capacity` (and pay more, obviously).
+1. KMS: "consult AWS about increating your capacity".
+
+
+
+
 ## <a name="per_bucket_configuration"></a>Configuring different S3 buckets with Per-Bucket Configuration
 
 Different S3 buckets can be accessed with different S3A client configurations.
@@ -1081,7 +1237,7 @@ The original S3A client implemented file writes by
 buffering all data to disk as it was written to the `OutputStream`.
 Only when the stream's `close()` method was called would the upload start.
 
-This can made output slow, especially on large uploads, and could even
+This made output slow, especially on large uploads, and could even
 fill up the disk space of small (virtual) disks.
 
 Hadoop 2.7 added the `S3AFastOutputStream` alternative, which Hadoop 2.8 expanded.

+ 5 - 2
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md

@@ -426,7 +426,7 @@ hadoop s3guard diff s3a://ireland-1
 Prints and optionally checks the s3guard and encryption status of a bucket.
 
 ```bash
-hadoop s3guard bucket-info [ -guarded ] [-unguarded] [-auth] [-nonauth] [-encryption ENCRYPTION] s3a://BUCKET
+hadoop s3guard bucket-info [ -guarded ] [-unguarded] [-auth] [-nonauth] [-magic] [-encryption ENCRYPTION] s3a://BUCKET
 ```
 
 Options
@@ -437,6 +437,7 @@ Options
 | `-unguarded` | Require S3Guard to be disabled |
 | `-auth` | Require the S3Guard mode to be "authoritative" |
 | `-nonauth` | Require the S3Guard mode to be "non-authoritative" |
+| `-magic` | Require the S3 filesystem to be support the "magic" committer |
 | `-encryption <type>` | Require a specific server-side encryption algorithm  |
 
 The server side encryption options are not directly related to S3Guard, but
@@ -445,10 +446,11 @@ it is often convenient to check them at the same time.
 Example
 
 ```bash
-hadoop s3guard bucket-info -guarded s3a://ireland-1
+hadoop s3guard bucket-info -guarded -magic s3a://ireland-1
 ```
 
 List the details of bucket `s3a://ireland-1`, mandating that it must have S3Guard enabled
+("-guarded") and that support for the magic S3A committer is enabled ("-magic")
 
 ```
 Filesystem s3a://ireland-1
@@ -476,6 +478,7 @@ Metadata Store Diagnostics:
     TableSizeBytes: 12812,ItemCount: 91,
     TableArn: arn:aws:dynamodb:eu-west-1:00000000:table/ireland-1,}
   write-capacity=20
+The "magic" committer is supported
 
 S3A Client
   Endpoint: fs.s3a.endpoint=s3-eu-west-1.amazonaws.com

+ 60 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md

@@ -629,6 +629,10 @@ Don't assume AWS S3 US-East only, do allow for working with external S3 implemen
 Those may be behind the latest S3 API features, not support encryption, session
 APIs, etc.
 
+They won't have the same CSV test files as some of the input tests rely on.
+Look at `ITestS3AInputStreamPerformance` to see how tests can be written
+to support the declaration of a specific large test file on alternate filesystems.
+
 
 ### Works Over Long-haul Links
 
@@ -662,6 +666,37 @@ listings, file status, ...), so help make failures easier to understand.
 At the very least, do not use `assertTrue()` or `assertFalse()` without
 including error messages.
 
+### Sets up its filesystem and checks for those settings
+
+Tests can overrun `createConfiguration()` to add new options to the configuration
+file for the S3A Filesystem instance used in their tests.
+
+However, filesystem caching may mean that a test suite may get a cached
+instance created with an differennnt configuration. For tests which don't need
+specific configurations caching is good: it reduces test setup time.
+
+For those tests which do need unique options (encryption, magic files),
+things can break, and they will do so in hard-to-replicate ways.
+
+Use `S3ATestUtils.disableFilesystemCaching(conf)` to disable caching when
+modifying the config. As an example from `AbstractTestS3AEncryption`:
+
+```java
+@Override
+protected Configuration createConfiguration() {
+  Configuration conf = super.createConfiguration();
+  S3ATestUtils.disableFilesystemCaching(conf);
+  conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+          getSSEAlgorithm().getMethod());
+  return conf;
+}
+```
+
+Then verify in the setup method or test cases that their filesystem actually has
+the desired feature (`fs.getConf().getProperty(...)`). This not only
+catches filesystem reuse problems, it catches the situation where the
+filesystem configuration in `auth-keys.xml` has explicit per-bucket settings
+which override the test suite's general option settings.
 
 ### Cleans Up Afterwards
 
@@ -677,6 +712,31 @@ get called.
 
 We really appreciate this &mdash; you will too.
 
+### Runs in parallel unless this is unworkable.
+
+Tests must be designed to run in parallel with other tests, all working
+with the same shared S3 bucket. This means
+
+* Uses relative and JVM-fork-unique paths provided by the method
+  `AbstractFSContractTestBase.path(String filepath)`.
+* Doesn't manipulate the root directory or make assertions about its contents
+(for example: delete its contents and assert that it is now empty).
+* Doesn't have a specific requirement of all active clients of the bucket
+(example: SSE-C tests which require all files, even directory markers,
+to be encrypted with the same key).
+* Doesn't use so much bandwidth that all other tests will be starved of IO and
+start timing out (e.g. the scale tests).
+
+Tests such as these can only be run as sequential tests. When adding one,
+exclude it in the POM file. from the parallel failsafe run and add to the
+sequential one afterwards. The IO heavy ones must also be subclasses of
+`S3AScaleTestBase` and so only run if the system/maven property
+`fs.s3a.scale.test.enabled` is true.
+
+## Individual test cases can be run in an IDE
+
+This is invaluable for debugging test failures.
+
 
 ## <a name="tips"></a> Tips
 

+ 124 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

@@ -721,3 +721,127 @@ http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) -
 http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Server: AmazonS3
 execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be kept alive for 60000 MILLISECONDS
 ```
+
+
+## <a name="retries"></a>  Reducing failures by configuring retry policy
+
+The S3A client can ba configured to rety those operations which are considered
+retriable. That can be because they are idempotent, or
+because there failure happened before the request was processed by S3.
+
+The number of retries and interval between each retry can be configured:
+
+```xml
+<property>
+  <name>fs.s3a.attempts.maximum</name>
+  <value>20</value>
+  <description>How many times we should retry commands on transient errors,
+  excluding throttling errors.</description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.interval</name>
+  <value>500ms</value>
+  <description>
+    Interval between retry attempts.
+  </description>
+</property>
+```
+
+Not all failures are retried. Specifically excluded are those considered
+unrecoverable:
+
+* Low-level networking: `UnknownHostException`, `NoRouteToHostException`.
+* 302 redirects
+* Missing resources, 404/`FileNotFoundException`
+* HTTP 416 response/`EOFException`. This can surface if the length of a file changes
+  while another client is reading it.
+* Failures during execution or result processing of non-idempotent operations where
+it is considered likely that the operation has already taken place.
+
+In future, others may be added to this list.
+
+When one of these failures arises in the S3/S3A client, the retry mechanism
+is bypassed and the operation will fail.
+
+*Warning*: the S3A client considers DELETE, PUT and COPY operations to
+be idempotent, and will retry them on failure. These are only really idempotent
+if no other client is attempting to manipulate the same objects, such as:
+renaming() the directory tree or uploading files to the same location.
+Please don't do that. Given that the emulated directory rename and delete operations
+aren't atomic, even without retries, multiple S3 clients working with the same
+paths can interfere with each other
+
+#### <a name="retries"></a> Throttling
+
+When many requests are made of a specific S3 bucket (or shard inside it),
+S3 will respond with a 503 "throttled" response.
+Throttling can be recovered from, provided overall load decreases.
+Furthermore, because it is sent before any changes are made to the object store,
+is inherently idempotent. For this reason, the client will always attempt to
+retry throttled requests.
+
+The limit of the number of times a throttled request can be retried,
+and the exponential interval increase between attempts, can be configured
+independently of the other retry limits.
+
+```xml
+<property>
+  <name>fs.s3a.retry.throttle.limit</name>
+  <value>20</value>
+  <description>
+    Number of times to retry any throttled request.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.retry.throttle.interval</name>
+  <value>500ms</value>
+  <description>
+    Interval between retry attempts on throttled requests.
+  </description>
+</property>
+```
+
+If a client is failing due to `AWSServiceThrottledException` failures,
+increasing the interval and limit *may* address this. However, it
+it is a sign of AWS services being overloaded by the sheer number of clients
+and rate of requests. Spreading data across different buckets, and/or using
+a more balanced directory structure may be beneficial.
+Consult [the AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/request-rate-perf-considerations.html).
+
+Reading or writing data encrypted with SSE-KMS forces S3 to make calls of
+the AWS KMS Key Management Service, which comes with its own
+[Request Rate Limits](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
+These default to 1200/second for an account, across all keys and all uses of
+them, which, for S3 means: across all buckets with data encrypted with SSE-KMS.
+
+###### Tips to Keep Throttling down
+
+* If you are seeing a lot of throttling responses on a large scale
+operation like a `distcp` copy, *reduce* the number of processes trying
+to work with the bucket (for distcp: reduce the number of mappers with the
+`-m` option).
+
+* If you are reading or writing lists of files, if you can randomize
+the list so they are not processed in a simple sorted order, you may
+reduce load on a specific shard of S3 data, so potentially increase throughput.
+
+* An S3 Bucket is throttled by requests coming from all
+simultaneous clients. Different applications and jobs may interfere with
+each other: consider that when troubleshooting.
+Partitioning data into different buckets may help isolate load here.
+
+* If you are using data encrypted with SSE-KMS, then the
+will also apply: these are stricter than the S3 numbers.
+If you believe that you are reaching these limits, you may be able to
+get them increased.
+Consult [the KMS Rate Limit documentation](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
+
+* S3Guard uses DynamoDB for directory and file lookups;
+it is rate limited to the amount of (guaranteed) IO purchased for a
+table. If significant throttling events/rate is observed here, the preallocated
+IOPs can be increased with the `s3guard set-capacity` command, or
+through the AWS Console. Throttling events in S3Guard are noted in logs, and
+also in the S3A metrics `s3guard_metadatastore_throttle_rate` and
+`s3guard_metadatastore_throttled`.

+ 11 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java

@@ -21,12 +21,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
 import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 /**
  * S3A contract tests covering getFileStatus.
+ * Some of the tests can take too long when the fault injection rate is high,
+ * so the test timeout is extended.
  */
 public class ITestS3AContractGetFileStatus
     extends AbstractContractGetFileStatusTest {
@@ -52,4 +55,12 @@ public class ITestS3AContractGetFileStatus
     maybeEnableS3Guard(conf);
     return conf;
   }
+
+  /**
+   * {@inheritDoc}
+   * @return S3A test timeout.
+   */
+  protected int getTestTimeoutMillis() {
+    return S3ATestConstants.S3A_TEST_TIMEOUT;
+  }
 }

+ 4 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java

@@ -26,6 +26,7 @@ import com.amazonaws.services.s3.AmazonS3;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
 import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
 
@@ -62,10 +63,12 @@ public abstract class AbstractS3AMockTest {
     // test we don't issue request to AWS DynamoDB service.
     conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
         MetadataStore.class);
+    // FS is always magic
+    conf.setBoolean(CommitConstants.MAGIC_COMMITTER_ENABLED, true);
     fs = new S3AFileSystem();
     URI uri = URI.create(FS_S3A + "://" + BUCKET);
     fs.initialize(uri, conf);
-    s3 = fs.getAmazonS3Client();
+    s3 = fs.getAmazonS3ClientForTesting("mocking");
   }
 
   @After

+ 21 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java

@@ -29,11 +29,13 @@ import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED;
 
 /**
  * An extension of the contract test base set up for S3A tests.
@@ -58,7 +60,11 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
 
   @Before
   public void nameThread() {
-    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+    Thread.currentThread().setName("JUnit-" + getMethodName());
+  }
+
+  protected String getMethodName() {
+    return methodName.getMethodName();
   }
 
   @Override
@@ -75,6 +81,19 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
     Configuration conf = super.createConfiguration();
     // patch in S3Guard options
     maybeEnableS3Guard(conf);
+    // set hadoop temp dir to a default value
+    String testUniqueForkId =
+        System.getProperty(TEST_UNIQUE_FORK_ID);
+    String tmpDir = conf.get(Constants.HADOOP_TMP_DIR, "target/build/test");
+    if (testUniqueForkId != null) {
+      // patch temp dir for the specific branch
+      tmpDir = tmpDir + File.pathSeparatorChar + testUniqueForkId;
+      conf.set(Constants.HADOOP_TMP_DIR, tmpDir);
+    }
+    conf.set(Constants.BUFFER_DIR, tmpDir);
+    // add this so that even on tests where the FS is shared,
+    // the FS is always "magic"
+    conf.setBoolean(MAGIC_COMMITTER_ENABLED, true);
     return conf;
   }
 
@@ -98,7 +117,7 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
    */
   protected void describe(String text, Object... args) {
     LOG.info("\n\n{}: {}\n",
-        methodName.getMethodName(),
+        getMethodName(),
         String.format(text, args));
   }
 

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -110,7 +110,7 @@ public class ITestS3AConfiguration {
     } else {
       conf.set(Constants.ENDPOINT, endpoint);
       fs = S3ATestUtils.createTestFileSystem(conf);
-      AmazonS3 s3 = fs.getAmazonS3Client();
+      AmazonS3 s3 = fs.getAmazonS3ClientForTesting("test endpoint");
       String endPointRegion = "";
       // Differentiate handling of "s3-" and "s3." based endpoint identifiers
       String[] endpointParts = StringUtils.split(endpoint, '.');
@@ -378,7 +378,7 @@ public class ITestS3AConfiguration {
     try {
       fs = S3ATestUtils.createTestFileSystem(conf);
       assertNotNull(fs);
-      AmazonS3 s3 = fs.getAmazonS3Client();
+      AmazonS3 s3 = fs.getAmazonS3ClientForTesting("configuration");
       assertNotNull(s3);
       S3ClientOptions clientOptions = getField(s3, S3ClientOptions.class,
           "clientOptions");
@@ -402,7 +402,7 @@ public class ITestS3AConfiguration {
     conf = new Configuration();
     fs = S3ATestUtils.createTestFileSystem(conf);
     assertNotNull(fs);
-    AmazonS3 s3 = fs.getAmazonS3Client();
+    AmazonS3 s3 = fs.getAmazonS3ClientForTesting("User Agent");
     assertNotNull(s3);
     ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
         "clientConfiguration");
@@ -416,7 +416,7 @@ public class ITestS3AConfiguration {
     conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
     fs = S3ATestUtils.createTestFileSystem(conf);
     assertNotNull(fs);
-    AmazonS3 s3 = fs.getAmazonS3Client();
+    AmazonS3 s3 = fs.getAmazonS3ClientForTesting("User agent");
     assertNotNull(s3);
     ClientConfiguration awsConf = getField(s3, ClientConfiguration.class,
         "clientConfiguration");

+ 2 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java

@@ -222,11 +222,10 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     //unencrypted can access until the final directory
     unencryptedFileSystem.listFiles(pathA, true);
     unencryptedFileSystem.listFiles(pathAB, true);
-    AWSS3IOException ex = intercept(AWSS3IOException.class,
+    AWSBadRequestException ex = intercept(AWSBadRequestException.class,
         () -> {
           unencryptedFileSystem.listFiles(pathABC, false);
         });
-    assertStatusCode(ex, 400);
   }
 
   /**
@@ -270,11 +269,10 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     unencryptedFileSystem.listStatus(pathA);
     unencryptedFileSystem.listStatus(pathAB);
 
-    AWSS3IOException ex = intercept(AWSS3IOException.class,
+    intercept(AWSBadRequestException.class,
         () -> {
           unencryptedFileSystem.listStatus(pathABC);
         });
-    assertStatusCode(ex, 400);
   }
 
   /**

+ 16 - 33
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java

@@ -23,9 +23,7 @@ import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.test.LambdaTestUtils;
 
 import org.junit.Assume;
 import org.junit.Test;
@@ -34,11 +32,12 @@ import org.slf4j.LoggerFactory;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.Callable;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.*;
 
 /**
  * Test S3A Failure translation, including a functional test
@@ -73,13 +72,9 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
       writeDataset(fs, testpath, shortDataset, shortDataset.length, 1024, true);
       // here the file length is less. Probe the file to see if this is true,
       // with a spin and wait
-      LambdaTestUtils.eventually(30 * 1000, 1000,
-          new Callable<Void>() {
-            @Override
-            public Void call() throws Exception {
-              assertEquals(shortLen, fs.getFileStatus(testpath).getLen());
-              return null;
-            }
+      eventually(30 * 1000, 1000,
+          () -> {
+            assertEquals(shortLen, fs.getFileStatus(testpath).getLen());
           });
 
       // here length is shorter. Assuming it has propagated to all replicas,
@@ -99,12 +94,8 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
           instream.read(instream.getPos(), buf, 0, buf.length));
 
       // now do a block read fully, again, backwards from the current pos
-      try {
-        instream.readFully(shortLen + 512, buf);
-        fail("Expected readFully to fail");
-      } catch (EOFException expected) {
-        LOG.debug("Expected EOF: ", expected);
-      }
+      intercept(EOFException.class, "", "readfully",
+          () -> instream.readFully(shortLen + 512, buf));
 
       assertIsEOF("read(offset)",
           instream.read(shortLen + 510, buf, 0, buf.length));
@@ -115,19 +106,10 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
       // delete the file. Reads must fail
       fs.delete(testpath, false);
 
-      try {
-        int r = instream.read();
-        fail("Expected an exception, got " + r);
-      } catch (FileNotFoundException e) {
-        // expected
-      }
-
-      try {
-        instream.readFully(2048, buf);
-        fail("Expected readFully to fail");
-      } catch (FileNotFoundException e) {
-        // expected
-      }
+      intercept(FileNotFoundException.class, "", "read()",
+          () -> instream.read());
+      intercept(FileNotFoundException.class, "", "readfully",
+          () -> instream.readFully(2048, buf));
 
     }
   }
@@ -149,7 +131,7 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
   }
 
   private void removeKeys(S3AFileSystem fileSystem, String... keys)
-      throws InvalidRequestException {
+      throws IOException {
     List<DeleteObjectsRequest.KeyVersion> request = new ArrayList<>(
         keys.length);
     for (String key : keys) {
@@ -168,14 +150,15 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
     timer.end("removeKeys");
   }
 
-  @Test(expected = MultiObjectDeleteException.class)
+  @Test
   public void testMultiObjectDeleteNoPermissions() throws Throwable {
     Configuration conf = getConfiguration();
     String csvFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
     Assume.assumeTrue("CSV test file is not the default",
         DEFAULT_CSVTEST_FILE.equals(csvFile));
     Path testFile = new Path(csvFile);
-    S3AFileSystem fs = (S3AFileSystem)FileSystem.newInstance(testFile.toUri(), conf);
-    removeKeys(fs, fs.pathToKey(testFile));
+    S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem(conf);
+    intercept(MultiObjectDeleteException.class,
+        () -> removeKeys(fs, fs.pathToKey(testFile)));
   }
 }

+ 2 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java

@@ -203,6 +203,8 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     // before the internal behavior w/ or w/o metadata store.
     assumeFalse(fs.hasMetadataStore());
 
+    skipDuringFaultInjection(fs);
+
     Path srcBaseDir = path("src");
     mkdirs(srcBaseDir);
     MetricDiff deleteRequests =

Some files were not shown because too many files changed in this diff