Parcourir la source

HADOOP-18889. S3A v2 SDK third party support (#6141)

Tune AWS v2 SDK changes based on testing with third party stores
including GCS.

Contains HADOOP-18889. S3A v2 SDK error translations and troubleshooting docs

* Changes needed to work with multiple third party stores
* New third_party_stores document on how to bind to and test
  third party stores, including google gcs (which works!)
* Troubleshooting docs mostly updated for v2 SDK

Exception translation/resilience

* New AWSUnsupportedFeatureException for unsupported/unavailable errors
* Handle 501 method unimplemented as one of these
* Error codes > 500 mapped to the AWSStatus500Exception if no explicit
  handler.
* Precondition errors handled a bit better
* GCS throttle exception also recognized.
* GCS raises 404 on a delete of a file which doesn't exist: swallow it.
* Error translation uses reflection to create IOE of the right type.
  All IOEs at the bottom of an AWS stack chain are regenerated.
  then a new exception of that specific type is created, with the top level ex
  its cause. This is done to retain the whole stack chain.
* Reduce the number of retries within the AWS SDK
* And those of s3a code.
* S3ARetryPolicy explicitly declare SocketException as connectivity failure
  but subclasses BindException
* SocketTimeoutException also considered connectivity
* Log at debug whenever retry policies looked up
* Reorder exceptions to alphabetical order, with commentary
* Review use of the Invoke.retry() method

 The reduction in retries is because its clear when you try to create a bucket
 which doesn't resolve that the time for even an UnknownHostException to
 eventually fail over 90s, which then hit the s3a retry code.
 - Reducing the SDK retries means these escalate to our code better.
 - Cutting back on our own retries makes it a bit more responsive for most real
 deployments.
 - maybeTranslateNetworkException() and s3a retry policy means that
   unknown host exception is recognised and fails fast.

Contributed by Steve Loughran
Steve Loughran il y a 1 an
Parent
commit
c11b6ab8e4
52 fichiers modifiés avec 1971 ajouts et 1181 suppressions
  1. 8 2
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  2. 3 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java
  3. 8 8
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
  4. 46 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSUnsupportedFeatureException.java
  5. 5 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  6. 8 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
  7. 39 13
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  8. 85 40
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
  9. 29 10
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  10. 4 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java
  11. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
  12. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
  13. 30 12
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java
  14. 9 6
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java
  15. 7 7
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSClientConfig.java
  16. 69 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
  17. 7 5
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InstantiationIOException.java
  18. 10 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
  19. 2 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java
  20. 3 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
  21. 25 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
  22. 26 169
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
  23. 470 159
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  24. 66 12
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
  25. 415 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md
  26. 120 587
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
  27. 13 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java
  28. 4 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java
  29. 7 5
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  30. 25 14
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java
  31. 4 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java
  32. 77 22
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
  33. 19 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
  34. 3 9
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
  35. 11 9
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java
  36. 1 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java
  37. 8 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java
  38. 15 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
  39. 48 9
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  40. 2 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java
  41. 41 8
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
  42. 19 12
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java
  43. 16 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
  44. 4 8
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java
  45. 115 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java
  46. 1 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java
  47. 10 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java
  48. 10 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java
  49. 3 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java
  50. 1 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java
  51. 12 0
      hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
  52. 6 2
      hadoop-tools/hadoop-aws/src/test/resources/log4j.properties

+ 8 - 2
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1398,8 +1398,14 @@
 
 
 <property>
 <property>
   <name>fs.s3a.attempts.maximum</name>
   <name>fs.s3a.attempts.maximum</name>
-  <value>20</value>
-  <description>How many times we should retry commands on transient errors.</description>
+  <value>5</value>
+  <description>
+    Number of times the AWS client library should retry errors before
+    escalating to the S3A code: {@value}.
+    The S3A connector does its own selective retries; the only time the AWS
+    SDK operations are not wrapped is during multipart copy via the AWS SDK
+    transfer manager.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>

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

@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.s3a;
 
 
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 
 
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
+
 /**
 /**
  * A 400 "Bad Request" exception was received.
  * A 400 "Bad Request" exception was received.
  * This is the general "bad parameters, headers, whatever" failure.
  * This is the general "bad parameters, headers, whatever" failure.
@@ -28,7 +30,7 @@ public class AWSBadRequestException extends AWSServiceIOException {
   /**
   /**
    * HTTP status code which signals this failure mode was triggered: {@value}.
    * HTTP status code which signals this failure mode was triggered: {@value}.
    */
    */
-  public static final int STATUS_CODE = 400;
+  public static final int STATUS_CODE = SC_400_BAD_REQUEST;
 
 
   /**
   /**
    * Instantiate.
    * Instantiate.

+ 8 - 8
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java

@@ -21,13 +21,13 @@ package org.apache.hadoop.fs.s3a;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 
 
 /**
 /**
- * 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>
+ * A 5xx response came back from a service.
+ * The 500 error considered retriable by the AWS SDK, which will have already
+ * tried it {@code fs.s3a.attempts.maximum} times before reaching s3a
+ * code.
+ * How it handles other 5xx errors is unknown: S3A FS code will treat them
+ * as unrecoverable on the basis that they indicate some third-party store
+ * or gateway problem.
  */
  */
 public class AWSStatus500Exception extends AWSServiceIOException {
 public class AWSStatus500Exception extends AWSServiceIOException {
   public AWSStatus500Exception(String operation,
   public AWSStatus500Exception(String operation,
@@ -37,6 +37,6 @@ public class AWSStatus500Exception extends AWSServiceIOException {
 
 
   @Override
   @Override
   public boolean retryable() {
   public boolean retryable() {
-    return true;
+    return false;
   }
   }
 }
 }

+ 46 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSUnsupportedFeatureException.java

@@ -0,0 +1,46 @@
+/*
+ * 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 software.amazon.awssdk.awscore.exception.AwsServiceException;
+
+/**
+ * A store returned an error indicating that it does not support a
+ * specific S3 feature such as the chosen ChangeDetectionPolicy or
+ * other AWS-S3 feature that the third-party store does not support.
+ * The workaround is to disable use of the feature.
+ * Unrecoverable.
+ */
+public class AWSUnsupportedFeatureException extends AWSServiceIOException {
+
+  /**
+   * Instantiate.
+   * @param operation operation which triggered this
+   * @param cause the underlying cause
+   */
+  public AWSUnsupportedFeatureException(String operation,
+      AwsServiceException cause) {
+    super(operation, cause);
+  }
+
+  @Override
+  public boolean retryable() {
+    return false;
+  }
+}

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

@@ -228,6 +228,9 @@ public final class Constants {
   /**
   /**
    * Number of times the AWS client library should retry errors before
    * Number of times the AWS client library should retry errors before
    * escalating to the S3A code: {@value}.
    * escalating to the S3A code: {@value}.
+   * The S3A connector does its own selective retries; the only time the AWS
+   * SDK operations are not wrapped is during multipart copy via the AWS SDK
+   * transfer manager.
    */
    */
   public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum";
   public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum";
 
 
@@ -235,7 +238,7 @@ public final class Constants {
    * Default number of times the AWS client library should retry errors before
    * Default number of times the AWS client library should retry errors before
    * escalating to the S3A code: {@value}.
    * escalating to the S3A code: {@value}.
    */
    */
-  public static final int DEFAULT_MAX_ERROR_RETRIES = 10;
+  public static final int DEFAULT_MAX_ERROR_RETRIES = 5;
 
 
   /**
   /**
    * Experimental/Unstable feature: should the AWS client library retry
    * Experimental/Unstable feature: should the AWS client library retry
@@ -264,7 +267,7 @@ public final class Constants {
   // milliseconds until we give up trying to establish a connection to s3
   // milliseconds until we give up trying to establish a connection to s3
   public static final String ESTABLISH_TIMEOUT =
   public static final String ESTABLISH_TIMEOUT =
       "fs.s3a.connection.establish.timeout";
       "fs.s3a.connection.establish.timeout";
-  public static final int DEFAULT_ESTABLISH_TIMEOUT = 50000;
+  public static final int DEFAULT_ESTABLISH_TIMEOUT = 5000;
 
 
   // milliseconds until we give up on a connection to s3
   // milliseconds until we give up on a connection to s3
   public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout";
   public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout";

+ 8 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java

@@ -223,6 +223,14 @@ public final class MultipartUtils {
     /** Iterator over the current listing. */
     /** Iterator over the current listing. */
     private ListIterator<MultipartUpload> batchIterator;
     private ListIterator<MultipartUpload> batchIterator;
 
 
+    /**
+     * Construct an iterator to list uploads under a path.
+     * @param storeContext store context
+     * @param s3 s3 client
+     * @param maxKeys max # of keys to list per batch
+     * @param prefix prefix
+     * @throws IOException listing failure.
+     */
     @Retries.RetryTranslated
     @Retries.RetryTranslated
     public UploadIterator(
     public UploadIterator(
         final StoreContext storeContext,
         final StoreContext storeContext,

+ 39 - 13
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -334,6 +334,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
   /** Exactly once log to warn about setting the region in config to avoid probe. */
   /** Exactly once log to warn about setting the region in config to avoid probe. */
   private static final LogExactlyOnce SET_REGION_WARNING = new LogExactlyOnce(LOG);
   private static final LogExactlyOnce SET_REGION_WARNING = new LogExactlyOnce(LOG);
+
+  /** Log to warn of storage class configuration problems. */
+  private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG);
+
   private static final Logger PROGRESS =
   private static final Logger PROGRESS =
       LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
       LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
   private LocalDirAllocator directoryAllocator;
   private LocalDirAllocator directoryAllocator;
@@ -1075,7 +1079,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
 
             if (exception.statusCode() == SC_404_NOT_FOUND) {
             if (exception.statusCode() == SC_404_NOT_FOUND) {
               throw new UnknownStoreException("s3a://" + bucket + "/",
               throw new UnknownStoreException("s3a://" + bucket + "/",
-                  " Bucket does " + "not exist");
+                  " Bucket does not exist: " + exception,
+                  exception);
             }
             }
 
 
             throw exception;
             throw exception;
@@ -1176,6 +1181,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
 
     // Any encoding type
     // Any encoding type
     String contentEncoding = getConf().getTrimmed(CONTENT_ENCODING, null);
     String contentEncoding = getConf().getTrimmed(CONTENT_ENCODING, null);
+    if (contentEncoding != null) {
+      LOG.debug("Using content encoding set in {} = {}", CONTENT_ENCODING,  contentEncoding);
+    }
 
 
     String storageClassConf = getConf()
     String storageClassConf = getConf()
         .getTrimmed(STORAGE_CLASS, "")
         .getTrimmed(STORAGE_CLASS, "")
@@ -1183,10 +1191,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     StorageClass storageClass = null;
     StorageClass storageClass = null;
     if (!storageClassConf.isEmpty()) {
     if (!storageClassConf.isEmpty()) {
       storageClass = StorageClass.fromValue(storageClassConf);
       storageClass = StorageClass.fromValue(storageClassConf);
-
+      LOG.debug("Using storage class {}", storageClass);
       if (storageClass.equals(StorageClass.UNKNOWN_TO_SDK_VERSION)) {
       if (storageClass.equals(StorageClass.UNKNOWN_TO_SDK_VERSION)) {
-        LOG.warn("Unknown storage class property {}: {}; falling back to default storage class",
-            STORAGE_CLASS, storageClassConf);
+        STORAGE_CLASS_WARNING.warn("Unknown storage class \"{}\" from option: {};"
+                + " falling back to default storage class",
+            storageClassConf, STORAGE_CLASS);
         storageClass = null;
         storageClass = null;
       }
       }
 
 
@@ -1432,7 +1441,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     public String getBucketLocation(String bucketName) throws IOException {
     public String getBucketLocation(String bucketName) throws IOException {
       final String region = trackDurationAndSpan(
       final String region = trackDurationAndSpan(
           STORE_EXISTS_PROBE, bucketName, null, () ->
           STORE_EXISTS_PROBE, bucketName, null, () ->
-              invoker.retry("getBucketLocation()", bucketName, true, () ->
+              once("getBucketLocation()", bucketName, () ->
                   // If accessPoint then region is known from Arn
                   // If accessPoint then region is known from Arn
                   accessPoint != null
                   accessPoint != null
                       ? accessPoint.getRegion()
                       ? accessPoint.getRegion()
@@ -2990,7 +2999,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
                  "deleting %s", key)) {
                  "deleting %s", key)) {
       invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
       invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
           DELETE_CONSIDERED_IDEMPOTENT,
           DELETE_CONSIDERED_IDEMPOTENT,
-          ()-> {
+          () -> {
             incrementStatistic(OBJECT_DELETE_OBJECTS);
             incrementStatistic(OBJECT_DELETE_OBJECTS);
             trackDurationOfInvocation(getDurationTrackerFactory(),
             trackDurationOfInvocation(getDurationTrackerFactory(),
                 OBJECT_DELETE_REQUEST.getSymbol(),
                 OBJECT_DELETE_REQUEST.getSymbol(),
@@ -2999,6 +3008,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
                     .build()));
                     .build()));
             return null;
             return null;
           });
           });
+    } catch (AwsServiceException ase) {
+      // 404 errors get swallowed; this can be raised by
+      // third party stores (GCS).
+      if (!isObjectNotFound(ase)) {
+        throw ase;
+      }
     }
     }
   }
   }
 
 
@@ -4284,13 +4299,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
   }
 
 
   /**
   /**
-   * Verify that the input stream is open. Non blocking; this gives
+   * Verify that the filesystem has not been closed. Non blocking; this gives
    * the last state of the volatile {@link #closed} field.
    * the last state of the volatile {@link #closed} field.
-   * @throws IOException if the connection is closed.
+   * @throws PathIOException if the FS is closed.
    */
    */
-  private void checkNotClosed() throws IOException {
+  private void checkNotClosed() throws PathIOException {
     if (isClosed) {
     if (isClosed) {
-      throw new IOException(uri + ": " + E_FS_CLOSED);
+      throw new PathIOException(uri.toString(), E_FS_CLOSED);
     }
     }
   }
   }
 
 
@@ -4440,7 +4455,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // This means the File was deleted since LIST enumerated it.
       // This means the File was deleted since LIST enumerated it.
       LOG.debug("getObjectMetadata({}) failed to find an expected file",
       LOG.debug("getObjectMetadata({}) failed to find an expected file",
           srcKey, e);
           srcKey, e);
-      // We create an exception, but the text depends on the S3Guard state
       throw new RemoteFileChangedException(
       throw new RemoteFileChangedException(
           keyToQualifiedPath(srcKey).toString(),
           keyToQualifiedPath(srcKey).toString(),
           action,
           action,
@@ -4451,6 +4465,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     CopyObjectRequest.Builder copyObjectRequestBuilder =
     CopyObjectRequest.Builder copyObjectRequestBuilder =
         getRequestFactory().newCopyObjectRequestBuilder(srcKey, dstKey, srcom);
         getRequestFactory().newCopyObjectRequestBuilder(srcKey, dstKey, srcom);
     changeTracker.maybeApplyConstraint(copyObjectRequestBuilder);
     changeTracker.maybeApplyConstraint(copyObjectRequestBuilder);
+    final CopyObjectRequest copyRequest = copyObjectRequestBuilder.build();
+    LOG.debug("Copy Request: {}", copyRequest);
     CopyObjectResponse response;
     CopyObjectResponse response;
 
 
     // transfer manager is skipped if disabled or the file is too small to worry about
     // transfer manager is skipped if disabled or the file is too small to worry about
@@ -4465,7 +4481,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
 
             Copy copy = transferManager.copy(
             Copy copy = transferManager.copy(
                 CopyRequest.builder()
                 CopyRequest.builder()
-                    .copyObjectRequest(copyObjectRequestBuilder.build())
+                    .copyObjectRequest(copyRequest)
                     .build());
                     .build());
 
 
             try {
             try {
@@ -4474,6 +4490,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             } catch (CompletionException e) {
             } catch (CompletionException e) {
               Throwable cause = e.getCause();
               Throwable cause = e.getCause();
               if (cause instanceof SdkException) {
               if (cause instanceof SdkException) {
+                // if this is a 412 precondition failure, it may
+                // be converted to a RemoteFileChangedException
                 SdkException awsException = (SdkException)cause;
                 SdkException awsException = (SdkException)cause;
                 changeTracker.processException(awsException, "copy");
                 changeTracker.processException(awsException, "copy");
                 throw awsException;
                 throw awsException;
@@ -4490,7 +4508,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           () -> {
           () -> {
             LOG.debug("copyFile: single part copy {} -> {} of size {}", srcKey, dstKey, size);
             LOG.debug("copyFile: single part copy {} -> {} of size {}", srcKey, dstKey, size);
             incrementStatistic(OBJECT_COPY_REQUESTS);
             incrementStatistic(OBJECT_COPY_REQUESTS);
-            return s3Client.copyObject(copyObjectRequestBuilder.build());
+            try {
+              return s3Client.copyObject(copyRequest);
+            } catch (SdkException awsException) {
+              // if this is a 412 precondition failure, it may
+              // be converted to a RemoteFileChangedException
+              changeTracker.processException(awsException, "copy");
+              // otherwise, rethrow
+              throw awsException;
+            }
           });
           });
     }
     }
 
 

+ 85 - 40
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java

@@ -22,7 +22,10 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.InterruptedIOException;
+import java.net.BindException;
+import java.net.ConnectException;
 import java.net.NoRouteToHostException;
 import java.net.NoRouteToHostException;
+import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
 import java.nio.file.AccessDeniedException;
 import java.nio.file.AccessDeniedException;
@@ -31,7 +34,6 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
 import software.amazon.awssdk.core.exception.SdkException;
 import software.amazon.awssdk.core.exception.SdkException;
-import org.apache.hadoop.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -45,13 +47,14 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.ConnectTimeoutException;
 
 
 
 
+import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.io.retry.RetryPolicies.*;
 import static org.apache.hadoop.io.retry.RetryPolicies.*;
-
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 
 
+
 /**
 /**
  * The S3A request retry policy.
  * The S3A request retry policy.
- *
+ * <p>
  * This uses the retry options in the configuration file to determine retry
  * This uses the retry options in the configuration file to determine retry
  * count and delays for "normal" retries and separately, for throttling;
  * count and delays for "normal" retries and separately, for throttling;
  * the latter is best handled for longer with an exponential back-off.
  * the latter is best handled for longer with an exponential back-off.
@@ -66,20 +69,25 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
  * For non-idempotent operations, only failures due to throttling or
  * For non-idempotent operations, only failures due to throttling or
  * from failures which are known to only arise prior to talking to S3
  * from failures which are known to only arise prior to talking to S3
  * are retried.
  * are retried.
- *
+ * <p>
  * The retry policy is all built around that of the normal IO exceptions,
  * The retry policy is all built around that of the normal IO exceptions,
  * particularly those extracted from
  * particularly those extracted from
  * {@link S3AUtils#translateException(String, Path, SdkException)}.
  * {@link S3AUtils#translateException(String, Path, SdkException)}.
  * Because the {@link #shouldRetry(Exception, int, int, boolean)} method
  * Because the {@link #shouldRetry(Exception, int, int, boolean)} method
  * does this translation if an {@code SdkException} is processed,
  * does this translation if an {@code SdkException} is processed,
  * the policy defined for the IOEs also applies to the original exceptions.
  * the policy defined for the IOEs also applies to the original exceptions.
- *
+ * <p>
  * Put differently: this retry policy aims to work for handlers of the
  * Put differently: this retry policy aims to work for handlers of the
  * untranslated exceptions, as well as the translated ones.
  * untranslated exceptions, as well as the translated ones.
+ * <p>
+ * Note that because delete is considered idempotent, all s3a operations currently
+ * declare themselves idempotent.
+ * This means the retry policy here is more complex than it needs to be -but it
+ * does force us to consider when retrying operations would not be safe.
  * @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/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/AmazonS3/latest/dev/ErrorBestPractices.html"> Amazon S3 Error Best Practices</a>
  */
  */
-@SuppressWarnings("visibilitymodifier")  // I want a struct of finals, for real.
+@SuppressWarnings("visibilitymodifier")  // we want a struct of finals, for real.
 public class S3ARetryPolicy implements RetryPolicy {
 public class S3ARetryPolicy implements RetryPolicy {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -122,8 +130,7 @@ public class S3ARetryPolicy implements RetryPolicy {
    * @param conf configuration to read.
    * @param conf configuration to read.
    */
    */
   public S3ARetryPolicy(Configuration conf) {
   public S3ARetryPolicy(Configuration conf) {
-    Preconditions.checkArgument(conf != null, "Null configuration");
-    this.configuration = conf;
+    this.configuration = requireNonNull(conf, "Null configuration");
 
 
     // base policy from configuration
     // base policy from configuration
     int limit = conf.getInt(RETRY_LIMIT, RETRY_LIMIT_DEFAULT);
     int limit = conf.getInt(RETRY_LIMIT, RETRY_LIMIT_DEFAULT);
@@ -188,35 +195,57 @@ public class S3ARetryPolicy implements RetryPolicy {
     // inherit policies.
     // inherit policies.
     Map<Class<? extends Exception>, RetryPolicy> policyMap = new HashMap<>();
     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);
     // Access denial and auth exceptions are not retried
     // Access denial and auth exceptions are not retried
     policyMap.put(AccessDeniedException.class, fail);
     policyMap.put(AccessDeniedException.class, fail);
-    policyMap.put(NoAuthWithAWSException.class, fail);
-    policyMap.put(FileNotFoundException.class, fail);
-    policyMap.put(UnknownStoreException.class, fail);
-    policyMap.put(InvalidRequestException.class, fail);
 
 
-    // once the file has changed, trying again is not going to help
-    policyMap.put(RemoteFileChangedException.class, fail);
+    // Treated as an immediate failure
+    policyMap.put(AWSBadRequestException.class, fail);
 
 
-    // likely only recovered by changing the policy configuration or s3
-    // implementation
-    policyMap.put(NoVersionAttributeException.class, fail);
+    // use specific retry policy for aws client exceptions
+    // nested IOExceptions will already have been extracted and used
+    // in this map.
+    policyMap.put(AWSClientIOException.class, retryAwsClientExceptions);
+
+    // server didn't respond.
+    policyMap.put(AWSNoResponseException.class, retryIdempotentCalls);
 
 
     // should really be handled by resubmitting to new location;
     // should really be handled by resubmitting to new location;
     // that's beyond the scope of this retry policy
     // that's beyond the scope of this retry policy
     policyMap.put(AWSRedirectException.class, fail);
     policyMap.put(AWSRedirectException.class, fail);
 
 
+    // generic exception from the service
+    policyMap.put(AWSServiceIOException.class, retryAwsClientExceptions);
+
     // throttled requests are can be retried, always
     // throttled requests are can be retried, always
     policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
     policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
 
 
+    // Status 5xx error code is an immediate failure
+    // this is sign of a server-side problem, and while
+    // rare in AWS S3, it does happen on third party stores.
+    // (out of disk space, etc).
+    // by the time we get here, the aws sdk will have
+    // already retried.
+    // there is specific handling for some 5XX codes (501, 503);
+    // this is for everything else
+    policyMap.put(AWSStatus500Exception.class, fail);
+
+    // subclass of AWSServiceIOException whose cause is always S3Exception
+    policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
+
+    // server doesn't support a feature.
+    // raised from a number of HTTP error codes -mostly from
+    // third-party stores which only support a subset of AWS S3
+    // operations.
+    policyMap.put(AWSUnsupportedFeatureException.class, fail);
+
+    // socket exception subclass we consider unrecoverable
+    // though this is normally only found when opening a port for listening.
+    // which is never done in S3A.
+    policyMap.put(BindException.class, fail);
+
     // connectivity problems are retried without worrying about idempotency
     // connectivity problems are retried without worrying about idempotency
     policyMap.put(ConnectTimeoutException.class, connectivityFailure);
     policyMap.put(ConnectTimeoutException.class, connectivityFailure);
+    policyMap.put(ConnectException.class, connectivityFailure);
 
 
     // this can be a sign of an HTTP connection breaking early.
     // this can be a sign of an HTTP connection breaking early.
     // which can be reacted to by another attempt if the request was idempotent.
     // which can be reacted to by another attempt if the request was idempotent.
@@ -224,27 +253,38 @@ public class S3ARetryPolicy implements RetryPolicy {
     // which isn't going to be recovered from
     // which isn't going to be recovered from
     policyMap.put(EOFException.class, retryIdempotentCalls);
     policyMap.put(EOFException.class, retryIdempotentCalls);
 
 
-    // policy on a 400/bad request still ambiguous.
-    // Treated as an immediate failure
-    policyMap.put(AWSBadRequestException.class, fail);
+    // object not found. 404 when not unknown bucket; 410 "gone"
+    policyMap.put(FileNotFoundException.class, fail);
 
 
-    // Status 500 error code is also treated as a connectivity problem
-    policyMap.put(AWSStatus500Exception.class, connectivityFailure);
+    // Interrupted, usually by other threads
+    policyMap.put(InterruptedException.class, fail);
+    // note this does not pick up subclasses (like socket timeout)
+    policyMap.put(InterruptedIOException.class, fail);
+    policyMap.put(InvalidRequestException.class, fail);
 
 
-    // server didn't respond.
-    policyMap.put(AWSNoResponseException.class, retryIdempotentCalls);
+    // auth failure. Possibly recoverable by reattempting with
+    // the credential provider, but not covered here.
+    policyMap.put(NoAuthWithAWSException.class, fail);
 
 
-    // use specific retry policy for aws client exceptions
-    policyMap.put(AWSClientIOException.class, retryAwsClientExceptions);
-    policyMap.put(AWSServiceIOException.class, retryAwsClientExceptions);
+    // network routing.
+    policyMap.put(NoRouteToHostException.class, fail);
 
 
-    // other operations
-    policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
-    policyMap.put(SocketTimeoutException.class, retryIdempotentCalls);
+    // likely only recovered by changing the policy configuration or s3
+    // implementation
+    policyMap.put(NoVersionAttributeException.class, fail);
+    // once the file has changed, trying again is not going to help
+    policyMap.put(RemoteFileChangedException.class, fail);
+    // general socket exceptions
+    policyMap.put(SocketException.class, connectivityFailure);
+    policyMap.put(SocketTimeoutException.class, connectivityFailure);
 
 
+    // assume that DNS wil not recover; SDK is likely to have retried.
+    policyMap.put(UnknownHostException.class, fail);
+    policyMap.put(UnknownStoreException.class, fail);
     // Unsupported requests do not work, however many times you try
     // Unsupported requests do not work, however many times you try
     policyMap.put(UnsupportedRequestException.class, fail);
     policyMap.put(UnsupportedRequestException.class, fail);
 
 
+
     return policyMap;
     return policyMap;
   }
   }
 
 
@@ -253,14 +293,19 @@ public class S3ARetryPolicy implements RetryPolicy {
       int retries,
       int retries,
       int failovers,
       int failovers,
       boolean idempotent) throws Exception {
       boolean idempotent) throws Exception {
-    Preconditions.checkArgument(exception != null, "Null exception");
-    Exception ex = exception;
+    Exception ex = requireNonNull(exception, "Null exception");
     if (exception instanceof SdkException) {
     if (exception instanceof SdkException) {
       // update the sdk exception for the purpose of exception
       // update the sdk exception for the purpose of exception
       // processing.
       // processing.
       ex = S3AUtils.translateException("", "", (SdkException) exception);
       ex = S3AUtils.translateException("", "", (SdkException) exception);
     }
     }
-    return retryPolicy.shouldRetry(ex, retries, failovers, idempotent);
+    LOG.debug("Retry probe for {} with {} retries and {} failovers,"
+            + " idempotent={}, due to {}",
+        ex.getClass().getSimpleName(), retries, failovers, idempotent, ex, ex);
+    // look in the retry policy map
+    final RetryAction action = retryPolicy.shouldRetry(ex, retries, failovers, idempotent);
+    LOG.debug("Retry action is {}", action);
+    return action;
   }
   }
 
 
   /**
   /**

+ 29 - 10
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -80,6 +80,7 @@ import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isAbstract;
 import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf;
 import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf;
 import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unsupportedConstructor;
 import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unsupportedConstructor;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*;
+import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractNetworkException;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
 import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
 
 
@@ -189,10 +190,14 @@ public final class S3AUtils {
       ioe = maybeTranslateCredentialException(path, exception);
       ioe = maybeTranslateCredentialException(path, exception);
       if (ioe != null) {
       if (ioe != null) {
         return ioe;
         return ioe;
-      } else {
-        // no custom handling.
-        return new AWSClientIOException(message, exception);
       }
       }
+      // network problems covered by an IOE inside the exception chain.
+      ioe = maybeExtractNetworkException(path, exception);
+      if (ioe != null) {
+        return ioe;
+      }
+      // no custom handling.
+      return new AWSClientIOException(message, exception);
     } else {
     } else {
       // "error response returned by an S3 or other service."
       // "error response returned by an S3 or other service."
       // These contain more details and should be translated based
       // These contain more details and should be translated based
@@ -207,6 +212,8 @@ public final class S3AUtils {
       if (ase.awsErrorDetails() != null) {
       if (ase.awsErrorDetails() != null) {
         message = message + ":" + ase.awsErrorDetails().errorCode();
         message = message + ":" + ase.awsErrorDetails().errorCode();
       }
       }
+
+      // big switch on the HTTP status code.
       switch (status) {
       switch (status) {
 
 
       case SC_301_MOVED_PERMANENTLY:
       case SC_301_MOVED_PERMANENTLY:
@@ -240,7 +247,8 @@ public final class S3AUtils {
           // this is a missing bucket
           // this is a missing bucket
           ioe = new UnknownStoreException(path, message, ase);
           ioe = new UnknownStoreException(path, message, ase);
         } else {
         } else {
-          // a normal unknown object
+          // a normal unknown object.
+          // Can also be raised by third-party stores when aborting an unknown multipart upload
           ioe = new FileNotFoundException(message);
           ioe = new FileNotFoundException(message);
           ioe.initCause(ase);
           ioe.initCause(ase);
         }
         }
@@ -253,10 +261,13 @@ public final class S3AUtils {
         ioe.initCause(ase);
         ioe.initCause(ase);
         break;
         break;
 
 
-      // method not allowed; seen on S3 Select.
-      // treated as a bad request
+      // errors which stores can return from requests which
+      // the store does not support.
       case SC_405_METHOD_NOT_ALLOWED:
       case SC_405_METHOD_NOT_ALLOWED:
-        ioe = new AWSBadRequestException(message, s3Exception);
+      case SC_412_PRECONDITION_FAILED:
+      case SC_415_UNSUPPORTED_MEDIA_TYPE:
+      case SC_501_NOT_IMPLEMENTED:
+        ioe = new AWSUnsupportedFeatureException(message, s3Exception);
         break;
         break;
 
 
       // out of range. This may happen if an object is overwritten with
       // out of range. This may happen if an object is overwritten with
@@ -275,7 +286,8 @@ public final class S3AUtils {
         break;
         break;
 
 
       // throttling
       // throttling
-      case SC_503_SERVICE_UNAVAILABLE:
+      case SC_429_TOO_MANY_REQUESTS_GCS:    // google cloud through this connector
+      case SC_503_SERVICE_UNAVAILABLE:      // AWS
         ioe = new AWSServiceThrottledException(message, ase);
         ioe = new AWSServiceThrottledException(message, ase);
         break;
         break;
 
 
@@ -293,8 +305,15 @@ public final class S3AUtils {
         // other 200: FALL THROUGH
         // other 200: FALL THROUGH
 
 
       default:
       default:
-        // no specific exit code. Choose an IOE subclass based on the class
-        // of the caught exception
+        // no specifically handled exit code.
+
+        // convert all unknown 500+ errors to a 500 exception
+        if (status > SC_500_INTERNAL_SERVER_ERROR) {
+          ioe = new AWSStatus500Exception(message, ase);
+          break;
+        }
+
+        // Choose an IOE subclass based on the class of the caught exception
         ioe = s3Exception != null
         ioe = s3Exception != null
             ? new AWSS3IOException(message, s3Exception)
             ? new AWSS3IOException(message, s3Exception)
             : new AWSServiceIOException(message, ase);
             : new AWSServiceIOException(message, ase);

+ 4 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java

@@ -29,6 +29,8 @@ import software.amazon.awssdk.services.s3.model.S3Object;
 
 
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 
 
+import static java.util.Objects.requireNonNull;
+
 /**
 /**
  * API version-independent container for S3 List responses.
  * API version-independent container for S3 List responses.
  */
  */
@@ -47,7 +49,7 @@ public class S3ListResult {
    * @return new list result container
    * @return new list result container
    */
    */
   public static S3ListResult v1(ListObjectsResponse result) {
   public static S3ListResult v1(ListObjectsResponse result) {
-    return new S3ListResult(result, null);
+    return new S3ListResult(requireNonNull(result), null);
   }
   }
 
 
   /**
   /**
@@ -56,7 +58,7 @@ public class S3ListResult {
    * @return new list result container
    * @return new list result container
    */
    */
   public static S3ListResult v2(ListObjectsV2Response result) {
   public static S3ListResult v2(ListObjectsV2Response result) {
-    return new S3ListResult(null, result);
+    return new S3ListResult(null, requireNonNull(result));
   }
   }
 
 
   /**
   /**

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

@@ -91,7 +91,7 @@ public class SimpleAWSCredentialsProvider implements AwsCredentialsProvider {
   public String toString() {
   public String toString() {
     return "SimpleAWSCredentialsProvider{" +
     return "SimpleAWSCredentialsProvider{" +
         "accessKey.empty=" + accessKey.isEmpty() +
         "accessKey.empty=" + accessKey.isEmpty() +
-        ", secretKey.empty'" + secretKey.isEmpty() +
+        ", secretKey.empty=" + secretKey.isEmpty() +
         '}';
         '}';
   }
   }
 
 

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

@@ -582,7 +582,7 @@ public enum Statistic {
       TYPE_COUNTER),
       TYPE_COUNTER),
   MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED(
   MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED(
       StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED,
       StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED,
-      "Multipart Upload Abort Unner Path Invoked",
+      "Multipart Upload Abort Under Path Invoked",
       TYPE_COUNTER),
       TYPE_COUNTER),
   MULTIPART_UPLOAD_COMPLETED(
   MULTIPART_UPLOAD_COMPLETED(
       StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED,
       StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED,

+ 30 - 12
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java

@@ -31,7 +31,10 @@ import software.amazon.awssdk.core.signer.NoOpSigner;
 import software.amazon.awssdk.core.signer.Signer;
 import software.amazon.awssdk.core.signer.Signer;
 
 
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
 
 
+import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unavailable;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
 
 
 /**
 /**
  * Signer factory used to register and create signers.
  * Signer factory used to register and create signers.
@@ -44,6 +47,9 @@ public final class SignerFactory {
   public static final String NO_OP_SIGNER = "NoOpSignerType";
   public static final String NO_OP_SIGNER = "NoOpSignerType";
   private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
   private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
 
 
+  /** The v2 signer is no longer available: {@value}. */
+  public static final String S3_V2_SIGNER = "S3SignerType";
+
   private static final Map<String, Class<? extends Signer>> SIGNERS
   private static final Map<String, Class<? extends Signer>> SIGNERS
       = new ConcurrentHashMap<>();
       = new ConcurrentHashMap<>();
 
 
@@ -69,12 +75,8 @@ public final class SignerFactory {
       final String signerType,
       final String signerType,
       final Class<? extends Signer> signerClass) {
       final Class<? extends Signer> signerClass) {
 
 
-    if (signerType == null) {
-      throw new IllegalArgumentException("signerType cannot be null");
-    }
-    if (signerClass == null) {
-      throw new IllegalArgumentException("signerClass cannot be null");
-    }
+    checkArgument(signerType != null, "signerType cannot be null");
+    checkArgument(signerClass != null, "signerClass cannot be null");
 
 
     SIGNERS.put(signerType, signerClass);
     SIGNERS.put(signerType, signerClass);
   }
   }
@@ -82,14 +84,21 @@ public final class SignerFactory {
   /**
   /**
    * Check if the signer has already been registered.
    * Check if the signer has already been registered.
    * @param signerType signer to get
    * @param signerType signer to get
+   * @throws IllegalArgumentException if the signer type is unknown.
    */
    */
   public static void verifySignerRegistered(String signerType) {
   public static void verifySignerRegistered(String signerType) {
-    Class<? extends Signer> signerClass = SIGNERS.get(signerType);
-    if (signerClass == null) {
-      throw new IllegalArgumentException("unknown signer type: " + signerType);
-    }
+    checkArgument(isSignerRegistered(signerType),
+        "unknown signer type: %s", signerType);
   }
   }
 
 
+  /**
+   * Check if the signer has already been registered.
+   * @param signerType signer to get
+   * @return true if the signer is registered.
+   */
+  public static boolean isSignerRegistered(String signerType) {
+    return SIGNERS.containsKey(signerType);
+  }
 
 
   /**
   /**
    * Create an instance of the given signer.
    * Create an instance of the given signer.
@@ -97,13 +106,22 @@ public final class SignerFactory {
    * @param signerType The signer type.
    * @param signerType The signer type.
    * @param configKey Config key used to configure the signer.
    * @param configKey Config key used to configure the signer.
    * @return The new signer instance.
    * @return The new signer instance.
-   * @throws IOException on any problem.
+   * @throws InstantiationIOException instantiation problems.
+   * @throws IOException on any other problem.
+   *
    */
    */
   public static Signer createSigner(String signerType, String configKey) throws IOException {
   public static Signer createSigner(String signerType, String configKey) throws IOException {
+    if (S3_V2_SIGNER.equals(signerType)) {
+      throw unavailable(null, null, configKey, S3_V2_SIGNER + " is no longer supported");
+    }
+    if (!isSignerRegistered(signerType)) {
+      throw unavailable(null, null, configKey, "unknown signer type: " + signerType);
+    }
     Class<?> signerClass = SIGNERS.get(signerType);
     Class<?> signerClass = SIGNERS.get(signerType);
+
     String className = signerClass.getName();
     String className = signerClass.getName();
 
 
-    LOG.debug("Signer class is {}", className);
+    LOG.debug("Signer class from {} and key {} is {}", signerType, configKey, className);
 
 
     Signer signer =
     Signer signer =
         S3AUtils.getInstanceFromReflection(className, null, null, Signer.class, "create",
         S3AUtils.getInstanceFromReflection(className, null, null, Signer.class, "create",

+ 9 - 6
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java

@@ -109,17 +109,20 @@ public class SignerManager implements Closeable {
     }
     }
   }
   }
 
 
-  /*
-   * Make sure the signer class is registered once with the AWS SDK
+  /**
+   * Make sure the signer class is registered once with the AWS SDK.
+   * @param signerName signer name
+   * @param signerClassName classname
+   * @param conf source configuration
+   * @throws RuntimeException if the class is not found
    */
    */
   private static void maybeRegisterSigner(String signerName,
   private static void maybeRegisterSigner(String signerName,
       String signerClassName, Configuration conf) {
       String signerClassName, Configuration conf) {
-    try {
-      SignerFactory.verifySignerRegistered(signerName);
-    } catch (IllegalArgumentException e) {
+
+    if (!SignerFactory.isSignerRegistered(signerName)) {
       // Signer is not registered with the AWS SDK.
       // Signer is not registered with the AWS SDK.
       // Load the class and register the signer.
       // Load the class and register the signer.
-      Class<? extends Signer> clazz = null;
+      Class<? extends Signer> clazz;
       try {
       try {
         clazz = (Class<? extends Signer>) conf.getClassByName(signerClassName);
         clazz = (Class<? extends Signer>) conf.getClassByName(signerClassName);
       } catch (ClassNotFoundException cnfe) {
       } catch (ClassNotFoundException cnfe) {

+ 7 - 7
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSClientConfig.java

@@ -117,14 +117,14 @@ public final class AWSClientConfig {
         S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
         S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
     int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
     int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
 
 
-    httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
-    httpClientBuilder.socketTimeout(Duration.ofSeconds(socketTimeout));
+    httpClientBuilder.connectionTimeout(Duration.ofMillis(connectionEstablishTimeout));
+    httpClientBuilder.socketTimeout(Duration.ofMillis(socketTimeout));
 
 
     // set the connection TTL irrespective of whether the connection is in use or not.
     // set the connection TTL irrespective of whether the connection is in use or not.
     // this can balance requests over different S3 servers, and avoid failed
     // this can balance requests over different S3 servers, and avoid failed
     // connections. See HADOOP-18845.
     // connections. See HADOOP-18845.
     long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1);
     long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1);
-    httpClientBuilder.connectionTimeToLive(Duration.ofSeconds(ttl));
+    httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl));
 
 
     NetworkBinding.bindSSLChannelMode(conf, httpClientBuilder);
     NetworkBinding.bindSSLChannelMode(conf, httpClientBuilder);
 
 
@@ -148,15 +148,15 @@ public final class AWSClientConfig {
         S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
         S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
     int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
     int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
 
 
-    httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
-    httpClientBuilder.readTimeout(Duration.ofSeconds(socketTimeout));
-    httpClientBuilder.writeTimeout(Duration.ofSeconds(socketTimeout));
+    httpClientBuilder.connectionTimeout(Duration.ofMillis(connectionEstablishTimeout));
+    httpClientBuilder.readTimeout(Duration.ofMillis(socketTimeout));
+    httpClientBuilder.writeTimeout(Duration.ofMillis(socketTimeout));
 
 
     // set the connection TTL irrespective of whether the connection is in use or not.
     // set the connection TTL irrespective of whether the connection is in use or not.
     // this can balance requests over different S3 servers, and avoid failed
     // this can balance requests over different S3 servers, and avoid failed
     // connections. See HADOOP-18845.
     // connections. See HADOOP-18845.
     long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1);
     long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1);
-    httpClientBuilder.connectionTimeToLive(Duration.ofSeconds(ttl));
+    httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl));
 
 
     // TODO: Don't think you can set a socket factory for the netty client.
     // TODO: Don't think you can set a socket factory for the netty client.
     //  NetworkBinding.bindSSLChannelMode(conf, awsConf);
     //  NetworkBinding.bindSSLChannelMode(conf, awsConf);

+ 69 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java

@@ -18,8 +18,13 @@
 
 
 package org.apache.hadoop.fs.s3a.impl;
 package org.apache.hadoop.fs.s3a.impl;
 
 
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 
 
+import org.apache.hadoop.fs.PathIOException;
+
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
 
 
 /**
 /**
@@ -35,7 +40,7 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
  * The existing code las been left in S3AUtils it is to avoid cherry-picking
  * The existing code las been left in S3AUtils it is to avoid cherry-picking
  * problems on backports.
  * problems on backports.
  */
  */
-public class ErrorTranslation {
+public final class ErrorTranslation {
 
 
   /**
   /**
    * Private constructor for utility class.
    * Private constructor for utility class.
@@ -66,6 +71,69 @@ public class ErrorTranslation {
     return e.statusCode() == SC_404_NOT_FOUND && !isUnknownBucket(e);
     return e.statusCode() == SC_404_NOT_FOUND && !isUnknownBucket(e);
   }
   }
 
 
+  /**
+   * Translate an exception if it or its inner exception is an
+   * IOException.
+   * If this condition is not met, null is returned.
+   * @param path path of operation.
+   * @param thrown exception
+   * @return a translated exception or null.
+   */
+  public static IOException maybeExtractNetworkException(String path, Throwable thrown) {
+
+    if (thrown == null) {
+      return null;
+    }
+
+    // look inside
+    Throwable cause = thrown.getCause();
+    while (cause != null && cause.getCause() != null) {
+      cause = cause.getCause();
+    }
+    if (!(cause instanceof IOException)) {
+      return null;
+    }
+
+    // the cause can be extracted to an IOE.
+    // rather than just return it, we try to preserve the stack trace
+    // of the outer exception.
+    // as a new instance is created through reflection, the
+    // class of the returned instance will be that of the innermost,
+    // unless no suitable constructor is available.
+    return wrapWithInnerIOE(path, thrown, (IOException) cause);
+
+  }
+
+  /**
+   * Given an outer and an inner exception, create a new IOE
+   * of the inner type, with the outer exception as the cause.
+   * The message is derived from both.
+   * This only works if the inner exception has a constructor which
+   * takes a string; if not a PathIOException is created.
+   * <p>
+   * See {@code NetUtils}.
+   * @param <T> type of inner exception.
+   * @param path path of the failure.
+   * @param outer outermost exception.
+   * @param inner inner exception.
+   * @return the new exception.
+   */
+  @SuppressWarnings("unchecked")
+  private static <T extends IOException> IOException wrapWithInnerIOE(
+      String path,
+      Throwable outer,
+      T inner) {
+    String msg = outer.toString() + ": " + inner.getMessage();
+    Class<? extends Throwable> clazz = inner.getClass();
+    try {
+      Constructor<? extends Throwable> ctor = clazz.getConstructor(String.class);
+      Throwable t = ctor.newInstance(msg);
+      return (T) (t.initCause(outer));
+    } catch (Throwable e) {
+      return new PathIOException(path, msg, outer);
+    }
+  }
+
   /**
   /**
    * AWS error codes explicitly recognized and processes specially;
    * AWS error codes explicitly recognized and processes specially;
    * kept in their own class for isolation.
    * kept in their own class for isolation.

+ 7 - 5
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InstantiationIOException.java

@@ -73,12 +73,14 @@ public class InstantiationIOException extends PathIOException {
 
 
   public InstantiationIOException(
   public InstantiationIOException(
       Kind kind,
       Kind kind,
-      @Nullable URI uri, String classname,
+      @Nullable URI uri,
+      @Nullable String classname,
       @Nullable String key,
       @Nullable String key,
       String message,
       String message,
-      Throwable cause) {
+      @Nullable Throwable cause) {
     super(uri!= null ? uri.toString() : "",
     super(uri!= null ? uri.toString() : "",
-        "Class " + classname + " " + message
+        (classname != null ? ("Class " + classname + " ") : "")
+            + message
             + (key != null ? (" (configuration key " + key + ")") : ""),
             + (key != null ? (" (configuration key " + key + ")") : ""),
         cause);
         cause);
     this.kind = kind;
     this.kind = kind;
@@ -137,8 +139,8 @@ public class InstantiationIOException extends PathIOException {
    */
    */
   public static InstantiationIOException unavailable(
   public static InstantiationIOException unavailable(
       @Nullable URI uri,
       @Nullable URI uri,
-      String classname,
-      String key,
+      @Nullable String classname,
+      @Nullable String key,
       String text) {
       String text) {
     return new InstantiationIOException(Kind.Unavailable,
     return new InstantiationIOException(Kind.Unavailable,
         uri, classname, key, text, null);
         uri, classname, key, text, null);

+ 10 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java

@@ -148,9 +148,15 @@ public final class InternalConstants {
   /** 412 status code: Precondition Failed. */
   /** 412 status code: Precondition Failed. */
   public static final int SC_412_PRECONDITION_FAILED = 412;
   public static final int SC_412_PRECONDITION_FAILED = 412;
 
 
+  /** 415 status code: Content type unsupported by this store. */
+  public static final int SC_415_UNSUPPORTED_MEDIA_TYPE = 415;
+
   /** 416 status code: Range Not Satisfiable. */
   /** 416 status code: Range Not Satisfiable. */
   public static final int SC_416_RANGE_NOT_SATISFIABLE = 416;
   public static final int SC_416_RANGE_NOT_SATISFIABLE = 416;
 
 
+  /** 429 status code: This is the google GCS throttle message. */
+  public static final int SC_429_TOO_MANY_REQUESTS_GCS = 429;
+
   /** 443 status code: No Response (unofficial). */
   /** 443 status code: No Response (unofficial). */
   public static final int SC_443_NO_RESPONSE = 443;
   public static final int SC_443_NO_RESPONSE = 443;
 
 
@@ -160,7 +166,10 @@ public final class InternalConstants {
   /** 500 status code: Internal Server Error. */
   /** 500 status code: Internal Server Error. */
   public static final int SC_500_INTERNAL_SERVER_ERROR = 500;
   public static final int SC_500_INTERNAL_SERVER_ERROR = 500;
 
 
-  /** 503 status code: Service Unavailable. */
+  /** 501 status code: method not implemented. */
+  public static final int SC_501_NOT_IMPLEMENTED = 501;
+
+  /** 503 status code: Service Unavailable. on AWS S3: throttle response. */
   public static final int SC_503_SERVICE_UNAVAILABLE = 503;
   public static final int SC_503_SERVICE_UNAVAILABLE = 503;
 
 
   /** Name of the log for throttling events. Value: {@value}. */
   /** Name of the log for throttling events. Value: {@value}. */

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

@@ -63,6 +63,7 @@ import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static software.amazon.awssdk.services.s3.model.StorageClass.UNKNOWN_TO_SDK_VERSION;
 
 
 /**
 /**
  * The standard implementation of the request factory.
  * The standard implementation of the request factory.
@@ -242,7 +243,7 @@ public class RequestFactoryImpl implements RequestFactory {
         .metadataDirective(MetadataDirective.REPLACE)
         .metadataDirective(MetadataDirective.REPLACE)
         .acl(cannedACL);
         .acl(cannedACL);
 
 
-    if (srcom.storageClass() != null) {
+    if (srcom.storageClass() != null && srcom.storageClass() != UNKNOWN_TO_SDK_VERSION) {
       copyObjectRequestBuilder.storageClass(srcom.storageClass());
       copyObjectRequestBuilder.storageClass(srcom.storageClass());
     }
     }
 
 

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

@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
-import java.nio.file.AccessDeniedException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Date;
 import java.util.Date;
@@ -400,9 +399,10 @@ public abstract class S3GuardTool extends Configured implements Tool,
       println(out, "Filesystem %s", fsUri);
       println(out, "Filesystem %s", fsUri);
       try {
       try {
         println(out, "Location: %s", fs.getBucketLocation());
         println(out, "Location: %s", fs.getBucketLocation());
-      } catch (AccessDeniedException e) {
+      } catch (IOException e) {
         // Caller cannot get the location of this bucket due to permissions
         // Caller cannot get the location of this bucket due to permissions
-        // in their role or the bucket itself.
+        // in their role or the bucket itself, or it is not an operation
+        // supported by this store.
         // Note and continue.
         // Note and continue.
         LOG.debug("failed to get bucket location", e);
         LOG.debug("failed to get bucket location", e);
         println(out, LOCATION_UNKNOWN);
         println(out, LOCATION_UNKNOWN);

+ 25 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java

@@ -35,6 +35,7 @@ import java.util.stream.Collectors;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
 import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
 import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.s3a.AWSBadRequestException;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -679,8 +680,30 @@ public final class MarkerTool extends S3GuardTool {
 
 
     int count = 0;
     int count = 0;
     boolean result = true;
     boolean result = true;
-    RemoteIterator<S3AFileStatus> listing = operations
-        .listObjects(path, storeContext.pathToKey(path));
+
+    // the path/key stuff loses any trailing / passed in.
+    // but this may actually be needed.
+    RemoteIterator<S3AFileStatus> listing = null;
+    String listkey = storeContext.pathToKey(path);
+    if (listkey.isEmpty()) {
+      // root. always give it a path to keep ranger happy.
+      listkey = "/";
+    }
+
+    try {
+      listing = operations.listObjects(path, listkey);
+    } catch (AWSBadRequestException e) {
+      // endpoint was unhappy. this is generally unrecoverable, but some
+      // third party stores do insist on a / here.
+      LOG.debug("Failed to list \"{}\"", listkey, e);
+      // now retry with a trailing / in case that works
+      if (listkey.endsWith("/")) {
+        // already has a trailing /, so fail
+        throw e;
+      }
+      // try again.
+      listing = operations.listObjects(path, listkey + "/");
+    }
     while (listing.hasNext()) {
     while (listing.hasNext()) {
       count++;
       count++;
       S3AFileStatus status = listing.next();
       S3AFileStatus status = listing.next();

+ 26 - 169
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md

@@ -371,16 +371,7 @@ The Assumed Role Credential Provider is enabled, but `fs.s3a.assumed.role.arn` i
 ```
 ```
 java.io.IOException: Unset property fs.s3a.assumed.role.arn
 java.io.IOException: Unset property fs.s3a.assumed.role.arn
   at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:76)
   at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:76)
-  at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
-  at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
-  at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
-  at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:583)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
+
 ```
 ```
 
 
 ### <a name="not_authorized_for_assumed_role"></a> "Not authorized to perform sts:AssumeRole"
 ### <a name="not_authorized_for_assumed_role"></a> "Not authorized to perform sts:AssumeRole"
@@ -390,17 +381,9 @@ or one to which the caller has no access.
 
 
 ```
 ```
 java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
 java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
- on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
   Not authorized to perform sts:AssumeRole (Service: AWSSecurityTokenService; Status Code: 403;
   Not authorized to perform sts:AssumeRole (Service: AWSSecurityTokenService; Status Code: 403;
    Error Code: AccessDenied; Request ID: aad4e59a-f4b0-11e7-8c78-f36aaa9457f6):AccessDenied
    Error Code: AccessDenied; Request ID: aad4e59a-f4b0-11e7-8c78-f36aaa9457f6):AccessDenied
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
+
 ```
 ```
 
 
 ### <a name="root_account"></a> "Roles may not be assumed by root accounts"
 ### <a name="root_account"></a> "Roles may not be assumed by root accounts"
@@ -411,31 +394,15 @@ the role.
 
 
 ```
 ```
 java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
 java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
- on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+
     Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied;
     Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied;
     Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2):
     Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2):
     No AWS Credentials provided by AssumedRoleCredentialProvider :
     No AWS Credentials provided by AssumedRoleCredentialProvider :
-    com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+    software.amazon.awssdk.services.sts.model.StsException:
     Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService;
     Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService;
      Status Code: 403; Error Code: AccessDenied; Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2)
      Status Code: 403; Error Code: AccessDenied; Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2)
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
-  ... 22 more
-Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
- Roles may not be assumed by root accounts.
-  (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied;
-   Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
+
+
 ```
 ```
 
 
 ### <a name="invalid_duration"></a> `Member must have value greater than or equal to 900`
 ### <a name="invalid_duration"></a> `Member must have value greater than or equal to 900`
@@ -444,7 +411,7 @@ The value of `fs.s3a.assumed.role.session.duration` is too low.
 
 
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
-com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+software.amazon.awssdk.services.sts.model.StsException:
 1 validation error detected: Value '20' at 'durationSeconds' failed to satisfy constraint:
 1 validation error detected: Value '20' at 'durationSeconds' failed to satisfy constraint:
 Member must have value greater than or equal to 900 (Service: AWSSecurityTokenService;
 Member must have value greater than or equal to 900 (Service: AWSSecurityTokenService;
 Status Code: 400; Error Code: ValidationError;
 Status Code: 400; Error Code: ValidationError;
@@ -459,7 +426,7 @@ The value of `fs.s3a.assumed.role.session.duration` is too high.
 
 
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
- com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+ software.amazon.awssdk.services.sts.model.StsException:
 The requested DurationSeconds exceeds the MaxSessionDuration set for this role.
 The requested DurationSeconds exceeds the MaxSessionDuration set for this role.
 (Service: AWSSecurityTokenService; Status Code: 400;
 (Service: AWSSecurityTokenService; Status Code: 400;
  Error Code: ValidationError; Request ID: 17875165-d0a7-11e8-b85f-d15a599a7f6d)
  Error Code: ValidationError; Request ID: 17875165-d0a7-11e8-b85f-d15a599a7f6d)
@@ -478,7 +445,7 @@ any role for up to  12h; attempting to use a larger number will fail.
 
 
 
 
 ```
 ```
-Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+Caused by: software.amazon.awssdk.services.sts.model.StsException:
 1 validation error detected:
 1 validation error detected:
 Value '345600' at 'durationSeconds' failed to satisfy constraint:
 Value '345600' at 'durationSeconds' failed to satisfy constraint:
 Member must have value less than or equal to 43200
 Member must have value less than or equal to 43200
@@ -492,7 +459,7 @@ For full sessions, the duration limit is 129600 seconds: 36h.
 
 
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request session credentials:
 org.apache.hadoop.fs.s3a.AWSBadRequestException: request session credentials:
-com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+software.amazon.awssdk.services.sts.model.StsException:
 1 validation error detected: Value '345600' at 'durationSeconds' failed to satisfy constraint:
 1 validation error detected: Value '345600' at 'durationSeconds' failed to satisfy constraint:
 Member must have value less than or equal to 129600
 Member must have value less than or equal to 129600
 (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
 (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
@@ -510,26 +477,12 @@ AWS specification of Role Policies.
 
 
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
 org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
- com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException:
+ software.amazon.awssdk.services.sts.model.MalformedPolicyDocumentException:
   The policy is not in the valid JSON format. (Service: AWSSecurityTokenService; Status Code: 400;
   The policy is not in the valid JSON format. (Service: AWSSecurityTokenService; Status Code: 400;
    Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c):
    Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c):
    MalformedPolicyDocument: The policy is not in the valid JSON format.
    MalformedPolicyDocument: The policy is not in the valid JSON format.
    (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
    (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
     Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c)
     Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c)
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
-Caused by: com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException:
- The policy is not in the valid JSON format.
-  (Service: AWSSecurityTokenService; Status Code: 400;
-   Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
 ```
 ```
 
 
 ### <a name="policy_syntax_error"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
 ### <a name="policy_syntax_error"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
@@ -539,27 +492,13 @@ The policy set in `fs.s3a.assumed.role.policy` is not valid JSON.
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException:
 org.apache.hadoop.fs.s3a.AWSBadRequestException:
 Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
 Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
- com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException:
+ software.amazon.awssdk.services.sts.model.MalformedPolicyDocumentException:
   Syntax errors in policy. (Service: AWSSecurityTokenService;
   Syntax errors in policy. (Service: AWSSecurityTokenService;
   Status Code: 400; Error Code: MalformedPolicyDocument;
   Status Code: 400; Error Code: MalformedPolicyDocument;
   Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45):
   Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45):
   MalformedPolicyDocument: Syntax errors in policy.
   MalformedPolicyDocument: Syntax errors in policy.
   (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
   (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
   Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45)
   Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45)
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
- (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
-  Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
-  ... 19 more
 ```
 ```
 
 
 ### <a name="recursive_auth"></a> `IOException`: "AssumedRoleCredentialProvider cannot be in fs.s3a.assumed.role.credentials.provider"
 ### <a name="recursive_auth"></a> `IOException`: "AssumedRoleCredentialProvider cannot be in fs.s3a.assumed.role.credentials.provider"
@@ -591,7 +530,7 @@ inner authentication which is breaking signature creation.
 
 
 ```
 ```
  org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
  org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider
-  on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+  on : software.amazon.awssdk.services.sts.model.StsException:
    'valid/20180109/us-east-1/sts/aws4_request' not a valid key=value pair (missing equal-sign) in Authorization header:
    'valid/20180109/us-east-1/sts/aws4_request' not a valid key=value pair (missing equal-sign) in Authorization header:
     'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request,
     'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request,
     SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date.
     SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date.
@@ -601,21 +540,7 @@ inner authentication which is breaking signature creation.
     in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request,
     in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request,
     SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date,
     SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date,
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: IncompleteSignature;
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: IncompleteSignature;
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
 
 
-Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
-    'valid/20180109/us-east-1/sts/aws4_request' not a valid key=value pair (missing equal-sign)
-    in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request,
-    SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date,
-    (Service: AWSSecurityTokenService; Status Code: 400; Error Code: IncompleteSignature;
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
 ```
 ```
 
 
 ### <a name="invalid_token"></a> `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid"
 ### <a name="invalid_token"></a> `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid"
@@ -625,27 +550,11 @@ The credentials used to authenticate with the AWS Security Token Service are inv
 ```
 ```
 [ERROR] Failures:
 [ERROR] Failures:
 [ERROR] java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
 [ERROR] java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
- com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+ software.amazon.awssdk.services.sts.model.StsException:
   The security token included in the request is invalid.
   The security token included in the request is invalid.
   (Service: AWSSecurityTokenService; Status Code: 403; Error Code: InvalidClientTokenId;
   (Service: AWSSecurityTokenService; Status Code: 403; Error Code: InvalidClientTokenId;
    Request ID: 74aa7f8a-f557-11e7-850c-33d05b3658d7):InvalidClientTokenId
    Request ID: 74aa7f8a-f557-11e7-850c-33d05b3658d7):InvalidClientTokenId
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-
-Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
-The security token included in the request is invalid.
- (Service: AWSSecurityTokenService; Status Code: 403; Error Code: InvalidClientTokenId;
- Request ID: 74aa7f8a-f557-11e7-850c-33d05b3658d7)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
-  ... 25 more
-```
+ ```
 
 
 ### <a name="invalid_session"></a> `AWSSecurityTokenServiceExceptiond`: "Member must satisfy regular expression pattern: `[\w+=,.@-]*`"
 ### <a name="invalid_session"></a> `AWSSecurityTokenServiceExceptiond`: "Member must satisfy regular expression pattern: `[\w+=,.@-]*`"
 
 
@@ -659,7 +568,7 @@ If set explicitly, it must be valid.
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSBadRequestException:
 org.apache.hadoop.fs.s3a.AWSBadRequestException:
  Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
  Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
-    com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+    software.amazon.awssdk.services.sts.model.StsException:
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
     failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
@@ -667,23 +576,7 @@ org.apache.hadoop.fs.s3a.AWSBadRequestException:
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
     failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
     (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616)
-  at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520)
-  at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252)
-  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
-  at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
-  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
-
-Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
-    1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
-    failed to satisfy constraint:
-    Member must satisfy regular expression pattern: [\w+=,.@-]*
-    (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
-```
+  ```
 
 
 
 
 ### <a name="access_denied"></a> `java.nio.file.AccessDeniedException` within a FileSystem API call
 ### <a name="access_denied"></a> `java.nio.file.AccessDeniedException` within a FileSystem API call
@@ -692,23 +585,11 @@ If an operation fails with an `AccessDeniedException`, then the role does not ha
 the permission for the S3 Operation invoked during the call.
 the permission for the S3 Operation invoked during the call.
 
 
 ```
 ```
-java.nio.file.AccessDeniedException: s3a://bucket/readonlyDir:
-  rename(s3a://bucket/readonlyDir, s3a://bucket/renameDest)
- on s3a://bucket/readonlyDir:
-  com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
-  (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 2805F2ABF5246BB1;
-   S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=),
-   S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=:AccessDenied
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:216)
-  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:143)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:853)
- ...
-Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
- (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 2805F2ABF5246BB1;
-  S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=),
-  S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
-  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
+> hadoop fs -touch  s3a://landsat-pds/a
+
+java.nio.file.AccessDeniedException: a: Writing Object on a:
+ software.amazon.awssdk.services.s3.model.S3Exception: Access Denied
+ (Service: S3, Status Code: 403, Request ID: F08EV50F85AYKF1V, Extended Request ID: 75vMz9xWNP5/lYplPSZfm/i4yQ5q0G32SIwOwfaj6a8gNCRj9tLBAqcLaaexT/aNg2DhWZQPvDU=):AccessDenied
 ```
 ```
 
 
 This is the policy restriction behaving as intended: the caller is trying to
 This is the policy restriction behaving as intended: the caller is trying to
@@ -743,28 +624,9 @@ If the client does have write access to the bucket, verify that the caller has
 
 
 ```
 ```
 java.nio.file.AccessDeniedException: test/testDTFileSystemClient: Writing Object on test/testDTFileSystemClient:
 java.nio.file.AccessDeniedException: test/testDTFileSystemClient: Writing Object on test/testDTFileSystemClient:
-  com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; 
+  software.amazon.awssdk.services.s3.model.S3Exception: Access Denied (Service: Amazon S3; Status Code: 403;
   Error Code: AccessDenied; Request ID: E86544FF1D029857)
   Error Code: AccessDenied; Request ID: E86544FF1D029857)
 
 
-    at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:243)
-    at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
-    at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:314)
-    at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:406)
-    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:310)
-    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:285)
-    at org.apache.hadoop.fs.s3a.WriteOperationHelper.retry(WriteOperationHelper.java:150)
-    at org.apache.hadoop.fs.s3a.WriteOperationHelper.putObject(WriteOperationHelper.java:460)
-    at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.lambda$putObject$0(S3ABlockOutputStream.java:438)
-    at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219)
-    at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219)
-    at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
-    at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:57)
-    at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
-    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
-    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
-    at java.lang.Thread.run(Thread.java:748)
-Caused by:  com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403;
-            Error Code: AccessDenied; Request ID: E86544FF1D029857)
 ```
 ```
 
 
 Note: the ability to read encrypted data in the store does not guarantee that the caller can encrypt new data.
 Note: the ability to read encrypted data in the store does not guarantee that the caller can encrypt new data.
@@ -779,14 +641,9 @@ This is a low-level networking error. Possible causes include:
 
 
 ```
 ```
 org.apache.hadoop.fs.s3a.AWSClientIOException: request session credentials:
 org.apache.hadoop.fs.s3a.AWSClientIOException: request session credentials:
-  com.amazonaws.SdkClientException:
 
 
   Unable to execute HTTP request: null: Unable to execute HTTP request: null
   Unable to execute HTTP request: null: Unable to execute HTTP request: null
-at com.amazonaws.thirdparty.apache.http.impl.conn.DefaultRoutePlanner.determineRoute(DefaultRoutePlanner.java:88)
-at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.determineRoute(InternalHttpClient.java:124)
-at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:183)
-at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
-at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+
 ```
 ```
 
 
 ###  <a name="credential_scope"></a> Error "Credential should be scoped to a valid region"
 ###  <a name="credential_scope"></a> Error "Credential should be scoped to a valid region"
@@ -800,7 +657,7 @@ Variant 1: `Credential should be scoped to a valid region, not 'us-west-1'` (or
 
 
 ```
 ```
 java.nio.file.AccessDeniedException: : request session credentials:
 java.nio.file.AccessDeniedException: : request session credentials:
-com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+software.amazon.awssdk.services.sts.model.StsException:
 Credential should be scoped to a valid region, not 'us-west-1'.
 Credential should be scoped to a valid region, not 'us-west-1'.
 (Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch; Request ID: d9065cc4-e2b9-11e8-8b7b-f35cb8d7aea4):SignatureDoesNotMatch
 (Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch; Request ID: d9065cc4-e2b9-11e8-8b7b-f35cb8d7aea4):SignatureDoesNotMatch
 ```
 ```
@@ -817,7 +674,7 @@ Variant 2: `Credential should be scoped to a valid region, not ''`
 
 
 ```
 ```
 java.nio.file.AccessDeniedException: : request session credentials:
 java.nio.file.AccessDeniedException: : request session credentials:
-com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+software.amazon.awssdk.services.sts.model.StsException:
   Credential should be scoped to a valid region, not ''. (
   Credential should be scoped to a valid region, not ''. (
   Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch;
   Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch;
   Request ID: bd3e5121-e2ac-11e8-a566-c1a4d66b6a16):SignatureDoesNotMatch
   Request ID: bd3e5121-e2ac-11e8-a566-c1a4d66b6a16):SignatureDoesNotMatch

+ 470 - 159
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -35,19 +35,20 @@ full details.
 
 
 * [Encryption](./encryption.html)
 * [Encryption](./encryption.html)
 * [Performance](./performance.html)
 * [Performance](./performance.html)
-* [S3Guard](./s3guard.html)
+* [The upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html)
+* [Working with Third-party S3 Stores](./third_party_stores.html)
 * [Troubleshooting](./troubleshooting_s3a.html)
 * [Troubleshooting](./troubleshooting_s3a.html)
+* [Prefetching](./prefetching.html)
 * [Controlling the S3A Directory Marker Behavior](directory_markers.html).
 * [Controlling the S3A Directory Marker Behavior](directory_markers.html).
+* [Auditing](./auditing.html).
 * [Committing work to S3 with the "S3A Committers"](./committers.html)
 * [Committing work to S3 with the "S3A Committers"](./committers.html)
 * [S3A Committers Architecture](./committer_architecture.html)
 * [S3A Committers Architecture](./committer_architecture.html)
 * [Working with IAM Assumed Roles](./assumed_roles.html)
 * [Working with IAM Assumed Roles](./assumed_roles.html)
 * [S3A Delegation Token Support](./delegation_tokens.html)
 * [S3A Delegation Token Support](./delegation_tokens.html)
 * [S3A Delegation Token Architecture](delegation_token_architecture.html).
 * [S3A Delegation Token Architecture](delegation_token_architecture.html).
-* [Auditing](./auditing.html).
 * [Auditing Architecture](./auditing_architecture.html).
 * [Auditing Architecture](./auditing_architecture.html).
 * [Testing](./testing.html)
 * [Testing](./testing.html)
-* [Prefetching](./prefetching.html)
-* [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html)
+* [S3Guard](./s3guard.html)
 
 
 ## <a name="overview"></a> Overview
 ## <a name="overview"></a> Overview
 
 
@@ -79,16 +80,15 @@ and compatible implementations.
 * Supports partitioned uploads for many-GB objects.
 * Supports partitioned uploads for many-GB objects.
 * Offers a high-performance random IO mode for working with columnar data such
 * Offers a high-performance random IO mode for working with columnar data such
 as Apache ORC and Apache Parquet files.
 as Apache ORC and Apache Parquet files.
-* Uses Amazon's Java S3 SDK with support for latest S3 features and authentication
+* Uses Amazon's Java V2 SDK with support for latest S3 features and authentication
 schemes.
 schemes.
 * Supports authentication via: environment variables, Hadoop configuration
 * Supports authentication via: environment variables, Hadoop configuration
 properties, the Hadoop key management store and IAM roles.
 properties, the Hadoop key management store and IAM roles.
 * Supports per-bucket configuration.
 * Supports per-bucket configuration.
 * Supports S3 "Server Side Encryption" for both reading and writing:
 * Supports S3 "Server Side Encryption" for both reading and writing:
  SSE-S3, SSE-KMS and SSE-C.
  SSE-S3, SSE-KMS and SSE-C.
+* Supports S3-CSE client side encryption.
 * Instrumented with Hadoop metrics.
 * Instrumented with Hadoop metrics.
-* Before S3 was consistent, provided a consistent view of inconsistent storage
-  through [S3Guard](./s3guard.html).
 * Actively maintained by the open source community.
 * Actively maintained by the open source community.
 
 
 
 
@@ -97,19 +97,17 @@ properties, the Hadoop key management store and IAM roles.
 There other Hadoop connectors to S3. Only S3A is actively maintained by
 There other Hadoop connectors to S3. Only S3A is actively maintained by
 the Hadoop project itself.
 the Hadoop project itself.
 
 
-1. Apache's Hadoop's original `s3://` client. This is no longer included in Hadoop.
 1. Amazon EMR's `s3://` client. This is from the Amazon EMR team, who actively
 1. Amazon EMR's `s3://` client. This is from the Amazon EMR team, who actively
 maintain it.
 maintain it.
-1. Apache's Hadoop's [`s3n:` filesystem client](./s3n.html).
-   This connector is no longer available: users must migrate to the newer `s3a:` client.
+
 
 
 
 
 ## <a name="getting_started"></a> Getting Started
 ## <a name="getting_started"></a> Getting Started
 
 
 S3A depends upon two JARs, alongside `hadoop-common` and its dependencies.
 S3A depends upon two JARs, alongside `hadoop-common` and its dependencies.
 
 
-* `hadoop-aws` JAR.
-* `aws-java-sdk-bundle` JAR.
+* `hadoop-aws` JAR. This contains the S3A connector.
+* `bundle` JAR. This contains the full shaded AWS V2 SDK.
 
 
 The versions of `hadoop-common` and `hadoop-aws` must be identical.
 The versions of `hadoop-common` and `hadoop-aws` must be identical.
 
 
@@ -189,7 +187,8 @@ of work, as opposed to HDFS or other "real" filesystem.
 
 
 The [S3A committers](./committers.html) are the sole mechanism available
 The [S3A committers](./committers.html) are the sole mechanism available
 to safely save the output of queries directly into S3 object stores
 to safely save the output of queries directly into S3 object stores
-through the S3A filesystem.
+through the S3A filesystem when the filesystem structure is
+how the table is represented.
 
 
 
 
 ### Warning #2: Object stores have different authorization models
 ### Warning #2: Object stores have different authorization models
@@ -199,10 +198,7 @@ authorization model of HDFS and traditional file systems.
 The S3A client simply reports stub information from APIs that would query this metadata:
 The S3A client simply reports stub information from APIs that would query this metadata:
 
 
 * File owner is reported as the current user.
 * File owner is reported as the current user.
-* File group also is reported as the current user. Prior to Apache Hadoop
-2.8.0, file group was reported as empty (no group associated), which is a
-potential incompatibility problem for scripts that perform positional parsing of
-shell output and other clients that expect to find a well-defined group.
+* File group also is reported as the current user.
 * Directory permissions are reported as 777.
 * Directory permissions are reported as 777.
 * File permissions are reported as 666.
 * File permissions are reported as 666.
 
 
@@ -239,11 +235,6 @@ However, with the upcoming upgrade to AWS Java SDK V2, these classes will need t
 updated to implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
 updated to implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
 For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html).
 For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html).
 
 
-*Important*: The S3A connector no longer supports username and secrets
-in URLs of the form `s3a://key:secret@bucket/`.
-It is near-impossible to stop those secrets being logged —which is why
-a warning has been printed since Hadoop 2.8 whenever such a URL was used.
-
 ### Authentication properties
 ### Authentication properties
 
 
 ```xml
 ```xml
@@ -369,13 +360,13 @@ There are a number of AWS Credential Providers inside the `hadoop-aws` JAR:
 | `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials                     |
 | `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials                     |
 
 
 
 
-There are also many in the Amazon SDKs, with the common ones being.
+There are also many in the Amazon SDKs, with the common ones being as follows
 
 
-| classname | description |
-|-----------|-------------|
-| `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | AWS Environment Variables |
-| `software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider`| EC2 Metadata Credentials |
-| `software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider`| EC2/k8s Metadata Credentials |
+| classname                                                                        | description                  |
+|----------------------------------------------------------------------------------|------------------------------|
+| `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | AWS Environment Variables    |
+| `software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider`     | EC2 Metadata Credentials     |
+| `software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider`           | EC2/k8s Metadata Credentials |
 
 
 
 
 
 
@@ -716,16 +707,145 @@ Here are the S3A properties for use in production; some testing-related
 options are covered in [Testing](./testing.md).
 options are covered in [Testing](./testing.md).
 
 
 ```xml
 ```xml
+
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>
+    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
+    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+    software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider,
+    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
+  </value>
+  <description>
+    Comma-separated class names of credential provider classes which implement
+    software.amazon.awssdk.auth.credentials.AwsCredentialsProvider.
+
+    When S3A delegation tokens are not enabled, this list will be used
+    to directly authenticate with S3 and other AWS services.
+    When S3A Delegation tokens are enabled, depending upon the delegation
+    token binding it may be used
+    to communicate wih the STS endpoint to request session/role
+    credentials.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.security.credential.provider.path</name>
+  <value />
+  <description>
+    Optional comma separated list of credential providers, a list
+    which is prepended to that set in hadoop.security.credential.provider.path
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.arn</name>
+  <value />
+  <description>
+    AWS ARN for the role to be assumed.
+    Required if the fs.s3a.aws.credentials.provider contains
+    org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.session.name</name>
+  <value />
+  <description>
+    Session name for the assumed role, must be valid characters according to
+    the AWS APIs.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    If not set, one is generated from the current Hadoop/Kerberos username.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.policy</name>
+  <value/>
+  <description>
+    JSON policy to apply to the role.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.session.duration</name>
+  <value>30m</value>
+  <description>
+    Duration of assumed roles before a refresh is attempted.
+    Used when session tokens are requested.
+    Range: 15m to 1h
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint</name>
+  <value/>
+  <description>
+    AWS Security Token Service Endpoint.
+    If unset, uses the default endpoint.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    Used by the AssumedRoleCredentialProvider and in Session and Role delegation
+    tokens.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint.region</name>
+  <value></value>
+  <description>
+    AWS Security Token Service Endpoint's region;
+    Needed if fs.s3a.assumed.role.sts.endpoint points to an endpoint
+    other than the default one and the v4 signature is used.
+    Used by the AssumedRoleCredentialProvider and in Session and Role delegation
+    tokens.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+  <description>
+    List of credential providers to authenticate with the STS endpoint and
+    retrieve short-lived role credentials.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    If unset, uses "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.delegation.token.binding</name>
+  <value></value>
+  <description>
+    The name of a class to provide delegation tokens support in S3A.
+    If unset: delegation token support is disabled.
+
+    Note: for job submission to actually collect these tokens,
+    Kerberos must be enabled.
+
+    Bindings available in hadoop-aws are:
+    org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding
+    org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding
+    org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding
+  </description>
+</property>
+
 <property>
 <property>
   <name>fs.s3a.connection.maximum</name>
   <name>fs.s3a.connection.maximum</name>
-  <value>15</value>
-  <description>Controls the maximum number of simultaneous connections to S3.</description>
+  <value>96</value>
+  <description>Controls the maximum number of simultaneous connections to S3.
+    This must be bigger than the value of fs.s3a.threads.max so as to stop
+    threads being blocked waiting for new HTTPS connections.
+    Why not equal? The AWS SDK transfer manager also uses these connections.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.connection.ssl.enabled</name>
   <name>fs.s3a.connection.ssl.enabled</name>
   <value>true</value>
   <value>true</value>
-  <description>Enables or disables SSL connections to S3.</description>
+  <description>Enables or disables SSL connections to AWS services.
+    Also sets the default port to use for the s3a proxy settings,
+    when not explicitly set in fs.s3a.proxy.port.</description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -736,16 +856,6 @@ options are covered in [Testing](./testing.md).
   </description>
   </description>
 </property>
 </property>
 
 
-<property>
-  <name>fs.s3a.endpoint.region</name>
-  <description>AWS S3 region for a bucket, which bypasses the parsing of
-    fs.s3a.endpoint to know the region. Would be helpful in avoiding errors
-    while using privateLink URL and explicitly set the bucket region.
-    If set to a blank string (or 1+ space), falls back to the
-    (potentially brittle) SDK region resolution process.
-  </description>
-</property>
-
 <property>
 <property>
   <name>fs.s3a.path.style.access</name>
   <name>fs.s3a.path.style.access</name>
   <value>false</value>
   <value>false</value>
@@ -788,8 +898,14 @@ options are covered in [Testing](./testing.md).
 
 
 <property>
 <property>
   <name>fs.s3a.attempts.maximum</name>
   <name>fs.s3a.attempts.maximum</name>
-  <value>20</value>
-  <description>How many times we should retry commands on transient errors.</description>
+  <value>5</value>
+  <description>
+    Number of times the AWS client library should retry errors before
+    escalating to the S3A code: {@value}.
+    The S3A connector does its own selective retries; the only time the AWS
+    SDK operations are not wrapped is during multipart copy via the AWS SDK
+    transfer manager.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -804,20 +920,6 @@ options are covered in [Testing](./testing.md).
   <description>Socket connection timeout in milliseconds.</description>
   <description>Socket connection timeout in milliseconds.</description>
 </property>
 </property>
 
 
-<property>
-  <name>fs.s3a.paging.maximum</name>
-  <value>5000</value>
-  <description>How many keys to request from S3 when doing
-     directory listings at a time.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.max</name>
-  <value>10</value>
-  <description> Maximum number of concurrent active (part)uploads,
-  which each use a thread from the threadpool.</description>
-</property>
-
 <property>
 <property>
   <name>fs.s3a.socket.send.buffer</name>
   <name>fs.s3a.socket.send.buffer</name>
   <value>8192</value>
   <value>8192</value>
@@ -830,6 +932,20 @@ options are covered in [Testing](./testing.md).
   <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
   <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
 </property>
 </property>
 
 
+<property>
+  <name>fs.s3a.paging.maximum</name>
+  <value>5000</value>
+  <description>How many keys to request from S3 when doing
+     directory listings at a time.</description>
+</property>
+
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>64</value>
+  <description>The total number of threads available in the filesystem for data
+    uploads *or any other queued filesystem operation*.</description>
+</property>
+
 <property>
 <property>
   <name>fs.s3a.threads.keepalivetime</name>
   <name>fs.s3a.threads.keepalivetime</name>
   <value>60</value>
   <value>60</value>
@@ -839,9 +955,25 @@ options are covered in [Testing](./testing.md).
 
 
 <property>
 <property>
   <name>fs.s3a.max.total.tasks</name>
   <name>fs.s3a.max.total.tasks</name>
-  <value>5</value>
-  <description>Number of (part)uploads allowed to the queue before
-  blocking additional uploads.</description>
+  <value>32</value>
+  <description>The number of operations which can be queued for execution.
+  This is in addition to the number of active threads in fs.s3a.threads.max.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.executor.capacity</name>
+  <value>16</value>
+  <description>The maximum number of submitted tasks which is a single
+    operation (e.g. rename(), delete()) may submit simultaneously for
+    execution -excluding the IO-heavy block uploads, whose capacity
+    is set in "fs.s3a.fast.upload.active.blocks"
+
+    All tasks are submitted to the shared thread pool whose size is
+    set in "fs.s3a.threads.max"; the value of capacity should be less than that
+    of the thread pool itself, as the goal is to stop a single operation
+    from overloading that thread pool.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -854,7 +986,7 @@ options are covered in [Testing](./testing.md).
 
 
 <property>
 <property>
   <name>fs.s3a.multipart.threshold</name>
   <name>fs.s3a.multipart.threshold</name>
-  <value>128MB</value>
+  <value>128M</value>
   <description>How big (in bytes) to split upload or copy operations up into.
   <description>How big (in bytes) to split upload or copy operations up into.
     This also controls the partition size in renamed files, as rename() involves
     This also controls the partition size in renamed files, as rename() involves
     copying the source file(s).
     copying the source file(s).
@@ -874,23 +1006,52 @@ options are covered in [Testing](./testing.md).
 <property>
 <property>
   <name>fs.s3a.acl.default</name>
   <name>fs.s3a.acl.default</name>
   <description>Set a canned ACL for newly created and copied objects. Value may be Private,
   <description>Set a canned ACL for newly created and copied objects. Value may be Private,
-    PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
-    or BucketOwnerFullControl.
+      PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
+      or BucketOwnerFullControl.
     If set, caller IAM role must have "s3:PutObjectAcl" permission on the bucket.
     If set, caller IAM role must have "s3:PutObjectAcl" permission on the bucket.
-    </description>
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.multipart.purge</name>
   <name>fs.s3a.multipart.purge</name>
   <value>false</value>
   <value>false</value>
   <description>True if you want to purge existing multipart uploads that may not have been
   <description>True if you want to purge existing multipart uploads that may not have been
-     completed/aborted correctly</description>
+    completed/aborted correctly. The corresponding purge age is defined in
+    fs.s3a.multipart.purge.age.
+    If set, when the filesystem is instantiated then all outstanding uploads
+    older than the purge age will be terminated -across the entire bucket.
+    This will impact multipart uploads by other applications and users. so should
+    be used sparingly, with an age value chosen to stop failed uploads, without
+    breaking ongoing operations.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.multipart.purge.age</name>
   <name>fs.s3a.multipart.purge.age</name>
   <value>86400</value>
   <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge</description>
+  <description>Minimum age in seconds of multipart uploads to purge
+    on startup if "fs.s3a.multipart.purge" is true
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.encryption.algorithm</name>
+  <description>Specify a server-side encryption or client-side
+    encryption algorithm for s3a: file system. Unset by default. It supports the
+    following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'SSE-C', and 'CSE-KMS'
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.encryption.key</name>
+  <description>Specific encryption key to use if fs.s3a.encryption.algorithm
+    has been set to 'SSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C
+    , the value of this property should be the Base64 encoded key. If you are
+    using SSE-KMS and leave this property empty, you'll be using your default's
+    S3 KMS key, otherwise you should set this property to the specific KMS key
+    id. In case of 'CSE-KMS' this value needs to be the AWS-KMS Key ID
+    generated from AWS console.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -900,23 +1061,19 @@ options are covered in [Testing](./testing.md).
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.encryption.algorithm</name>
-  <description>Specify a server-side encryption or client-side
-     encryption algorithm for s3a: file system. Unset by default. It supports the
-     following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'SSE-C', and 'CSE-KMS'
-  </description>
+  <name>fs.s3a.accesspoint.required</name>
+  <value>false</value>
+  <description>Require that all S3 access is made through Access Points and not through
+  buckets directly. If enabled, use per-bucket overrides to allow bucket access to a specific set
+  of buckets.</description>
 </property>
 </property>
 
 
 <property>
 <property>
-    <name>fs.s3a.encryption.key</name>
-    <description>Specific encryption key to use if fs.s3a.encryption.algorithm
-        has been set to 'SSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C
-    , the value of this property should be the Base64 encoded key. If you are
-     using SSE-KMS and leave this property empty, you'll be using your default's
-     S3 KMS key, otherwise you should set this property to the specific KMS key
-     id. In case of 'CSE-KMS' this value needs to be the AWS-KMS Key ID
-     generated from AWS console.
-    </description>
+  <name>fs.s3a.block.size</name>
+  <value>32M</value>
+  <description>Block size to use when reading files using s3a: file system.
+    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
@@ -930,9 +1087,51 @@ options are covered in [Testing](./testing.md).
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.block.size</name>
-  <value>32M</value>
-  <description>Block size to use when reading files using s3a: file system.
+  <name>fs.s3a.fast.upload.buffer</name>
+  <value>disk</value>
+  <description>
+    The buffering mechanism to for data being written.
+    Values: disk, array, bytebuffer.
+
+    "disk" will use the directories listed in fs.s3a.buffer.dir as
+    the location(s) to save data prior to being uploaded.
+
+    "array" uses arrays in the JVM heap
+
+    "bytebuffer" uses off-heap memory within the JVM.
+
+    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
+    of blocks set by:
+
+        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
+
+    If using either of these mechanisms, keep this value low
+
+    The total number of threads performing work across all threads is set by
+    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
+    work items.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.fast.upload.active.blocks</name>
+  <value>4</value>
+  <description>
+    Maximum Number of blocks a single output stream can have
+    active (uploading, or queued to the central FileSystem
+    instance's pool of queued operations.
+
+    This stops a single stream overloading the shared thread pool.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.readahead.range</name>
+  <value>64K</value>
+  <description>Bytes to read ahead during a seek() before closing and
+  re-opening the S3 HTTP connection. This option will be overridden if
+  any call to setReadahead() is made to an open stream.
+  A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
   </description>
   </description>
 </property>
 </property>
 
 
@@ -958,118 +1157,232 @@ options are covered in [Testing](./testing.md).
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.AbstractFileSystem.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3A</value>
-  <description>The implementation class of the S3A AbstractFileSystem.</description>
+  <name>fs.s3a.retry.limit</name>
+  <value>7</value>
+  <description>
+    Number of times to retry any repeatable S3 client request on failure,
+    excluding throttling requests.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.readahead.range</name>
-  <value>64K</value>
-  <description>Bytes to read ahead during a seek() before closing and
-  re-opening the S3 HTTP connection. This option will be overridden if
-  any call to setReadahead() is made to an open stream.</description>
+  <name>fs.s3a.retry.interval</name>
+  <value>500ms</value>
+  <description>
+    Initial retry interval when retrying operations for any reason other
+    than S3 throttle errors.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.input.async.drain.threshold</name>
-  <value>64K</value>
-  <description>Bytes to read ahead during a seek() before closing and
-  re-opening the S3 HTTP connection. This option will be overridden if
-  any call to setReadahead() is made to an open stream.</description>
+  <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>100ms</value>
+  <description>
+    Initial between retry attempts on throttled requests, +/- 50%. chosen at random.
+    i.e. for an intial value of 3000ms, the initial delay would be in the range 1500ms to 4500ms.
+    Backoffs are exponential; again randomness is used to avoid the thundering heard problem.
+    500ms is the default value used by the AWS S3 Retry policy.
+  </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>true</value>
+  <description>
+    Enable support in the S3A filesystem for the "Magic" committer.
+  </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>append</value>
+  <description>
+    Staging committer conflict resolution policy.
+    Supported: "fail", "append", "replace".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.committer.abort.pending.uploads</name>
+  <value>true</value>
+  <description>
+    Should the committers abort all pending uploads to the destination
+    directory?
+
+    Set to false if more than one job is writing to the same directory tree.
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.list.version</name>
   <name>fs.s3a.list.version</name>
   <value>2</value>
   <value>2</value>
-  <description>Select which version of the S3 SDK's List Objects API to use.
-  Currently support 2 (default) and 1 (older API).</description>
+  <description>
+    Select which version of the S3 SDK's List Objects API to use.  Currently
+    support 2 (default) and 1 (older API).
+  </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.connection.request.timeout</name>
   <name>fs.s3a.connection.request.timeout</name>
   <value>0</value>
   <value>0</value>
   <description>
   <description>
-  Time out on HTTP requests to the AWS service; 0 means no timeout.
-  Measured in seconds; the usual time suffixes are all supported
+    Time out on HTTP requests to the AWS service; 0 means no timeout.
+    Measured in seconds; the usual time suffixes are all supported
 
 
-  Important: this is the maximum duration of any AWS service call,
-  including upload and copy operations. If non-zero, it must be larger
-  than the time to upload multi-megabyte blocks to S3 from the client,
-  and to rename many-GB files. Use with care.
+    Important: this is the maximum duration of any AWS service call,
+    including upload and copy operations. If non-zero, it must be larger
+    than the time to upload multi-megabyte blocks to S3 from the client,
+    and to rename many-GB files. Use with care.
 
 
-  Values that are larger than Integer.MAX_VALUE milliseconds are
-  converged to Integer.MAX_VALUE milliseconds
+    Values that are larger than Integer.MAX_VALUE milliseconds are
+    converged to Integer.MAX_VALUE milliseconds
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.bucket.probe</name>
-  <value>0</value>
+  <name>fs.s3a.etag.checksum.enabled</name>
+  <value>false</value>
   <description>
   <description>
-     The value can be 0 (default), 1 or 2.
-     When set to 0, bucket existence checks won't be done
-     during initialization thus making it faster.
-     Though it should be noted that when the bucket is not available in S3,
-     or if fs.s3a.endpoint points to the wrong instance of a private S3 store
-     consecutive calls like listing, read, write etc. will fail with
-     an UnknownStoreException.
-     When set to 1, the bucket existence check will be done using the
-     V1 API of the S3 protocol which doesn't verify the client's permissions
-     to list or read data in the bucket.
-     When set to 2, the bucket existence check will be done using the
-     V2 API of the S3 protocol which does verify that the
-     client has permission to read the bucket.
+    Should calls to getFileChecksum() return the etag value of the remote
+    object.
+    WARNING: if enabled, distcp operations between HDFS and S3 will fail unless
+    -skipcrccheck is set.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.object.content.encoding</name>
-  <value></value>
+  <name>fs.s3a.change.detection.source</name>
+  <value>etag</value>
   <description>
   <description>
-    Content encoding: gzip, deflate, compress, br, etc.
-    This will be set in the "Content-Encoding" header of the object,
-    and returned in HTTP HEAD/GET requests.
+    Select which S3 object attribute to use for change detection.
+    Currently support 'etag' for S3 object eTags and 'versionid' for
+    S3 object version IDs.  Use of version IDs requires object versioning to be
+    enabled for each S3 bucket utilized.  Object versioning is disabled on
+    buckets by default. When version ID is used, the buckets utilized should
+    have versioning enabled before any data is written.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.create.storage.class</name>
-  <value></value>
+  <name>fs.s3a.change.detection.mode</name>
+  <value>server</value>
   <description>
   <description>
-      Storage class: standard, reduced_redundancy, intelligent_tiering, etc.
-      Specify the storage class for S3A PUT object requests.
-      If not set the storage class will be null
-      and mapped to default standard class on S3.
+    Determines how change detection is applied to alert to inconsistent S3
+    objects read during or after an overwrite. Value 'server' indicates to apply
+    the attribute constraint directly on GetObject requests to S3. Value 'client'
+    means to do a client-side comparison of the attribute value returned in the
+    response.  Value 'server' would not work with third-party S3 implementations
+    that do not support these constraints on GetObject. Values 'server' and
+    'client' generate RemoteObjectChangedException when a mismatch is detected.
+    Value 'warn' works like 'client' but generates only a warning.  Value 'none'
+    will ignore change detection completely.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.prefetch.enabled</name>
-  <value>false</value>
+  <name>fs.s3a.change.detection.version.required</name>
+  <value>true</value>
+  <description>
+    Determines if S3 object version attribute defined by
+    fs.s3a.change.detection.source should be treated as required.  If true and the
+    referred attribute is unavailable in an S3 GetObject response,
+    NoVersionAttributeException is thrown.  Setting to 'true' is encouraged to
+    avoid potential for inconsistent reads with third-party S3 implementations or
+    against S3 buckets that have object versioning disabled.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.ssl.channel.mode</name>
+  <value>default_jsse</value>
   <description>
   <description>
-    Enables prefetching and caching when reading from input stream.
+    If secure connections to S3 are enabled, configures the SSL
+    implementation used to encrypt connections to S3. Supported values are:
+    "default_jsse", "default_jsse_with_gcm", "default", and "openssl".
+    "default_jsse" uses the Java Secure Socket Extension package (JSSE).
+    However, when running on Java 8, the GCM cipher is removed from the list
+    of enabled ciphers. This is due to performance issues with GCM in Java 8.
+    "default_jsse_with_gcm" uses the JSSE with the default list of cipher
+    suites. "default_jsse_with_gcm" is equivalent to the behavior prior to
+    this feature being introduced. "default" attempts to use OpenSSL rather
+    than the JSSE for SSL encryption, if OpenSSL libraries cannot be loaded,
+    it falls back to the "default_jsse" behavior. "openssl" attempts to use
+    OpenSSL as well, but fails if OpenSSL libraries cannot be loaded.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
-  <name>fs.s3a.prefetch.block.size</name>
-  <value>8MB</value>
+  <name>fs.s3a.downgrade.syncable.exceptions</name>
+  <value>true</value>
   <description>
   <description>
-      The size of a single prefetched block of data.
-      Decreasing this will increase the number of prefetches required, and may negatively impact performance.
+    Warn but continue when applications use Syncable.hsync when writing
+    to S3A.
   </description>
   </description>
 </property>
 </property>
 
 
+<!--
+The switch to turn S3A auditing on or off.
+-->
 <property>
 <property>
-  <name>fs.s3a.prefetch.block.count</name>
-  <value>8</value>
+  <name>fs.s3a.audit.enabled</name>
+  <value>true</value>
   <description>
   <description>
-      Maximum number of blocks prefetched concurrently at any given time.
+    Should auditing of S3A requests be enabled?
   </description>
   </description>
 </property>
 </property>
-```
 
 
+```
 ## <a name="retry_and_recovery"></a>Retry and Recovery
 ## <a name="retry_and_recovery"></a>Retry and Recovery
 
 
 The S3A client makes a best-effort attempt at recovering from network failures;
 The S3A client makes a best-effort attempt at recovering from network failures;
@@ -1089,8 +1402,10 @@ not the failing operation is idempotent.
 * Interruptions: `InterruptedIOException`, `InterruptedException`.
 * Interruptions: `InterruptedIOException`, `InterruptedException`.
 * Rejected HTTP requests: `InvalidRequestException`
 * Rejected HTTP requests: `InvalidRequestException`
 
 
-These are all considered unrecoverable: S3A will make no attempt to recover
-from them.
+These and others are all considered unrecoverable: S3A will make no attempt to recover
+from them. The AWS SDK itself may retry before the S3A connector sees the exception.
+As an example, the SDK will retry on `UnknownHostException` in case it is a transient
+DNS error.
 
 
 ### Possibly Recoverable Problems: Retry
 ### Possibly Recoverable Problems: Retry
 
 
@@ -1141,17 +1456,9 @@ only succeed if the first `delete()` call has already succeeded.
 Because S3 is eventually consistent *and* doesn't support an
 Because S3 is eventually consistent *and* doesn't support an
 atomic create-no-overwrite operation, the choice is more ambiguous.
 atomic create-no-overwrite operation, the choice is more ambiguous.
 
 
-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`).
-
-
-
-
-
+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.
 
 
 ### Throttled requests from S3
 ### Throttled requests from S3
 
 
@@ -1657,13 +1964,17 @@ the storage class you want.
 ```
 ```
 
 
 Please note that S3A does not support reading from archive storage classes at the moment.
 Please note that S3A does not support reading from archive storage classes at the moment.
-`AccessDeniedException` with InvalidObjectState will be thrown if you're trying to do so.
+`AccessDeniedException` with `InvalidObjectState` will be thrown if you're trying to do so.
+
+When a file is "renamed" through the s3a connector it is copied then deleted.
+Storage Classes will normally be propagated.
+
 
 
-## <a name="upload"></a>Configuring S3A for S3 on Outposts
+## <a name="outposts"></a>Configuring S3A for S3 on Outposts
 
 
 S3A now supports [S3 on Outposts](https://docs.aws.amazon.com/AmazonS3/latest/userguide/S3onOutposts.html).
 S3A now supports [S3 on Outposts](https://docs.aws.amazon.com/AmazonS3/latest/userguide/S3onOutposts.html).
 Accessing data through an access point is done by using its Amazon Resource Name (ARN), as opposed to just the bucket name.
 Accessing data through an access point is done by using its Amazon Resource Name (ARN), as opposed to just the bucket name.
-The only supported storage class on Outposts is **OUTPOSTS**, and by default objects are encrypted with [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-outposts-data-encryption.html).
+The only supported storage class on Outposts is `OUTPOSTS`, and by default objects are encrypted with [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-outposts-data-encryption.html).
 You can set the Access Point ARN property using the following per bucket configuration property:
 You can set the Access Point ARN property using the following per bucket configuration property:
 
 
 ```xml
 ```xml

+ 66 - 12
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md

@@ -377,6 +377,19 @@ This adds extra overhead to every operation, but helps verify that the connector
 not keeping markers where it needs to be deleting them -and hence backwards compatibility
 not keeping markers where it needs to be deleting them -and hence backwards compatibility
 is maintained.
 is maintained.
 
 
+## <a name="enabling-prefetch"></a> Enabling prefetch for all tests
+
+The tests are run with prefetch if the `prefetch` property is set in the
+maven build. This can be combined with the scale tests as well.
+
+```bash
+mvn verify -Dprefetch
+
+mvn verify -Dparallel-tests -Dprefetch -DtestsThreadCount=8
+
+mvn verify -Dparallel-tests -Dprefetch -Dscale -DtestsThreadCount=8
+```
+
 ## <a name="scale"></a> Scale Tests
 ## <a name="scale"></a> Scale Tests
 
 
 There are a set of tests designed to measure the scalability and performance
 There are a set of tests designed to measure the scalability and performance
@@ -519,7 +532,7 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
 The tests are executed in an order to only clean up created files after
 The tests are executed in an order to only clean up created files after
 the end of all the tests. If the tests are interrupted, the test data will remain.
 the end of all the tests. If the tests are interrupted, the test data will remain.
 
 
-## <a name="alternate_s3"></a> Load tests.
+## <a name="load"></a> Load tests.
 
 
 Some are designed to overload AWS services with more
 Some are designed to overload AWS services with more
 requests per second than an AWS account is permitted.
 requests per second than an AWS account is permitted.
@@ -545,6 +558,32 @@ which address issues. In particular, we encourage testing of Hadoop release
 candidates, as these third-party endpoints get even less testing than the
 candidates, as these third-party endpoints get even less testing than the
 S3 endpoint itself.
 S3 endpoint itself.
 
 
+The core XML settings to turn off tests of features unavailable
+on third party stores.
+
+```xml
+  <property>
+    <name>test.fs.s3a.encryption.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>test.fs.s3a.create.storage.class.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>fs.s3a.select.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>test.fs.s3a.sts.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>test.fs.s3a.create.create.acl.enabled</name>
+    <value>false</value>
+ < /property>
+```
+
 ### Public datasets used in tests
 ### Public datasets used in tests
 
 
 Some tests rely on the presence of existing public datasets available on Amazon S3.
 Some tests rely on the presence of existing public datasets available on Amazon S3.
@@ -585,7 +624,7 @@ S3 storage class, these tests might fail. They can be disabled.
 </property>
 </property>
 ```
 ```
 
 
-### Configuring the CSV file read tests**
+### Configuring the CSV file read tests
 
 
 To test on alternate infrastructures supporting
 To test on alternate infrastructures supporting
 the same APIs, the option `fs.s3a.scale.test.csvfile` must either be
 the same APIs, the option `fs.s3a.scale.test.csvfile` must either be
@@ -618,19 +657,20 @@ your `core-site.xml` file, so that trying to use S3 select fails fast with
 a meaningful error ("S3 Select not supported") rather than a generic Bad Request
 a meaningful error ("S3 Select not supported") rather than a generic Bad Request
 exception.
 exception.
 
 
-### <a name="enabling-prefetch"></a> Enabling prefetch for all tests
+### Disabling V1 List API tests
 
 
-The tests are run with prefetch if the `prefetch` property is set in the
-maven build. This can be combined with the scale tests as well.
 
 
-```bash
-mvn verify -Dprefetch
-
-mvn verify -Dparallel-tests -Dprefetch -DtestsThreadCount=8
-
-mvn verify -Dparallel-tests -Dprefetch -Dscale -DtestsThreadCount=8
+If `ITestS3AContractGetFileStatusV1List` fails with any error about unsupported API.
+```xml
+  <property>
+    <name>test.fs.s3a.list.v1.enabled</name>
+    <value>false</value>
+  </property>
 ```
 ```
 
 
+Note: there's no equivalent for turning off v2 listing API, which all stores are now
+expected to support.
+
 
 
 ### Testing Requester Pays
 ### Testing Requester Pays
 
 
@@ -697,6 +737,20 @@ The default is ""; meaning "use the amazon default endpoint" (`sts.amazonaws.com
 Consult the [AWS documentation](https://docs.aws.amazon.com/general/latest/gr/rande.html#sts_region)
 Consult the [AWS documentation](https://docs.aws.amazon.com/general/latest/gr/rande.html#sts_region)
 for the full list of locations.
 for the full list of locations.
 
 
+### Disabling Content Encoding tests
+
+Tests in `ITestS3AContentEncoding` may need disabling
+```xml
+  <property>
+    <name>test.fs.s3a.content.encoding.enabled</name>
+    <value>false</value>
+  </property>
+```
+### Tests which may fail (and which you can ignore)
+
+* `ITestS3AContractMultipartUploader` tests `testMultipartUploadAbort` and `testSingleUpload` raising `FileNotFoundException`
+* `ITestS3AMiscOperations.testEmptyFileChecksums`: if the FS encrypts data always.
+
 ## <a name="debugging"></a> Debugging Test failures
 ## <a name="debugging"></a> Debugging Test failures
 
 
 Logging at debug level is the standard way to provide more diagnostics output;
 Logging at debug level is the standard way to provide more diagnostics output;
@@ -943,7 +997,7 @@ sequential one afterwards. The IO heavy ones must also be subclasses of
 `S3AScaleTestBase` and so only run if the system/maven property
 `S3AScaleTestBase` and so only run if the system/maven property
 `fs.s3a.scale.test.enabled` is true.
 `fs.s3a.scale.test.enabled` is true.
 
 
-## Individual test cases can be run in an IDE
+### Individual test cases can be run in an IDE
 
 
 This is invaluable for debugging test failures.
 This is invaluable for debugging test failures.
 
 

+ 415 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md

@@ -0,0 +1,415 @@
+<!---
+  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.
+-->
+
+# Working with Third-party S3 Stores
+
+The S3A connector works well with third-party S3 stores if the following requirements are met:
+
+* It correctly implements the core S3 REST API, including support for uploads and the V2 listing API.
+* The store supports the AWS V4 signing API *or* a custom signer is switched to.
+  This release does not support the legacy v2 signing API.
+* Errors are reported with the same HTTPS status codes as the S3 store. Error messages do not
+  need to be consistent.
+* The store is consistent.
+
+There are also specific deployment requirements:
+* The clock on the store and the client are close enough that signing works.
+* The client is correctly configured to connect to the store *and not use unavailable features*
+* If HTTPS authentication is used, the client/JVM TLS configurations allows it to authenticate the endpoint.
+
+The features which may be unavailable include:
+
+* Checksum-based server-side change detection during copy/read (`fs.s3a.change.detection.mode=server`)
+* Object versioning and version-based change detection (`fs.s3a.change.detection.source=versionid` and `fs.s3a.versioned.store=true`)
+* Bulk delete (`fs.s3a.multiobjectdelete.enable=true`)
+* Encryption. (`fs.s3a.encryption.algorithm`)
+* Storage class set in `fs.s3a.create.storage.class`
+* Content encodings as set with `fs.s3a.object.content.encoding`.
+* Optional Bucket Probes at startup (`fs.s3a.bucket.probe = 0`).
+  This is now the default -do not change it.
+* List API to use (`fs.s3a.list.version = 1`)
+
+## Configuring s3a to connect to a third party store
+
+
+### Connecting to a third party object store over HTTPS
+
+The core setting for a third party store is to change the endpoint in `fs.s3a.endpoint`.
+
+This can be a URL or a hostname/hostname prefix
+For third-party stores without virtual hostname support, providing the URL is straightforward;
+path style access must also be enabled in `fs.s3a.path.style.access`.
+
+The v4 signing algorithm requires a region to be set in `fs.s3a.endpoint.region`.
+A non-empty value is generally sufficient, though some deployments may require
+a specific value.
+
+Finally, assuming the credential source is the normal access/secret key
+then these must be set, either in XML or (preferred) in a JCEKS file.
+
+```xml
+
+  <property>
+    <name>fs.s3a.endpoint</name>
+    <value>https://storeendpoint.example.com</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.path.style.access</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.endpoint.region</name>
+    <value>anything</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.access.key</name>
+    <value>13324445</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.secret.key</name>
+    <value>4C6B906D-233E-4E56-BCEA-304CC73A14F8</value>
+  </property>
+
+```
+
+If per-bucket settings are used here, then third-party stores and credentials may be used alongside an AWS store.
+
+# Troubleshooting
+
+The most common problem when talking to third-party stores are
+
+1. The S3A client is still configured to talk to the AWS S3 endpoint. This leads to authentication failures and/or reports that the bucket is unknown.
+2. Path access has not been enabled, the client is generating a host name for the target bucket and it does not exist.
+3. Invalid authentication credentials.
+4. JVM HTTPS settings include the certificates needed to negotiate a TLS connection with the store.
+
+
+## How to improve troubleshooting
+
+### log more network info
+
+There are some very low level logs.
+```properties
+# Log all HTTP requests made; includes S3 interaction. This may
+# include sensitive information such as account IDs in HTTP headers.
+log4j.logger.software.amazon.awssdk.request=DEBUG
+
+# Turn on low level HTTP protocol debugging
+log4j.logger.org.apache.http.wire=DEBUG
+
+# async client
+log4j.logger.io.netty.handler.logging=DEBUG
+log4j.logger.io.netty.handler.codec.http2.Http2FrameLogger=DEBUG
+```
+
+### Cut back on retries, shorten timeouts
+
+By default, there's a lot of retries going on in the AWS connector (which even retries on DNS failures)
+and in the S3A code which invokes it.
+
+Normally this helps prevent long-lived jobs from failing due to a transient network problem, however
+it means that when trying to debug connectivity problems, the commands can hang for a long time
+as they keep trying to reconnect to ports which are never going to be available.
+
+```xml
+
+  <property>
+    <name>fs.iostatistics.logging.level</name>
+    <value>info</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.nonexistent-bucket-example.attempts.maximum</name>
+    <value>0</value>
+  </property>
+
+  <property>
+   <name>fs.s3a.bucket.nonexistent-bucket-example.retry.limit</name>
+   <value>1</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.nonexistent-bucket-example.connection.timeout</name>
+    <value>500</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.nonexistent-bucket-example.connection.establish.timeout</name>
+    <value>500</value>
+  </property>
+```
+## Cloudstore's Storediag
+
+There's an external utility, [cloudstore](https://github.com/steveloughran/cloudstore) whose [storediag](https://github.com/steveloughran/cloudstore#command-storediag) exists to debug the connection settings to hadoop cloud storage.
+
+```bash
+hadoop jar cloudstore-1.0.jar storediag s3a://nonexistent-bucket-example/
+```
+
+The main reason it's not an ASF release is that it allows for a rapid release cycle, sometimes hours; if anyone doesn't trust
+third-party code then they can download and build it themselves.
+
+
+# Problems
+
+## S3A client still pointing at AWS endpoint
+
+This is the most common initial problem, as it happens by default.
+
+To fix, set `fs.s3a.endpoint` to the URL of the internal store.
+
+### `org.apache.hadoop.fs.s3a.UnknownStoreException: `s3a://nonexistent-bucket-example/':  Bucket does not exist`
+
+Either the bucket doesn't exist, or the bucket does exist but the endpoint is still set to an AWS endpoint.
+
+```
+stat: `s3a://nonexistent-bucket-example/':  Bucket does not exist
+```
+The hadoop filesystem commands don't log stack traces on failure -adding this adds too much risk
+of breaking scripts, and the output is very uninformative
+
+```
+stat: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example:
+software.amazon.awssdk.services.s3.model.S3Exception: null
+(Service: S3, Status Code: 403, Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null
+```
+
+It is possible to turn on debugging
+
+```
+log4j.logger.org.apache.hadoop.fs.shell=DEBUG
+```
+
+After which useful stack traces are logged.
+
+```
+org.apache.hadoop.fs.s3a.UnknownStoreException: `s3a://nonexistent-bucket-example/':  Bucket does not exist
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$null$3(S3AFileSystem.java:1075)
+    at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122)
+    at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376)
+    at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468)
+    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372)
+    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getS3Region$4(S3AFileSystem.java:1039)
+    at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543)
+    at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524)
+    at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2631)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.getS3Region(S3AFileSystem.java:1038)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:982)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:622)
+    at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3452)
+```
+
+### `S3Exception: null (Service: S3, Status Code: 403...` or `AccessDeniedException`
+
+* Endpoint is default
+* Credentials were not issued by AWS .
+* `fs.s3a.endpoint.region` unset.
+
+If the client doesn't have any AWS credentials (from hadoop settings, environment variables or elsewhere) then
+the binding will fail even before the existence of the bucket can be probed for.
+
+```bash
+hadoop fs -stat s3a://nonexistent-bucket-example
+```
+
+
+```
+stat: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example:
+software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status Code: 403,
+ Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null
+```
+
+Or with a more detailed stack trace:
+
+```
+java.nio.file.AccessDeniedException: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example: software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status Code: 403, Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null
+        at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:235)
+        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:124)
+        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376)
+        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468)
+        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372)
+        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getS3Region$4(S3AFileSystem.java:1039)
+        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543)
+        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524)
+        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2631)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.getS3Region(S3AFileSystem.java:1038)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:982)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:622)
+        at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3452)
+```
+
+## `Received an UnknownHostException when attempting to interact with a service`
+
+
+### Hypothesis 1: Region set, but not endpoint
+
+The bucket `fs.s3a.endpoint.region` region setting is valid internally, but as the endpoint
+is still AWS, this region is not recognised.
+The S3A client's creation of an endpoint URL generates an unknown host.
+
+```xml
+  <property>
+    <name>fs.s3a.bucket.nonexistent-bucket-example.endpoint.region</name>
+    <value>internal</value>
+  </property>
+```
+
+
+```
+ls: software.amazon.awssdk.core.exception.SdkClientException:
+    Received an UnknownHostException when attempting to interact with a service.
+    See cause for the exact endpoint that is failing to resolve.
+    If this is happening on an endpoint that previously worked, there may be
+    a network connectivity issue or your DNS cache could be storing endpoints for too long.:
+    nonexistent-bucket-example.s3.internal.amazonaws.com: nodename nor servname provided, or not known
+
+
+```
+
+### Hypothesis 2: region set, endpoint set, but `fs.s3a.path.style.access` is still set to `false`
+
+* The bucket `fs.s3a.endpoint.region` region setting is valid internally,
+* and `fs.s3a.endpoint` is set to a hostname (not a URL).
+* `fs.s3a.path.style.access` set to `false`
+
+```
+ls: software.amazon.awssdk.core.exception.SdkClientException:
+    Received an UnknownHostException when attempting to interact with a service.
+    See cause for the exact endpoint that is failing to resolve.
+    If this is happening on an endpoint that previously worked, there may be
+    a network connectivity issue or your DNS cache could be storing endpoints for too long.:
+    nonexistent-bucket-example.localhost: nodename nor servname provided, or not known
+```
+
+Fix: path style access
+
+```xml
+  <property>
+    <name>fs.s3a.bucket.nonexistent-bucket-example.path.style.access</name>
+    <value>true</value>
+  </property>
+```
+
+# Connecting to Google Cloud Storage through the S3A connector
+
+It *is* possible to connect to google cloud storage through the S3A connector.
+However, Google provide their own [Cloud Storage connector](https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage).
+That is a well maintained Hadoop filesystem client which uses their XML API,
+And except for some very unusual cases, that is the connector to use.
+
+When interacting with a GCS container through the S3A connector may make sense
+* The installation doesn't have the gcs-connector JAR.
+* The different credential mechanism may be convenient.
+* There's a desired to use S3A Delegation Tokens to pass secrets with a job.
+* There's a desire to use an external S3A extension (delegation tokens etc.)
+
+The S3A connector binding works through the Google Cloud [S3 Storage API](https://cloud.google.com/distributed-cloud/hosted/docs/ga/gdch/apis/storage-s3-rest-api),
+which is a subset of the AWS API.
+
+
+To get a compatible access and secret key, follow the instructions of
+[Simple migration from Amazon S3 to Cloud Storage](https://cloud.google.com/storage/docs/aws-simple-migration#defaultproj).
+
+Here are the per-bucket setings for an example bucket "gcs-container"
+in Google Cloud Storage. Note the multiobject delete option must be disabled;
+this makes renaming and deleting significantly slower.
+
+
+```xml
+<configuration>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.access.key</name>
+    <value>GOOG1EZ....</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.secret.key</name>
+    <value>SECRETS</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.endpoint</name>
+    <value>https://storage.googleapis.com</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.bucket.probe</name>
+    <value>0</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.list.version</name>
+    <value>1</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.multiobjectdelete.enable</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.select.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.path.style.access</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.s3a.bucket.gcs-container.endpoint.region</name>
+    <value>dummy</value>
+  </property>
+
+</configuration>
+```
+
+This is a very rarely used configuration -however, it can be done, possibly as a way to interact with Google Cloud Storage in a deployment
+which lacks the GCS connector.
+
+It is also a way to regression test foundational S3A third-party store compatibility if you lack access to to any alternative.
+
+```xml
+<configuration>
+  <property>
+    <name>test.fs.s3a.encryption.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>fs.s3a.scale.test.csvfile</name>
+    <value></value>
+  </property>
+  <property>
+    <name>test.fs.s3a.sts.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>test.fs.s3a.content.encoding.enabled</name>
+    <value>false</value>
+  </property>
+</configuration>
+```
+
+_Note_ If anyone is set up to test this reguarly, please let the hadoop developer team know if regressions do surface,
+as it is not a common test configuration.

Fichier diff supprimé car celui-ci est trop grand
+ 120 - 587
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md


+ 13 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
 import java.net.URI;
 import java.net.URI;
+import java.nio.file.AccessDeniedException;
 import java.util.UUID;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 
 
@@ -33,11 +34,13 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESSPOINT_REQUIRED;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESSPOINT_REQUIRED;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
 import static org.apache.hadoop.fs.s3a.Constants.FS_S3A;
 import static org.apache.hadoop.fs.s3a.Constants.FS_S3A;
+import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
 import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
 import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -69,8 +72,14 @@ public class ITestS3ABucketExistence extends AbstractS3ATestBase {
     assertTrue("getFileStatus on root should always return a directory",
     assertTrue("getFileStatus on root should always return a directory",
             fs.getFileStatus(root).isDirectory());
             fs.getFileStatus(root).isDirectory());
 
 
-    expectUnknownStore(
-        () -> fs.listStatus(root));
+    try {
+      expectUnknownStore(
+          () -> fs.listStatus(root));
+    } catch (AccessDeniedException e) {
+      // this is a sign that there's tests with a third-party bucket and
+      // interacting with aws is not going to authenticate
+      skip("no aws credentials");
+    }
 
 
     Path src = new Path(root, "testfile");
     Path src = new Path(root, "testfile");
     Path dest = new Path(root, "dst");
     Path dest = new Path(root, "dst");
@@ -129,7 +138,8 @@ public class ITestS3ABucketExistence extends AbstractS3ATestBase {
     removeBaseAndBucketOverrides(conf,
     removeBaseAndBucketOverrides(conf,
         S3A_BUCKET_PROBE,
         S3A_BUCKET_PROBE,
         ENDPOINT,
         ENDPOINT,
-        AWS_REGION);
+        AWS_REGION,
+        PATH_STYLE_ACCESS);
     conf.setInt(S3A_BUCKET_PROBE, probe);
     conf.setInt(S3A_BUCKET_PROBE, probe);
     conf.set(AWS_REGION, EU_WEST_1);
     conf.set(AWS_REGION, EU_WEST_1);
     return conf;
     return conf;

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

@@ -40,7 +40,9 @@ import org.apache.hadoop.fs.s3a.impl.StoreContext;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.CANNED_ACL;
 import static org.apache.hadoop.fs.s3a.Constants.CANNED_ACL;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfACLTestsDisabled;
 
 
 /**
 /**
  * Tests of ACL handling in the FS.
  * Tests of ACL handling in the FS.
@@ -55,6 +57,8 @@ public class ITestS3ACannedACLs extends AbstractS3ATestBase {
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
+    skipIfACLTestsDisabled(conf);
+    disableFilesystemCaching(conf);
     removeBaseAndBucketOverrides(conf,
     removeBaseAndBucketOverrides(conf,
         CANNED_ACL);
         CANNED_ACL);
     conf.set(CANNED_ACL, LOG_DELIVERY_WRITE);
     conf.set(CANNED_ACL, LOG_DELIVERY_WRITE);

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

@@ -19,8 +19,9 @@
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
 import java.io.File;
 import java.io.File;
+import java.io.IOException;
+import java.net.ConnectException;
 import java.net.URI;
 import java.net.URI;
-import java.nio.file.AccessDeniedException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
@@ -157,7 +158,7 @@ public class ITestS3AConfiguration {
     conf.setInt(Constants.PROXY_PORT, 1);
     conf.setInt(Constants.PROXY_PORT, 1);
     String proxy =
     String proxy =
         conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT);
         conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT);
-    expectFSCreateFailure(AWSClientIOException.class,
+    expectFSCreateFailure(ConnectException.class,
         conf, "when using proxy " + proxy);
         conf, "when using proxy " + proxy);
   }
   }
 
 
@@ -211,10 +212,10 @@ public class ITestS3AConfiguration {
     conf.unset(Constants.PROXY_PORT);
     conf.unset(Constants.PROXY_PORT);
     conf.set(Constants.PROXY_HOST, "127.0.0.1");
     conf.set(Constants.PROXY_HOST, "127.0.0.1");
     conf.set(Constants.SECURE_CONNECTIONS, "true");
     conf.set(Constants.SECURE_CONNECTIONS, "true");
-    expectFSCreateFailure(AWSClientIOException.class,
+    expectFSCreateFailure(ConnectException.class,
         conf, "Expected a connection error for proxy server");
         conf, "Expected a connection error for proxy server");
     conf.set(Constants.SECURE_CONNECTIONS, "false");
     conf.set(Constants.SECURE_CONNECTIONS, "false");
-    expectFSCreateFailure(AWSClientIOException.class,
+    expectFSCreateFailure(ConnectException.class,
         conf, "Expected a connection error for proxy server");
         conf, "Expected a connection error for proxy server");
   }
   }
 
 
@@ -552,6 +553,7 @@ public class ITestS3AConfiguration {
     config.set(SIGNING_ALGORITHM_STS, "CustomSTSSigner");
     config.set(SIGNING_ALGORITHM_STS, "CustomSTSSigner");
 
 
     config.set(AWS_REGION, EU_WEST_1);
     config.set(AWS_REGION, EU_WEST_1);
+    disableFilesystemCaching(config);
     fs = S3ATestUtils.createTestFileSystem(config);
     fs = S3ATestUtils.createTestFileSystem(config);
 
 
     S3Client s3Client = getS3Client("testS3SpecificSignerOverride");
     S3Client s3Client = getS3Client("testS3SpecificSignerOverride");
@@ -564,7 +566,7 @@ public class ITestS3AConfiguration {
     intercept(StsException.class, "", () ->
     intercept(StsException.class, "", () ->
         stsClient.getSessionToken());
         stsClient.getSessionToken());
 
 
-    intercept(AccessDeniedException.class, "", () ->
+    final IOException ioe = intercept(IOException.class, "", () ->
         Invoker.once("head", bucket, () ->
         Invoker.once("head", bucket, () ->
             s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build())));
             s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build())));
 
 

+ 25 - 14
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java

@@ -29,12 +29,16 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.CONTENT_ENCODING;
 import static org.apache.hadoop.fs.s3a.Constants.CONTENT_ENCODING;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.raiseAsAssumption;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfNotEnabled;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_ENCODING;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_ENCODING;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.decodeBytes;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.decodeBytes;
 
 
 /**
 /**
  * Tests of content encoding object meta data.
  * Tests of content encoding object meta data.
+ * Some stores don't support gzip; rejection of the content encoding
+ * is downgraded to a skipped test.
  */
  */
 public class ITestS3AContentEncoding extends AbstractS3ATestBase {
 public class ITestS3AContentEncoding extends AbstractS3ATestBase {
 
 
@@ -43,6 +47,8 @@ public class ITestS3AContentEncoding extends AbstractS3ATestBase {
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
+    skipIfNotEnabled(conf, KEY_CONTENT_ENCODING_ENABLED,
+        "Skipping storage class ACL tests");
     removeBaseAndBucketOverrides(conf, CONTENT_ENCODING);
     removeBaseAndBucketOverrides(conf, CONTENT_ENCODING);
     conf.set(CONTENT_ENCODING, GZIP);
     conf.set(CONTENT_ENCODING, GZIP);
 
 
@@ -51,20 +57,25 @@ public class ITestS3AContentEncoding extends AbstractS3ATestBase {
 
 
   @Test
   @Test
   public void testCreatedObjectsHaveEncoding() throws Throwable {
   public void testCreatedObjectsHaveEncoding() throws Throwable {
-    S3AFileSystem fs = getFileSystem();
-    Path dir = methodPath();
-    fs.mkdirs(dir);
-    // even with content encoding enabled, directories do not have
-    // encoding.
-    Assertions.assertThat(getEncoding(dir))
-        .describedAs("Encoding of object %s", dir)
-        .isNull();
-    Path path = new Path(dir, "1");
-    ContractTestUtils.touch(fs, path);
-    assertObjectHasEncoding(path);
-    Path path2 = new Path(dir, "2");
-    fs.rename(path, path2);
-    assertObjectHasEncoding(path2);
+    try {
+      S3AFileSystem fs = getFileSystem();
+      Path dir = methodPath();
+      fs.mkdirs(dir);
+      // even with content encoding enabled, directories do not have
+      // encoding.
+      Assertions.assertThat(getEncoding(dir))
+          .describedAs("Encoding of object %s", dir)
+          .isNull();
+      Path path = new Path(dir, "1");
+      ContractTestUtils.touch(fs, path);
+      assertObjectHasEncoding(path);
+      Path path2 = new Path(dir, "2");
+      fs.rename(path, path2);
+      assertObjectHasEncoding(path2);
+    } catch (AWSUnsupportedFeatureException e) {
+      LOG.warn("Object store does not support {} content encoding", GZIP, e);
+      raiseAsAssumption(e);
+    }
   }
   }
 
 
   /**
   /**

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

@@ -24,7 +24,9 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.LIST_VERSION;
 import static org.apache.hadoop.fs.s3a.Constants.LIST_VERSION;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_LIST_V1_ENABLED;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfNotEnabled;
 
 
 /**
 /**
  * S3A contract tests for getFileStatus, using the v1 List Objects API.
  * S3A contract tests for getFileStatus, using the v1 List Objects API.
@@ -48,6 +50,8 @@ public class ITestS3AContractGetFileStatusV1List
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
     disableFilesystemCaching(conf);
     disableFilesystemCaching(conf);
+    skipIfNotEnabled(conf, KEY_LIST_V1_ENABLED,
+        "Skipping V1 listing tests");
     conf.setInt(Constants.MAX_PAGING_KEYS, 2);
     conf.setInt(Constants.MAX_PAGING_KEYS, 2);
 
 
     // Use v1 List Objects API
     // Use v1 List Objects API

+ 77 - 22
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.fs.s3a;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
@@ -38,7 +40,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.Statistic.STORE_REGION_PROBE;
 import static org.apache.hadoop.fs.s3a.Statistic.STORE_REGION_PROBE;
+import static org.apache.hadoop.io.IOUtils.closeStream;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 
 /**
 /**
@@ -49,6 +54,26 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
 
 
   private static final String AWS_ENDPOINT_TEST = "test-endpoint";
   private static final String AWS_ENDPOINT_TEST = "test-endpoint";
 
 
+  private static final String USW_2_BUCKET = "landsat-pds";
+
+  public static final String USW_2_STORE = "s3a://" + USW_2_BUCKET;
+
+  /**
+   * If anyone were ever to create a bucket with this UUID pair it would break the tests.
+   */
+  public static final String UNKNOWN_BUCKET = "23FA76D4-5F17-48B8-9D7D-9050269D0E40"
+      + "-8281BAF2-DBCF-47AA-8A27-F2FA3589656A";
+
+  /**
+   * New FS instance which will be closed in teardown.
+   */
+  private S3AFileSystem newFS;
+
+  @Override
+  public void teardown() throws Exception {
+    closeStream(newFS);
+    super.teardown();
+  }
 
 
   /**
   /**
    * Test to verify that not setting the region config, will lead to the client factory making
    * Test to verify that not setting the region config, will lead to the client factory making
@@ -57,50 +82,80 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
    */
    */
   @Test
   @Test
   public void testWithoutRegionConfig() throws IOException {
   public void testWithoutRegionConfig() throws IOException {
+    describe("Verify that region lookup takes place");
+
     Configuration conf = getConfiguration();
     Configuration conf = getConfiguration();
-    String bucket = getFileSystem().getBucket();
-    conf.unset(String.format("fs.s3a.bucket.%s.endpoint.region", bucket));
-    conf.unset(AWS_REGION);
+    removeBaseAndBucketOverrides(conf, AWS_REGION, PATH_STYLE_ACCESS);
+    conf.setBoolean(PATH_STYLE_ACCESS, false);
+
+    newFS = new S3AFileSystem();
 
 
-    S3AFileSystem fs = new S3AFileSystem();
-    fs.initialize(getFileSystem().getUri(), conf);
+    try {
+      newFS.initialize(getFileSystem().getUri(), conf);
+      newFS.getS3AInternals().getBucketMetadata();
+    } catch (UnknownHostException | UnknownStoreException | AccessDeniedException allowed) {
+      // these are all valid failure modes from different test environments.
+    }
+    assertRegionProbeCount(1);
+  }
+
+  @Test
+  public void testUnknownBucket() throws Exception {
+    describe("Verify unknown bucket probe failure");
+    Configuration conf = getConfiguration();
+    removeBaseAndBucketOverrides(UNKNOWN_BUCKET, conf, AWS_REGION, PATH_STYLE_ACCESS);
 
 
-    fs.getS3AInternals().getBucketMetadata();
+    newFS = new S3AFileSystem();
 
 
-    Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
-        .describedAs("Region is not configured, region probe should have been made").isEqualTo(1);
+    try {
+      newFS.initialize(new URI("s3a://" + UNKNOWN_BUCKET), conf);
+      newFS.getS3AInternals().getBucketMetadata();
+      // expect a failure by here
+      fail("Expected failure, got " + newFS);
+    } catch (UnknownHostException | UnknownStoreException expected) {
+      // this is good.
+    }
+    assertRegionProbeCount(1);
   }
   }
 
 
 
 
   @Test
   @Test
   public void testWithRegionConfig() throws IOException, URISyntaxException {
   public void testWithRegionConfig() throws IOException, URISyntaxException {
+    describe("Verify that region lookup is skipped if the region property is set");
     Configuration conf = getConfiguration();
     Configuration conf = getConfiguration();
-    conf.set(AWS_REGION, "us-east-2");
+    removeBaseAndBucketOverrides(conf, AWS_REGION, PATH_STYLE_ACCESS);
 
 
-    S3AFileSystem fs = new S3AFileSystem();
-    fs.initialize(new URI("s3a://landsat-pds"), conf);
+    conf.set(AWS_REGION, "us-east-2");
 
 
-    Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
-        .describedAs("Region is configured, region probe should not have been made").isEqualTo(0);
+    newFS = new S3AFileSystem();
+    newFS.initialize(new URI(USW_2_STORE), conf);
+    assertRegionProbeCount(0);
   }
   }
 
 
   @Test
   @Test
   public void testRegionCache() throws IOException, URISyntaxException {
   public void testRegionCache() throws IOException, URISyntaxException {
+    describe("Verify that region lookup is cached on the second attempt");
     Configuration conf = getConfiguration();
     Configuration conf = getConfiguration();
-    conf.unset(AWS_REGION);
-    conf.unset("fs.s3a.bucket.landsat-pds.endpoint.region");
-    S3AFileSystem fs = new S3AFileSystem();
+    removeBaseAndBucketOverrides(USW_2_BUCKET, conf, AWS_REGION, PATH_STYLE_ACCESS);
+
+    newFS = new S3AFileSystem();
 
 
-    fs.initialize(new URI("s3a://landsat-pds"), conf);
+    newFS.initialize(new URI(USW_2_STORE), conf);
 
 
-    Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
-        .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(1);
+    assertRegionProbeCount(1);
+    closeStream(newFS);
 
 
-    fs.initialize(new URI("s3a://landsat-pds"), conf);
+    // create a new instance
+    newFS = new S3AFileSystem();
+    newFS.initialize(new URI(USW_2_STORE), conf);
 
 
     // value should already be cached.
     // value should already be cached.
-    Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
-        .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(0);
+    assertRegionProbeCount(0);
+  }
+
+  private void assertRegionProbeCount(final int expected) {
+    Assertions.assertThat(newFS.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
+        .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(expected);
   }
   }
 
 
   @Test
   @Test

+ 19 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java

@@ -45,6 +45,7 @@ import java.util.stream.Collectors;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.isBulkDeleteEnabled;
 import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf;
 import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
 import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
@@ -89,7 +90,22 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
     S3AFileSystem fs =  getFileSystem();
     S3AFileSystem fs =  getFileSystem();
     Path path = path("largeDir");
     Path path = path("largeDir");
     mkdirs(path);
     mkdirs(path);
-    createFiles(fs, path, 1, 1005, 0);
+    final boolean bulkDeleteEnabled = isBulkDeleteEnabled(getFileSystem());
+
+    // with single object delete, only create a few files for a faster
+    // test run.
+    int filesToCreate;
+    int pages;
+    if (bulkDeleteEnabled) {
+      filesToCreate = 1005;
+      pages = 5;
+    } else {
+      filesToCreate = 250;
+      pages = 0;
+    }
+
+
+    createFiles(fs, path, 1, filesToCreate, 0);
     RemoteIterator<LocatedFileStatus> locatedFileStatusRemoteIterator =
     RemoteIterator<LocatedFileStatus> locatedFileStatusRemoteIterator =
             fs.listFiles(path, false);
             fs.listFiles(path, false);
     List<String> keys  = toList(mappingRemoteIterator(locatedFileStatusRemoteIterator,
     List<String> keys  = toList(mappingRemoteIterator(locatedFileStatusRemoteIterator,
@@ -102,9 +118,10 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
     }
     }
     Long bulkDeleteReqAfter = getNumberOfBulkDeleteRequestsMadeTillNow(fs);
     Long bulkDeleteReqAfter = getNumberOfBulkDeleteRequestsMadeTillNow(fs);
     // number of delete requests is 5 as we have default page size of 250.
     // number of delete requests is 5 as we have default page size of 250.
+
     Assertions.assertThat(bulkDeleteReqAfter - bulkDeleteReqBefore)
     Assertions.assertThat(bulkDeleteReqAfter - bulkDeleteReqBefore)
             .describedAs("Number of batched bulk delete requests")
             .describedAs("Number of batched bulk delete requests")
-            .isEqualTo(5);
+            .isEqualTo(pages);
   }
   }
 
 
   private Long getNumberOfBulkDeleteRequestsMadeTillNow(S3AFileSystem fs) {
   private Long getNumberOfBulkDeleteRequestsMadeTillNow(S3AFileSystem fs) {

+ 3 - 9
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -182,14 +182,8 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
    * non-null.
    * non-null.
    */
    */
   private void assumeNoDefaultEncryption() throws IOException {
   private void assumeNoDefaultEncryption() throws IOException {
-    try {
-      skipIfClientSideEncryption();
-      Assume.assumeThat(getDefaultEncryption(), nullValue());
-    } catch (AccessDeniedException e) {
-      // if the user can't check the default encryption, assume that it is
-      // null and keep going
-      LOG.warn("User does not have permission to call getBucketEncryption()");
-    }
+    skipIfClientSideEncryption();
+    Assume.assumeThat(getDefaultEncryption(), nullValue());
   }
   }
 
 
   /**
   /**
@@ -418,7 +412,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
           () -> s3.getBucketEncryption(GetBucketEncryptionRequest.builder()
           () -> s3.getBucketEncryption(GetBucketEncryptionRequest.builder()
               .bucket(fs.getBucket())
               .bucket(fs.getBucket())
               .build()));
               .build()));
-    } catch (FileNotFoundException e) {
+    } catch (FileNotFoundException | AccessDeniedException | AWSBadRequestException e) {
       return null;
       return null;
     }
     }
   }
   }

+ 11 - 9
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java

@@ -18,18 +18,19 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
 import software.amazon.awssdk.services.s3.model.MultipartUpload;
 import software.amazon.awssdk.services.s3.model.MultipartUpload;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 
-import org.junit.Test;
-
-
 import java.io.IOException;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.Set;
 
 
+import static org.apache.hadoop.util.functional.RemoteIterators.foreach;
 
 
 /**
 /**
  * Tests for {@link MultipartUtils}.
  * Tests for {@link MultipartUtils}.
@@ -101,17 +102,18 @@ public class ITestS3AMultipartUtils extends AbstractS3ATestBase {
 
 
     // Don't modify passed-in set, use copy.
     // Don't modify passed-in set, use copy.
     Set<MultipartTestUtils.IdKey> uploads = new HashSet<>(ourUploads);
     Set<MultipartTestUtils.IdKey> uploads = new HashSet<>(ourUploads);
-    while (list.hasNext()) {
-      MultipartTestUtils.IdKey listing = toIdKey(list.next());
-      if (uploads.contains(listing)) {
+    foreach(list, (upload) -> {
+      MultipartTestUtils.IdKey listing = toIdKey(upload);
+      if (uploads.remove(listing)) {
         LOG.debug("Matched: {},{}", listing.getKey(), listing.getUploadId());
         LOG.debug("Matched: {},{}", listing.getKey(), listing.getUploadId());
-        uploads.remove(listing);
       } else {
       } else {
         LOG.debug("Not our upload {},{}", listing.getKey(),
         LOG.debug("Not our upload {},{}", listing.getKey(),
             listing.getUploadId());
             listing.getUploadId());
       }
       }
-    }
-    assertTrue("Not all our uploads were listed", uploads.isEmpty());
+    });
+    Assertions.assertThat(uploads)
+        .describedAs("Uploads which we expected to be listed.")
+        .isEmpty();
   }
   }
 
 
   private MultipartTestUtils.IdKey toIdKey(MultipartUpload mu) {
   private MultipartTestUtils.IdKey toIdKey(MultipartUpload mu) {

+ 1 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java

@@ -74,6 +74,7 @@ public class ITestS3AStorageClass extends AbstractS3ATestBase {
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
+    skipIfStorageClassTestsDisabled(conf);
     disableFilesystemCaching(conf);
     disableFilesystemCaching(conf);
     removeBaseAndBucketOverrides(conf, STORAGE_CLASS, FAST_UPLOAD_BUFFER);
     removeBaseAndBucketOverrides(conf, STORAGE_CLASS, FAST_UPLOAD_BUFFER);
     conf.set(FAST_UPLOAD_BUFFER, fastUploadBufferType);
     conf.set(FAST_UPLOAD_BUFFER, fastUploadBufferType);
@@ -81,12 +82,6 @@ public class ITestS3AStorageClass extends AbstractS3ATestBase {
     return conf;
     return conf;
   }
   }
 
 
-  @Override
-  public void setup() throws Exception {
-    super.setup();
-    skipIfStorageClassTestsDisabled(getConfiguration());
-  }
-
   /*
   /*
    * This test ensures the default storage class configuration (no config or null)
    * This test ensures the default storage class configuration (no config or null)
    * works well with create and copy operations
    * works well with create and copy operations

+ 8 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java

@@ -199,5 +199,13 @@ public final class MultipartTestUtils {
     public int hashCode() {
     public int hashCode() {
       return Objects.hash(key, uploadId);
       return Objects.hash(key, uploadId);
     }
     }
+
+    @Override
+    public String toString() {
+      return "IdKey{" +
+          "key='" + key + '\'' +
+          ", uploadId='" + uploadId + '\'' +
+          '}';
+    }
   }
   }
 }
 }

+ 15 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java

@@ -58,6 +58,21 @@ public interface S3ATestConstants {
    */
    */
   String KEY_STORAGE_CLASS_TESTS_ENABLED = TEST_FS_S3A + "create.storage.class.enabled";
   String KEY_STORAGE_CLASS_TESTS_ENABLED = TEST_FS_S3A + "create.storage.class.enabled";
 
 
+  /**
+   * A property set to true if ACL tests are enabled: {@value}.
+   */
+  String KEY_ACL_TESTS_ENABLED = TEST_FS_S3A + "create.acl.enabled";
+
+  /**
+   * A property set to true if V1 tests are enabled: {@value}.
+   */
+  String KEY_LIST_V1_ENABLED = TEST_FS_S3A + "list.v1.enabled";
+
+  /**
+   * A property set to true if content encoding tests are enabled: {@value}.
+   */
+  String KEY_CONTENT_ENCODING_ENABLED = TEST_FS_S3A + "content.encoding.enabled";
+
   /**
   /**
    * Tell tests that they are being executed in parallel: {@value}.
    * Tell tests that they are being executed in parallel: {@value}.
    */
    */

+ 48 - 9
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -61,12 +61,13 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 import org.apache.hadoop.util.functional.FutureIO;
 import org.apache.hadoop.util.functional.FutureIO;
 
 
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Assume;
+import org.junit.AssumptionViolatedException;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 
 
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.File;
 import java.io.File;
@@ -466,8 +467,20 @@ public final class S3ATestUtils {
    */
    */
   public static void skipIfEncryptionTestsDisabled(
   public static void skipIfEncryptionTestsDisabled(
       Configuration configuration) {
       Configuration configuration) {
-    if (!configuration.getBoolean(KEY_ENCRYPTION_TESTS, true)) {
-      skip("Skipping encryption tests");
+    skipIfNotEnabled(configuration, KEY_ENCRYPTION_TESTS, "Skipping encryption tests");
+  }
+
+  /**
+   * Skip a test suite/casee if a configuration has been explicitly disabled.
+   * @param configuration configuration to probe
+   * @param key key to resolve
+   * @param message assertion text
+   */
+  public static void skipIfNotEnabled(final Configuration configuration,
+      final String key,
+      final String message) {
+    if (!configuration.getBoolean(key, true)) {
+      skip(message);
     }
     }
   }
   }
 
 
@@ -477,9 +490,18 @@ public final class S3ATestUtils {
    */
    */
   public static void skipIfStorageClassTestsDisabled(
   public static void skipIfStorageClassTestsDisabled(
       Configuration configuration) {
       Configuration configuration) {
-    if (!configuration.getBoolean(KEY_STORAGE_CLASS_TESTS_ENABLED, true)) {
-      skip("Skipping storage class tests");
-    }
+    skipIfNotEnabled(configuration, KEY_STORAGE_CLASS_TESTS_ENABLED,
+        "Skipping storage class tests");
+  }
+
+  /**
+   * Skip a test if ACL class tests are disabled.
+   * @param configuration configuration to probe
+   */
+  public static void skipIfACLTestsDisabled(
+      Configuration configuration) {
+    skipIfNotEnabled(configuration, KEY_ACL_TESTS_ENABLED,
+        "Skipping storage class ACL tests");
   }
   }
 
 
   /**
   /**
@@ -635,9 +657,7 @@ public final class S3ATestUtils {
    * @param conf configuration to examine
    * @param conf configuration to examine
    */
    */
   public static void assumeSessionTestsEnabled(final Configuration conf) {
   public static void assumeSessionTestsEnabled(final Configuration conf) {
-    if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
-      skip("STS functional tests disabled");
-    }
+    skipIfNotEnabled(conf, TEST_STS_ENABLED, "STS functional tests disabled");
   }
   }
 
 
   /**
   /**
@@ -1245,6 +1265,14 @@ public final class S3ATestUtils {
     Assume.assumeTrue(message, condition);
     Assume.assumeTrue(message, condition);
   }
   }
 
 
+  /**
+   * Convert a throwable to an assumption failure.
+   * @param t thrown exception.
+   */
+  public static void raiseAsAssumption(Throwable t) {
+    throw new AssumptionViolatedException(t.toString(), t);
+  }
+
   /**
   /**
    * Get the statistics from a wrapped block output stream.
    * Get the statistics from a wrapped block output stream.
    * @param out output stream
    * @param out output stream
@@ -1515,4 +1543,15 @@ public final class S3ATestUtils {
   public static void disablePrefetching(Configuration conf) {
   public static void disablePrefetching(Configuration conf) {
     removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
     removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
   }
   }
+
+  /**
+   * Does this FS support multi object delete?
+   * @param fs filesystem
+   * @return true if multi-delete is enabled.
+   */
+
+  public static boolean isBulkDeleteEnabled(FileSystem fs) {
+    return fs.getConf().getBoolean(Constants.ENABLE_MULTI_DELETE,
+        true);
+  }
 }
 }

+ 2 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java

@@ -43,6 +43,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.verifyExceptionClass;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.verifyExceptionClass;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
 
 /**
 /**
@@ -104,7 +105,7 @@ public class TestInvoker extends Assert {
           .cause(new Local.ConnectTimeoutException("timeout"))
           .cause(new Local.ConnectTimeoutException("timeout"))
           .build();
           .build();
   private static final AwsServiceException BAD_REQUEST = serviceException(
   private static final AwsServiceException BAD_REQUEST = serviceException(
-      AWSBadRequestException.STATUS_CODE,
+      SC_400_BAD_REQUEST,
       "bad request");
       "bad request");
 
 
   @Before
   @Before

+ 41 - 8
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java

@@ -70,12 +70,15 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
   public void setup() throws Exception {
   public void setup() throws Exception {
     super.setup();
     super.setup();
     final S3AFileSystem fs = getFileSystem();
     final S3AFileSystem fs = getFileSystem();
-    regionName = determineRegion(fs.getBucket());
+    final Configuration conf = fs.getConf();
+    endpoint = conf.getTrimmed(Constants.ENDPOINT, Constants.CENTRAL_ENDPOINT);
+    LOG.info("Test endpoint is {}", endpoint);
+    regionName = conf.getTrimmed(Constants.AWS_REGION, "");
+    if (regionName.isEmpty()) {
+      regionName = determineRegion(fs.getBucket());
+    }
     LOG.info("Determined region name to be [{}] for bucket [{}]", regionName,
     LOG.info("Determined region name to be [{}] for bucket [{}]", regionName,
         fs.getBucket());
         fs.getBucket());
-    endpoint = fs.getConf()
-        .get(Constants.ENDPOINT, Constants.CENTRAL_ENDPOINT);
-    LOG.info("Test endpoint is {}", endpoint);
   }
   }
 
 
   @Test
   @Test
@@ -118,11 +121,14 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
           .isGreaterThan(invocationCount);
           .isGreaterThan(invocationCount);
 
 
       Assertions.assertThat(CustomSigner.lastStoreValue)
       Assertions.assertThat(CustomSigner.lastStoreValue)
-          .as("Store value should not be null").isNotNull();
+          .as("Store value should not be null in %s", CustomSigner.description())
+          .isNotNull();
       Assertions.assertThat(CustomSigner.lastStoreValue.conf)
       Assertions.assertThat(CustomSigner.lastStoreValue.conf)
-          .as("Configuration should not be null").isNotNull();
+          .as("Configuration should not be null  in %s", CustomSigner.description())
+          .isNotNull();
       Assertions.assertThat(CustomSigner.lastStoreValue.conf.get(TEST_ID_KEY))
       Assertions.assertThat(CustomSigner.lastStoreValue.conf.get(TEST_ID_KEY))
-          .as("Configuration TEST_KEY mismatch").isEqualTo(identifier);
+          .as("Configuration TEST_KEY mismatch in %s", CustomSigner.description())
+          .isEqualTo(identifier);
 
 
       return fs;
       return fs;
     });
     });
@@ -196,8 +202,9 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
       try {
       try {
         lastStoreValue = CustomSignerInitializer
         lastStoreValue = CustomSignerInitializer
             .getStoreValue(bucketName, UserGroupInformation.getCurrentUser());
             .getStoreValue(bucketName, UserGroupInformation.getCurrentUser());
+        LOG.info("Store value for bucket {} is {}", bucketName, lastStoreValue);
       } catch (IOException e) {
       } catch (IOException e) {
-        throw new RuntimeException("Failed to get current Ugi", e);
+        throw new RuntimeException("Failed to get current Ugi " + e, e);
       }
       }
       if (bucketName.equals("kms")) {
       if (bucketName.equals("kms")) {
         Aws4Signer realKMSSigner = Aws4Signer.create();
         Aws4Signer realKMSSigner = Aws4Signer.create();
@@ -247,6 +254,14 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
     public static int getInvocationCount() {
     public static int getInvocationCount() {
       return INVOCATION_COUNT.get();
       return INVOCATION_COUNT.get();
     }
     }
+
+    public static String description() {
+      return "CustomSigner{"
+          + "invocations=" + INVOCATION_COUNT.get()
+          + ", instantiations=" + INSTANTIATION_COUNT.get()
+          + ", lastStoreValue=" + lastStoreValue
+          + "}";
+    }
   }
   }
 
 
   @Private
   @Private
@@ -260,6 +275,7 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
         DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) {
         DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) {
       StoreKey storeKey = new StoreKey(bucketName, storeUgi);
       StoreKey storeKey = new StoreKey(bucketName, storeUgi);
       StoreValue storeValue = new StoreValue(storeConf, dtProvider);
       StoreValue storeValue = new StoreValue(storeConf, dtProvider);
+      LOG.info("Registering store {} with value {}", storeKey, storeValue);
       knownStores.put(storeKey, storeValue);
       knownStores.put(storeKey, storeValue);
     }
     }
 
 
@@ -267,6 +283,7 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
     public void unregisterStore(String bucketName, Configuration storeConf,
     public void unregisterStore(String bucketName, Configuration storeConf,
         DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) {
         DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) {
       StoreKey storeKey = new StoreKey(bucketName, storeUgi);
       StoreKey storeKey = new StoreKey(bucketName, storeUgi);
+      LOG.info("Unregistering store {}", storeKey);
       knownStores.remove(storeKey);
       knownStores.remove(storeKey);
     }
     }
 
 
@@ -302,6 +319,14 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
       public int hashCode() {
       public int hashCode() {
         return Objects.hash(bucketName, ugi);
         return Objects.hash(bucketName, ugi);
       }
       }
+
+      @Override
+      public String toString() {
+        return "StoreKey{" +
+            "bucketName='" + bucketName + '\'' +
+            ", ugi=" + ugi +
+            '}';
+      }
     }
     }
 
 
     static class StoreValue {
     static class StoreValue {
@@ -313,6 +338,14 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
         this.conf = conf;
         this.conf = conf;
         this.dtProvider = dtProvider;
         this.dtProvider = dtProvider;
       }
       }
+
+      @Override
+      public String toString() {
+        return "StoreValue{" +
+            "conf=" + conf +
+            ", dtProvider=" + dtProvider +
+            '}';
+      }
     }
     }
   }
   }
 }
 }

+ 19 - 12
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java

@@ -24,7 +24,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Objects;
-import java.util.concurrent.TimeUnit;
 
 
 import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
 import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
 import software.amazon.awssdk.core.signer.Signer;
 import software.amazon.awssdk.core.signer.Signer;
@@ -32,26 +31,27 @@ import software.amazon.awssdk.http.SdkHttpFullRequest;
 import software.amazon.awssdk.http.SdkHttpMethod;
 import software.amazon.awssdk.http.SdkHttpMethod;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.junit.Before;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
-import org.junit.rules.Timeout;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.auth.TestSignerManager.SignerInitializerForTest.StoreValue;
 import org.apache.hadoop.fs.s3a.auth.TestSignerManager.SignerInitializerForTest.StoreValue;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
+import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS;
 import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS;
+import static org.apache.hadoop.fs.s3a.auth.SignerFactory.S3_V2_SIGNER;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 
 /**
 /**
  * Tests for the SignerManager.
  * Tests for the SignerManager.
  */
  */
-public class TestSignerManager {
+public class TestSignerManager extends AbstractHadoopTestBase {
 
 
   private static final Text TEST_TOKEN_KIND = new Text("TestTokenKind");
   private static final Text TEST_TOKEN_KIND = new Text("TestTokenKind");
   private static final Text TEST_TOKEN_SERVICE = new Text("TestTokenService");
   private static final Text TEST_TOKEN_SERVICE = new Text("TestTokenService");
@@ -61,9 +61,6 @@ public class TestSignerManager {
   private static final String TESTUSER1 = "testuser1";
   private static final String TESTUSER1 = "testuser1";
   private static final String TESTUSER2 = "testuser2";
   private static final String TESTUSER2 = "testuser2";
 
 
-  @Rule public Timeout testTimeout = new Timeout(10_000L,
-      TimeUnit.MILLISECONDS);
-
   @Before
   @Before
   public void beforeTest() {
   public void beforeTest() {
     SignerForTest1.reset();
     SignerForTest1.reset();
@@ -95,11 +92,8 @@ public class TestSignerManager {
     // Make sure the config is respected.
     // Make sure the config is respected.
     signerManager.initCustomSigners();
     signerManager.initCustomSigners();
     // Simulate a call from the AWS SDK to create the signer.
     // Simulate a call from the AWS SDK to create the signer.
-    LambdaTestUtils.intercept(Exception.class,
+    intercept(InstantiationIOException.class,
         () -> SignerFactory.createSigner("testsignerUnregistered", null));
         () -> SignerFactory.createSigner("testsignerUnregistered", null));
-    // Expecting generic Exception.class to handle future implementation
-    // changes.
-    // For now, this is an NPE
   }
   }
 
 
   @Test
   @Test
@@ -588,4 +582,17 @@ public class TestSignerManager {
     return SdkHttpFullRequest.builder().host(host).protocol("https").method(SdkHttpMethod.GET)
     return SdkHttpFullRequest.builder().host(host).protocol("https").method(SdkHttpMethod.GET)
         .build();
         .build();
   }
   }
+
+  @Test
+  public void testV2SignerRejected() throws Throwable {
+    intercept(InstantiationIOException.class, "no longer supported",
+        () -> SignerFactory.createSigner(S3_V2_SIGNER, "key"));
+  }
+
+  @Test
+  public void testUnknownSignerRejected() throws Throwable {
+    intercept(InstantiationIOException.class, "unknownSigner",
+        () -> SignerFactory.createSigner("unknownSigner", "key"));
+  }
+
 }
 }

+ 16 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java

@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
+import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -39,8 +40,10 @@ import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.fs.s3a.commit.impl.CommitContext;
 import org.apache.hadoop.fs.s3a.commit.impl.CommitContext;
 import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations;
 import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations;
 import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles;
 import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 
 import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
 import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.extractXAttrLongValue;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.extractXAttrLongValue;
 
 
@@ -118,6 +121,19 @@ public class ITestS3AHugeMagicCommits extends AbstractSTestS3AHugeFiles {
     return magicOutputFile;
     return magicOutputFile;
   }
   }
 
 
+  @Test
+  public void test_000_CleanupPendingUploads() throws IOException {
+    describe("Cleanup any existing pending uploads");
+    final S3AFileSystem fs = getFileSystem();
+    final String key = fs.pathToKey(finalDirectory);
+    final AuditSpan span = fs.getAuditSpanSource().createSpan(
+        MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(),
+        key, null);
+    final int count = fs.createWriteOperationHelper(span)
+        .abortMultipartUploadsUnderPath(key + "/");
+    LOG.info("Aborted {} uploads under {}", count, key);
+  }
+
   @Override
   @Override
   public void test_030_postCreationAssertions() throws Throwable {
   public void test_030_postCreationAssertions() throws Throwable {
     describe("Committing file");
     describe("Committing file");

+ 4 - 8
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java

@@ -37,7 +37,6 @@ import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_OP_XATTR_LIST;
 import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_MAP;
 import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_MAP;
 import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_NAMED;
 import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_NAMED;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
-import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_APPLICATION_XML;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_X_DIRECTORY;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_X_DIRECTORY;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_LENGTH;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_LENGTH;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_TYPE;
 import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_TYPE;
@@ -74,15 +73,12 @@ public class ITestXAttrCost extends AbstractS3ACostTest {
             fs.listXAttrs(root),
             fs.listXAttrs(root),
         with(INVOCATION_OP_XATTR_LIST, GET_METADATA_ON_OBJECT));
         with(INVOCATION_OP_XATTR_LIST, GET_METADATA_ON_OBJECT));
 
 
-    // verify this contains all the standard markers,
-    // but not the magic marker header
+    // don't make any assertions on the headers entries
+    // as different S3 providers may have different headers
+    // and they may even change over time.
     Assertions.assertThat(headerList)
     Assertions.assertThat(headerList)
         .describedAs("Headers on root object")
         .describedAs("Headers on root object")
-        .containsOnly(
-            XA_CONTENT_LENGTH,
-            XA_CONTENT_TYPE);
-    assertHeaderEntry(xAttrs, XA_CONTENT_TYPE)
-        .isEqualTo(CONTENT_TYPE_APPLICATION_XML);
+        .hasSize(xAttrs.size());
   }
   }
 
 
   /**
   /**

+ 115 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java

@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.NoRouteToHostException;
+import java.net.UnknownHostException;
+
+import org.junit.Test;
+import software.amazon.awssdk.core.exception.SdkClientException;
+
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractNetworkException;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Unit tests related to the {@link ErrorTranslation} class.
+ */
+public class TestErrorTranslation extends AbstractHadoopTestBase {
+
+  /**
+   * Create an sdk exception with the given cause.
+   * @param message error message
+   * @param cause cause
+   * @return a new exception
+   */
+  private static SdkClientException sdkException(
+      String message,
+      Throwable cause) {
+    return SdkClientException.builder()
+        .message(message)
+        .cause(cause)
+        .build();
+  }
+
+  @Test
+  public void testUnknownHostExceptionExtraction() throws Throwable {
+    final SdkClientException thrown = sdkException("top",
+        sdkException("middle",
+            new UnknownHostException("bottom")));
+    final IOException ioe = intercept(UnknownHostException.class, "top",
+        () -> {
+          throw maybeExtractNetworkException("", thrown);
+        });
+
+    // the wrapped exception is the top level one: no stack traces have
+    // been lost
+    if (ioe.getCause() != thrown) {
+      throw new AssertionError("Cause of " + ioe + " is not " + thrown, thrown);
+    }
+
+  }
+
+  @Test
+  public void testNoRouteToHostExceptionExtraction() throws Throwable {
+    intercept(NoRouteToHostException.class, "top",
+        () -> {
+          throw maybeExtractNetworkException("p2",
+              sdkException("top",
+                  sdkException("middle",
+                      new NoRouteToHostException("bottom"))));
+        });
+  }
+
+  @Test
+  public void testConnectExceptionExtraction() throws Throwable {
+    intercept(ConnectException.class, "top",
+        () -> {
+          throw maybeExtractNetworkException("p1",
+              sdkException("top",
+                  sdkException("middle",
+                      new ConnectException("bottom"))));
+        });
+  }
+  @Test
+  public void testNoConstructorExtraction() throws Throwable {
+    intercept(PathIOException.class, NoConstructorIOE.MESSAGE,
+        () -> {
+          throw maybeExtractNetworkException("p1",
+              sdkException("top",
+                  sdkException("middle",
+                      new NoConstructorIOE())));
+        });
+  }
+
+
+  public static final class NoConstructorIOE extends IOException {
+
+    public static final String MESSAGE = "no-arg constructor";
+
+    public NoConstructorIOE() {
+      super(MESSAGE);
+    }
+  }
+
+}

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

@@ -32,7 +32,6 @@ import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
-import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.Statistic;
 import org.apache.hadoop.fs.s3a.Statistic;
@@ -135,9 +134,7 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
     setupCostValidator();
     setupCostValidator();
 
 
     // determine bulk delete settings
     // determine bulk delete settings
-    final Configuration fsConf = getFileSystem().getConf();
-    isBulkDelete = fsConf.getBoolean(Constants.ENABLE_MULTI_DELETE,
-        true);
+    isBulkDelete = isBulkDeleteEnabled(getFileSystem());
     deleteMarkerStatistic = isBulkDelete()
     deleteMarkerStatistic = isBulkDelete()
         ? OBJECT_BULK_DELETE_REQUEST
         ? OBJECT_BULK_DELETE_REQUEST
         : OBJECT_DELETE_REQUEST;
         : OBJECT_DELETE_REQUEST;

+ 10 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java

@@ -455,6 +455,7 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase {
 
 
     Path src = basePath;
     Path src = basePath;
     Path dest = new Path(methodPath(), "dest");
     Path dest = new Path(methodPath(), "dest");
+    getFileSystem().delete(dest, true);
     assertRenamed(src, dest);
     assertRenamed(src, dest);
 
 
     assertPathDoesNotExist("source", src);
     assertPathDoesNotExist("source", src);
@@ -610,15 +611,18 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase {
             RequestBody.fromString(content)));
             RequestBody.fromString(content)));
   }
   }
   /**
   /**
-   * Delete an object.
+   * Delete an object; exceptions are swallowed.
    * @param key key
    * @param key key
-   * @param content string
    */
    */
   private void deleteObject(final String key) throws Exception {
   private void deleteObject(final String key) throws Exception {
-    exec("DELETE " + key, () -> {
-      s3client.deleteObject(b -> b.bucket(bucket).key(key));
-      return "deleted " + key;
-    });
+    try {
+      exec("DELETE " + key, () -> {
+        s3client.deleteObject(b -> b.bucket(bucket).key(key));
+        return "deleted " + key;
+      });
+    } catch (IOException ignored) {
+
+    }
   }
   }
 
 
   /**
   /**

+ 10 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java

@@ -99,16 +99,17 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
     // still be there
     // still be there
     Path simpleFile = file(new Path(dir, "simple.txt"));
     Path simpleFile = file(new Path(dir, "simple.txt"));
 
 
-    boolean rawAndKeeping = !isDeleting();
-    boolean rawAndDeleting = isDeleting();
+    boolean keeping = !isDeleting();
+    boolean deleting = isDeleting();
+    boolean bulkDelete = isBulkDelete();
     verifyMetrics(() -> {
     verifyMetrics(() -> {
           fs.delete(simpleFile, false);
           fs.delete(simpleFile, false);
           return "after fs.delete(simpleFile) " + getMetricSummary();
           return "after fs.delete(simpleFile) " + getMetricSummary();
         },
         },
-        probe(rawAndKeeping, OBJECT_METADATA_REQUESTS,
+        probe(keeping, OBJECT_METADATA_REQUESTS,
             FILESTATUS_FILE_PROBE_H),
             FILESTATUS_FILE_PROBE_H),
         // if deleting markers, look for the parent too
         // if deleting markers, look for the parent too
-        probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
+        probe(deleting, OBJECT_METADATA_REQUESTS,
             FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
             FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
         with(OBJECT_LIST_REQUEST,
         with(OBJECT_LIST_REQUEST,
             FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
             FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
@@ -116,7 +117,9 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
         with(FILES_DELETED, 1),
         with(FILES_DELETED, 1),
 
 
         // a single DELETE call is made to delete the object
         // a single DELETE call is made to delete the object
-        with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
+        probe(bulkDelete, OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
+        probe(!bulkDelete, OBJECT_DELETE_REQUEST,
+            DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST),
 
 
         // keeping: create no parent dirs or delete parents
         // keeping: create no parent dirs or delete parents
         withWhenKeeping(DIRECTORIES_CREATED, 0),
         withWhenKeeping(DIRECTORIES_CREATED, 0),
@@ -127,7 +130,8 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
         // a bulk delete for all parents is issued.
         // a bulk delete for all parents is issued.
         // the number of objects in it depends on the depth of the tree;
         // the number of objects in it depends on the depth of the tree;
         // don't worry about that
         // don't worry about that
-        withWhenDeleting(OBJECT_BULK_DELETE_REQUEST, DELETE_MARKER_REQUEST)
+        probe(deleting && bulkDelete, OBJECT_BULK_DELETE_REQUEST,
+            DELETE_MARKER_REQUEST)
     );
     );
 
 
     // there is an empty dir for a parent
     // there is an empty dir for a parent

+ 3 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.fs.s3a.scale;
 package org.apache.hadoop.fs.s3a.scale;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSUnsupportedFeatureException;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
 import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
 
 import java.nio.file.AccessDeniedException;
 import java.nio.file.AccessDeniedException;
 
 
@@ -54,8 +54,7 @@ public class ITestS3AHugeFilesSSECDiskBlocks
   public void setup() throws Exception {
   public void setup() throws Exception {
     try {
     try {
       super.setup();
       super.setup();
-      skipIfEncryptionTestsDisabled(getConfiguration());
-    } catch (AccessDeniedException e) {
+    } catch (AccessDeniedException | AWSUnsupportedFeatureException e) {
       skip("Bucket does not allow " + S3AEncryptionMethods.SSE_C + " encryption method");
       skip("Bucket does not allow " + S3AEncryptionMethods.SSE_C + " encryption method");
     }
     }
   }
   }
@@ -67,7 +66,7 @@ public class ITestS3AHugeFilesSSECDiskBlocks
     removeBaseAndBucketOverrides(conf, S3_ENCRYPTION_KEY,
     removeBaseAndBucketOverrides(conf, S3_ENCRYPTION_KEY,
         S3_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_ALGORITHM,
         S3_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_ALGORITHM,
         SERVER_SIDE_ENCRYPTION_KEY);
         SERVER_SIDE_ENCRYPTION_KEY);
-    S3ATestUtils.disableFilesystemCaching(conf);
+    skipIfEncryptionTestsDisabled(conf);
     conf.set(Constants.S3_ENCRYPTION_ALGORITHM,
     conf.set(Constants.S3_ENCRYPTION_ALGORITHM,
         getSSEAlgorithm().getMethod());
         getSSEAlgorithm().getMethod());
     conf.set(Constants.S3_ENCRYPTION_KEY, KEY_1);
     conf.set(Constants.S3_ENCRYPTION_KEY, KEY_1);

+ 1 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java

@@ -46,15 +46,10 @@ public class ITestS3AHugeFilesStorageClass extends AbstractSTestS3AHugeFiles {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(ITestS3AHugeFilesStorageClass.class);
   private static final Logger LOG = LoggerFactory.getLogger(ITestS3AHugeFilesStorageClass.class);
 
 
-  @Override
-  public void setup() throws Exception {
-    super.setup();
-    skipIfStorageClassTestsDisabled(getConfiguration());
-  }
-
   @Override
   @Override
   protected Configuration createScaleConfiguration() {
   protected Configuration createScaleConfiguration() {
     Configuration conf = super.createScaleConfiguration();
     Configuration conf = super.createScaleConfiguration();
+    skipIfStorageClassTestsDisabled(conf);
     disableFilesystemCaching(conf);
     disableFilesystemCaching(conf);
     removeBaseAndBucketOverrides(conf, STORAGE_CLASS);
     removeBaseAndBucketOverrides(conf, STORAGE_CLASS);
 
 

+ 12 - 0
hadoop-tools/hadoop-aws/src/test/resources/core-site.xml

@@ -189,6 +189,18 @@
     <value>10s</value>
     <value>10s</value>
   </property>
   </property>
 
 
+  <property>
+    <name>fs.s3a.attempts.maximum</name>
+    <value>1</value>
+    <description>How many times should the SDK retry commands on (probably) transient errors.</description>
+  </property>
+
+  <property>
+    <name>fs.s3a.retry.limit</name>
+    <value>3</value>
+    <description>Fail fairly fast</description>
+  </property>
+
   <!--
   <!--
   To run these tests.
   To run these tests.
 
 

+ 6 - 2
hadoop-tools/hadoop-aws/src/test/resources/log4j.properties

@@ -64,10 +64,14 @@ log4j.logger.org.apache.hadoop.fs.s3a.SDKV2Upgrade=WARN
 
 
 # Log all HTTP requests made; includes S3 interaction. This may
 # Log all HTTP requests made; includes S3 interaction. This may
 # include sensitive information such as account IDs in HTTP headers.
 # include sensitive information such as account IDs in HTTP headers.
-#log4j.logger.com.amazonaws.request=DEBUG
+# log4j.logger.software.amazon.awssdk.request=DEBUG
 
 
 # Turn on low level HTTP protocol debugging
 # Turn on low level HTTP protocol debugging
-#log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG
+#log4j.logger.org.apache.http.wire=DEBUG
+
+# async client
+#log4j.logger.io.netty.handler.logging=DEBUG
+#log4j.logger.io.netty.handler.codec.http2.Http2FrameLogger=DEBUG
 
 
 log4j.logger.org.apache.hadoop.mapreduce.lib.output=DEBUG
 log4j.logger.org.apache.hadoop.mapreduce.lib.output=DEBUG
 log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO
 log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO

Certains fichiers n'ont pas été affichés car il y a eu trop de fichiers modifiés dans ce diff