Przeglądaj źródła

HADOOP-15141 Support IAM Assumed roles in S3A. Contributed by Steve Loughran.

Aaron Fabbri 7 lat temu
rodzic
commit
268ab4e027
18 zmienionych plików z 1471 dodań i 59 usunięć
  1. 59 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  2. 24 24
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
  3. 197 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AssumedRoleCredentialProvider.java
  4. 37 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  5. 5 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  6. 44 11
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  7. 2 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
  8. 595 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
  9. 48 7
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  10. 42 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
  11. 4 1
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
  12. 52 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCpAssumedRole.java
  13. 324 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAssumeRole.java
  14. 10 10
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
  15. 7 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
  16. 16 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  17. 2 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
  18. 3 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java

+ 59 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -955,6 +955,65 @@
   </description>
   </description>
 </property>
 </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.session.duration</name>
+  <value>30m</value>
+  <description>
+    Duration of assumed roles before a refresh is attempted.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.policy</name>
+  <value/>
+  <description>
+    JSON policy containing more restrictions to apply to the role.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint</name>
+  <value/>
+  <description>
+    AWS Simple Token Service Endpoint. If unset, uses the default endpoint.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </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>
 <property>
   <name>fs.s3a.connection.maximum</name>
   <name>fs.s3a.connection.maximum</name>
   <value>15</value>
   <value>15</value>

+ 24 - 24
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java

@@ -26,12 +26,16 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
+
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
+import java.util.stream.Collectors;
 
 
 /**
 /**
  * A list of providers.
  * A list of providers.
@@ -50,7 +54,8 @@ import java.util.List;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class AWSCredentialProviderList implements AWSCredentialsProvider {
+public class AWSCredentialProviderList implements AWSCredentialsProvider,
+    AutoCloseable {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
       AWSCredentialProviderList.class);
       AWSCredentialProviderList.class);
@@ -84,23 +89,6 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider {
     providers.add(p);
     providers.add(p);
   }
   }
 
 
-  /**
-   * Reuse the last provider?
-   * @param reuseLastProvider flag to indicate the last provider should
-   * be re-used
-   */
-  public void setReuseLastProvider(boolean reuseLastProvider) {
-    this.reuseLastProvider = reuseLastProvider;
-  }
-
-  /**
-   * query the {@link #reuseLastProvider} flag.
-   * @return the current flag state.
-   */
-  public boolean isReuseLastProvider() {
-    return reuseLastProvider;
-  }
-
   /**
   /**
    * Refresh all child entries.
    * Refresh all child entries.
    */
    */
@@ -178,12 +166,9 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider {
    * If there are no providers, "" is returned.
    * If there are no providers, "" is returned.
    */
    */
   public String listProviderNames() {
   public String listProviderNames() {
-    StringBuilder sb = new StringBuilder(providers.size() * 32);
-    for (AWSCredentialsProvider provider : providers) {
-      sb.append(provider.getClass().getSimpleName());
-      sb.append(' ');
-    }
-    return sb.toString();
+    return providers.stream()
+        .map(provider -> provider.getClass().getSimpleName() + ' ')
+        .collect(Collectors.joining());
   }
   }
 
 
   /**
   /**
@@ -196,4 +181,19 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider {
     return "AWSCredentialProviderList: " +
     return "AWSCredentialProviderList: " +
         StringUtils.join(providers, " ");
         StringUtils.join(providers, " ");
   }
   }
+
+  /**
+   * Close routine will close all providers in the list which implement
+   * {@code Closeable}.
+   * This matters because some providers start a background thread to
+   * refresh their secrets.
+   */
+  @Override
+  public void close() {
+    for(AWSCredentialsProvider p: providers) {
+      if (p instanceof Closeable) {
+        IOUtils.closeStream((Closeable)p);
+      }
+    }
+  }
 }
 }

+ 197 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AssumedRoleCredentialProvider.java

@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
+import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider;
+import static org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses;
+
+/**
+ * Support IAM Assumed roles by instantiating an instance of
+ * {@code STSAssumeRoleSessionCredentialsProvider} from configuration
+ * properties, including wiring up the inner authenticator, and,
+ * unless overridden, creating a session name from the current user.
+ */
+public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
+    Closeable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AssumedRoleCredentialProvider.class);
+  public static final String NAME
+      = "org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider";
+
+  static final String E_FORBIDDEN_PROVIDER =
+      "AssumedRoleCredentialProvider cannot be in "
+          + ASSUMED_ROLE_CREDENTIALS_PROVIDER;
+
+  public static final String E_NO_ROLE = "Unset property "
+      + ASSUMED_ROLE_ARN;
+
+  private final STSAssumeRoleSessionCredentialsProvider stsProvider;
+
+  private final String sessionName;
+
+  private final long duration;
+
+  private final String arn;
+
+  /**
+   * Instantiate.
+   * This calls {@link #getCredentials()} to fail fast on the inner
+   * role credential retrieval.
+   * @param conf configuration
+   * @throws IOException on IO problems and some parameter checking
+   * @throws IllegalArgumentException invalid parameters
+   * @throws AWSSecurityTokenServiceException problems getting credentials
+   */
+  public AssumedRoleCredentialProvider(Configuration conf) throws IOException {
+
+    arn = conf.getTrimmed(ASSUMED_ROLE_ARN, "");
+    if (StringUtils.isEmpty(arn)) {
+      throw new IOException(E_NO_ROLE);
+    }
+
+    // build up the base provider
+    Class<?>[] awsClasses = loadAWSProviderClasses(conf,
+        ASSUMED_ROLE_CREDENTIALS_PROVIDER,
+        SimpleAWSCredentialsProvider.class);
+    AWSCredentialProviderList credentials = new AWSCredentialProviderList();
+    for (Class<?> aClass : awsClasses) {
+      if (this.getClass().equals(aClass)) {
+        throw new IOException(E_FORBIDDEN_PROVIDER);
+      }
+      credentials.add(createAWSCredentialProvider(conf, aClass));
+    }
+
+    // then the STS binding
+    sessionName = conf.getTrimmed(ASSUMED_ROLE_SESSION_NAME,
+        buildSessionName());
+    duration = conf.getTimeDuration(ASSUMED_ROLE_SESSION_DURATION,
+        ASSUMED_ROLE_SESSION_DURATION_DEFAULT, TimeUnit.SECONDS);
+    String policy = conf.getTrimmed(ASSUMED_ROLE_POLICY, "");
+
+    LOG.info("{}", this);
+    STSAssumeRoleSessionCredentialsProvider.Builder builder
+        = new STSAssumeRoleSessionCredentialsProvider.Builder(arn, sessionName);
+    builder.withRoleSessionDurationSeconds((int) duration);
+    if (StringUtils.isNotEmpty(policy)) {
+      LOG.debug("Scope down policy {}", policy);
+      builder.withScopeDownPolicy(policy);
+    }
+    String epr = conf.get(ASSUMED_ROLE_STS_ENDPOINT, "");
+    if (StringUtils.isNotEmpty(epr)) {
+      LOG.debug("STS Endpoint: {}", epr);
+      builder.withServiceEndpoint(epr);
+    }
+    LOG.debug("Credentials to obtain role credentials: {}", credentials);
+    builder.withLongLivedCredentialsProvider(credentials);
+    stsProvider = builder.build();
+    // and force in a fail-fast check just to keep the stack traces less
+    // convoluted
+    getCredentials();
+  }
+
+  /**
+   * Get credentials.
+   * @return the credentials
+   * @throws AWSSecurityTokenServiceException if none could be obtained.
+   */
+  @Override
+  public AWSCredentials getCredentials() {
+    try {
+      return stsProvider.getCredentials();
+    } catch (AWSSecurityTokenServiceException e) {
+      LOG.error("Failed to get credentials for role {}",
+          arn, e);
+      throw e;
+    }
+  }
+
+  @Override
+  public void refresh() {
+    stsProvider.refresh();
+  }
+
+  /**
+   * Propagate the close() call to the inner stsProvider.
+   */
+  @Override
+  public void close() {
+    stsProvider.close();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AssumedRoleCredentialProvider{");
+    sb.append("role='").append(arn).append('\'');
+    sb.append(", session'").append(sessionName).append('\'');
+    sb.append(", duration=").append(duration);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Build the session name from the current user's shortname.
+   * @return a string for the session name.
+   * @throws IOException failure to get the current user
+   */
+  static String buildSessionName() throws IOException {
+    return sanitize(UserGroupInformation.getCurrentUser()
+        .getShortUserName());
+  }
+
+  /**
+   * Build a session name from the string, sanitizing it for the permitted
+   * characters.
+   * @param session source session
+   * @return a string for use in role requests.
+   */
+  @VisibleForTesting
+  static String sanitize(String session) {
+    StringBuilder r = new StringBuilder(session.length());
+    for (char c: session.toCharArray()) {
+      if ("abcdefghijklmnopqrstuvwxyz0123456789,.@-".contains(
+          Character.toString(c).toLowerCase(Locale.ENGLISH))) {
+        r.append(c);
+      } else {
+        r.append('-');
+      }
+    }
+    return r.toString();
+  }
+
+}

+ 37 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -64,6 +64,43 @@ public final class Constants {
   // session token for when using TemporaryAWSCredentialsProvider
   // session token for when using TemporaryAWSCredentialsProvider
   public static final String SESSION_TOKEN = "fs.s3a.session.token";
   public static final String SESSION_TOKEN = "fs.s3a.session.token";
 
 
+  /**
+   * AWS Role to request.
+   */
+  public static final String ASSUMED_ROLE_ARN =
+      "fs.s3a.assumed.role.arn";
+
+  /**
+   * Session name for the assumed role, must be valid characters according
+   * to the AWS APIs.
+   * If not set, one is generated from the current Hadoop/Kerberos username.
+   */
+  public static final String ASSUMED_ROLE_SESSION_NAME =
+      "fs.s3a.assumed.role.session.name";
+
+  /**
+   * Duration of assumed roles before a refresh is attempted.
+   */
+  public static final String ASSUMED_ROLE_SESSION_DURATION =
+      "fs.s3a.assumed.role.session.duration";
+
+  /** Simple Token Service Endpoint. If unset, uses the default endpoint. */
+  public static final String ASSUMED_ROLE_STS_ENDPOINT =
+      "fs.s3a.assumed.role.sts.endpoint";
+
+  public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "30m";
+
+  /** list of providers to authenticate for the assumed role. */
+  public static final String ASSUMED_ROLE_CREDENTIALS_PROVIDER =
+      "fs.s3a.assumed.role.credentials.provider";
+
+  /** JSON policy containing more restrictions to apply to the role. */
+  public static final String ASSUMED_ROLE_POLICY =
+      "fs.s3a.assumed.role.policy";
+
+  public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT =
+      SimpleAWSCredentialsProvider.NAME;
+
   // number of simultaneous connections to s3
   // number of simultaneous connections to s3
   public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
   public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
   public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;
   public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;

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

@@ -2226,10 +2226,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       }
       }
     } catch (AmazonServiceException e) {
     } catch (AmazonServiceException e) {
       if (e.getStatusCode() != 404) {
       if (e.getStatusCode() != 404) {
-        throw translateException("getFileStatus", key, e);
+        throw translateException("getFileStatus", path, e);
       }
       }
     } catch (AmazonClientException e) {
     } catch (AmazonClientException e) {
-      throw translateException("getFileStatus", key, e);
+      throw translateException("getFileStatus", path, e);
     }
     }
 
 
     LOG.debug("Not Found: {}", path);
     LOG.debug("Not Found: {}", path);
@@ -2835,7 +2835,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     sb.append(", metastore=").append(metadataStore);
     sb.append(", metastore=").append(metadataStore);
     sb.append(", authoritative=").append(allowAuthoritative);
     sb.append(", authoritative=").append(allowAuthoritative);
     sb.append(", useListV1=").append(useListV1);
     sb.append(", useListV1=").append(useListV1);
-    sb.append(", magicCommitter=").append(isMagicCommitEnabled());
+    if (committerIntegration != null) {
+      sb.append(", magicCommitter=").append(isMagicCommitEnabled());
+    }
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", statistics {")
     sb.append(", statistics {")

+ 44 - 11
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -57,6 +57,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Modifier;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
@@ -494,7 +495,8 @@ public final class S3AUtils {
   }
   }
 
 
   /**
   /**
-   * Create the AWS credentials from the providers and the URI.
+   * Create the AWS credentials from the providers, the URI and
+   * the key {@link Constants#AWS_CREDENTIALS_PROVIDER} in the configuration.
    * @param binding Binding URI, may contain user:pass login details
    * @param binding Binding URI, may contain user:pass login details
    * @param conf filesystem configuration
    * @param conf filesystem configuration
    * @return a credentials provider list
    * @return a credentials provider list
@@ -505,14 +507,8 @@ public final class S3AUtils {
       URI binding, Configuration conf) throws IOException {
       URI binding, Configuration conf) throws IOException {
     AWSCredentialProviderList credentials = new AWSCredentialProviderList();
     AWSCredentialProviderList credentials = new AWSCredentialProviderList();
 
 
-    Class<?>[] awsClasses;
-    try {
-      awsClasses = conf.getClasses(AWS_CREDENTIALS_PROVIDER);
-    } catch (RuntimeException e) {
-      Throwable c = e.getCause() != null ? e.getCause() : e;
-      throw new IOException("From option " + AWS_CREDENTIALS_PROVIDER +
-          ' ' + c, c);
-    }
+    Class<?>[] awsClasses = loadAWSProviderClasses(conf,
+        AWS_CREDENTIALS_PROVIDER);
     if (awsClasses.length == 0) {
     if (awsClasses.length == 0) {
       S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
       S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
       credentials.add(new BasicAWSCredentialsProvider(
       credentials.add(new BasicAWSCredentialsProvider(
@@ -530,6 +526,25 @@ public final class S3AUtils {
     return credentials;
     return credentials;
   }
   }
 
 
+  /**
+   * Load list of AWS credential provider/credential provider factory classes.
+   * @param conf configuration
+   * @param key key
+   * @param defaultValue list of default values
+   * @return the list of classes, possibly empty
+   * @throws IOException on a failure to load the list.
+   */
+  static Class<?>[] loadAWSProviderClasses(Configuration conf,
+      String key,
+      Class<?>... defaultValue) throws IOException {
+    try {
+      return conf.getClasses(key, defaultValue);
+    } catch (RuntimeException e) {
+      Throwable c = e.getCause() != null ? e.getCause() : e;
+      throw new IOException("From option " + key + ' ' + c, c);
+    }
+  }
+
   /**
   /**
    * Create an AWS credential provider from its class by using reflection.  The
    * Create an AWS credential provider from its class by using reflection.  The
    * class must implement one of the following means of construction, which are
    * class must implement one of the following means of construction, which are
@@ -551,7 +566,7 @@ public final class S3AUtils {
    */
    */
   static AWSCredentialsProvider createAWSCredentialProvider(
   static AWSCredentialsProvider createAWSCredentialProvider(
       Configuration conf, Class<?> credClass) throws IOException {
       Configuration conf, Class<?> credClass) throws IOException {
-    AWSCredentialsProvider credentials = null;
+    AWSCredentialsProvider credentials;
     String className = credClass.getName();
     String className = credClass.getName();
     if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
     if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
       throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
       throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
@@ -590,9 +605,27 @@ public final class S3AUtils {
           + "accepting Configuration, or a public factory method named "
           + "accepting Configuration, or a public factory method named "
           + "getInstance that accepts no arguments, or a public default "
           + "getInstance that accepts no arguments, or a public default "
           + "constructor.", className, AWS_CREDENTIALS_PROVIDER));
           + "constructor.", className, AWS_CREDENTIALS_PROVIDER));
+    } catch (InvocationTargetException e) {
+      Throwable targetException = e.getTargetException();
+      if (targetException == null) {
+        targetException =  e;
+      }
+      if (targetException instanceof IOException) {
+        throw (IOException) targetException;
+      } else if (targetException instanceof SdkBaseException) {
+        throw translateException("Instantiate " + className, "",
+            (SdkBaseException) targetException);
+      } else {
+        // supported constructor or factory method found, but the call failed
+        throw new IOException(className + " " + INSTANTIATION_EXCEPTION
+            + ": " + targetException,
+            targetException);
+      }
     } catch (ReflectiveOperationException | IllegalArgumentException e) {
     } catch (ReflectiveOperationException | IllegalArgumentException e) {
       // supported constructor or factory method found, but the call failed
       // supported constructor or factory method found, but the call failed
-      throw new IOException(className + " " + INSTANTIATION_EXCEPTION +".", e);
+      throw new IOException(className + " " + INSTANTIATION_EXCEPTION
+          + ": " + e,
+          e);
     }
     }
   }
   }
 
 

+ 2 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java

@@ -88,7 +88,7 @@ public final class S3xLoginHelper {
   /**
   /**
    * Extract the login details from a URI, logging a warning if
    * Extract the login details from a URI, logging a warning if
    * the URI contains these.
    * the URI contains these.
-   * @param name URI of the filesystem
+   * @param name URI of the filesystem, can be null
    * @return a login tuple, possibly empty.
    * @return a login tuple, possibly empty.
    */
    */
   public static Login extractLoginDetailsWithWarnings(URI name) {
   public static Login extractLoginDetailsWithWarnings(URI name) {
@@ -101,7 +101,7 @@ public final class S3xLoginHelper {
 
 
   /**
   /**
    * Extract the login details from a URI.
    * Extract the login details from a URI.
-   * @param name URI of the filesystem
+   * @param name URI of the filesystem, may be null
    * @return a login tuple, possibly empty.
    * @return a login tuple, possibly empty.
    */
    */
   public static Login extractLoginDetails(URI name) {
   public static Login extractLoginDetails(URI name) {

+ 595 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md

@@ -0,0 +1,595 @@
+<!---
+  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 IAM Assumed Roles
+
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+AWS ["IAM Assumed Roles"](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles.html)
+allows applications to change the AWS role with which to authenticate with AWS services.
+The assumed roles can have different rights from the main user login.
+
+The S3A connector supports assumed roles for authentication with AWS.
+A full set of login credentials must be provided, which will be used
+to obtain the assumed role and refresh it regularly.
+By using per-filesystem configuration, it is possible to use different
+assumed roles for different buckets.
+
+## Using IAM Assumed Roles
+
+### Before You Begin
+
+This document assumes you know about IAM Assumed roles, what they
+are, how to configure their policies, etc.
+
+* You need a role to assume, and know its "ARN".
+* You need a pair of long-lived IAM User credentials, not the root account set.
+* Have the AWS CLI installed, and test that it works there.
+* Give the role access to S3, and, if using S3Guard, to DynamoDB.
+
+
+Trying to learn how IAM Assumed Roles work by debugging stack traces from
+the S3A client is "suboptimal".
+
+### <a name="how_it_works"></a> How the S3A connector support IAM Assumed Roles.
+
+To use assumed roles, the client must be configured to use the
+*Assumed Role Credential Provider*, `org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider`,
+in the configuration option `fs.s3a.aws.credentials.provider`.
+
+This AWS Credential provider will read in the `fs.s3a.assumed.role` options needed to connect to the
+Session Token Service [Assumed Role API](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html),
+first authenticating with the full credentials, then assuming the specific role
+specified. It will then refresh this login at the configured rate of
+`fs.s3a.assumed.role.session.duration`
+
+To authenticate with the STS service both for the initial credential retrieval
+and for background refreshes, a different credential provider must be
+created, one which uses long-lived credentials (secret keys, environment variables).
+Short lived credentials (e.g other session tokens, EC2 instance credentials) cannot be used.
+
+A list of providers can be set in `s.s3a.assumed.role.credentials.provider`;
+if unset the standard `BasicAWSCredentialsProvider` credential provider is used,
+which uses `fs.s3a.access.key` and `fs.s3a.secret.key`.
+
+Note: although you can list other AWS credential providers in  to the
+Assumed Role Credential Provider, it can only cause confusion.
+
+### <a name="using"></a> Using Assumed Roles
+
+To use assumed roles, the S3A client credentials provider must be set to
+the `AssumedRoleCredentialProvider`, and `fs.s3a.assumed.role.arn` to
+the previously created ARN.
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider</value>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.arn</name>
+  <value>arn:aws:iam::90066806600238:role/s3-restricted</value>
+</property>
+```
+
+The STS service itself needs the caller to be authenticated, *which can
+only be done with a set of long-lived credentials*.
+This means the normal `fs.s3a.access.key` and `fs.s3a.secret.key`
+pair, environment variables, or some other supplier of long-lived secrets.
+
+The default is the `fs.s3a.access.key` and `fs.s3a.secret.key` pair.
+If you wish to use a different authentication mechanism, set it in the property
+`fs.s3a.assumed.role.credentials.provider`.
+
+```xml
+<property>
+  <name>fs.s3a.assumed.role.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+</property>
+```
+
+Requirements for long-lived credentials notwithstanding, this option takes the
+same values as `fs.s3a.aws.credentials.provider`.
+
+The safest way to manage AWS secrets is via
+[Hadoop Credential Providers](index.html#hadoop_credential_providers).
+
+### <a name="configuration"></a>Assumed Role Configuration Options
+
+Here are the full set of configuration options.
+
+```xml
+<property>
+  <name>fs.s3a.assumed.role.arn</name>
+  <value />
+  <description>
+    AWS ARN for the role to be assumed.
+    Requires the fs.s3a.aws.credentials.provider list to contain
+    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.
+    If not set, one is generated from the current Hadoop/Kerberos username.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.session.duration</name>
+  <value>30m</value>
+  <description>
+    Duration of assumed roles before a refresh is attempted.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.policy</name>
+  <value/>
+  <description>
+    Extra policy containing more restrictions to apply to the role.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint</name>
+  <value/>
+  <description>
+    AWS Simple Token Service Endpoint. If unset, uses the default endpoint.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.credentials.provider</name>
+  <value/>
+  <description>
+    Credential providers used to authenticate with the STS endpoint and retrieve
+    the role tokens.
+    If unset, uses "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider".
+  </description>
+</property>
+```
+
+## <a name="troubleshooting"></a> Troubleshooting Assumed Roles
+
+1. Make sure the role works and the user trying to enter it can do so from AWS
+the command line before trying to use the S3A client.
+1. Try to access the S3 bucket with reads and writes from the AWS CLI.
+1. Then, with the hadoop settings updated, try to read data from the `hadoop fs` CLI:
+`hadoop fs -ls -p s3a://bucket/`
+1. Then, with the hadoop CLI, try to create a new directory with a request such as
+`hadoop fs -mkdirs -p s3a://bucket/path/p1/`
+
+### <a name="no_role"></a>IOException: "Unset property fs.s3a.assumed.role.arn"
+
+The Assumed Role Credential Provider is enabled, but `fs.s3a.assumed.role.arn` is unset.
+
+```
+java.io.IOException: Unset property fs.s3a.assumed.role.arn
+  at org.apache.hadoop.fs.s3a.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"
+
+This can arise if the role ARN set in `fs.s3a.assumed.role.arn` is invalid
+or one to which the caller has no access.
+
+```
+java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider
+ on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+  Not authorized to perform sts:AssumeRole (Service: AWSSecurityTokenService; Status Code: 403;
+   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"
+
+You can't use assume a role with the root acount of an AWS account;
+you need to create a new user and give it the permission to change into
+the role.
+
+```
+java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider
+ on : 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):
+    No AWS Credentials provided by AssumedRoleCredentialProvider :
+    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 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> "Assume Role session duration should be in the range of 15min - 1Hr"
+
+The value of `fs.s3a.assumed.role.session.duration` is out of range.
+
+```
+java.lang.IllegalArgumentException: Assume Role session duration should be in the range of 15min - 1Hr
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$Builder.withRoleSessionDurationSeconds(STSAssumeRoleSessionCredentialsProvider.java:437)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:86)
+```
+
+
+### <a name="malformed_policy"></a> `MalformedPolicyDocumentException` "The policy is not in the valid JSON format"
+
+
+The policy set in `fs.s3a.assumed.role.policy` is not valid according to the
+AWS specification of Role Policies.
+
+```
+rg.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider on :
+ 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):
+   MalformedPolicyDocument: 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 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)
+  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)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
+  at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
+  at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
+  at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
+  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)
+  ... 19 more
+```
+
+### <a name="malformed_policy"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
+
+The policy set in `fs.s3a.assumed.role.policy` is not valid JSON.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider on :
+ com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException:
+  Syntax errors in policy. (Service: AWSSecurityTokenService;
+  Status Code: 400; Error Code: MalformedPolicyDocument;
+  Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45):
+  MalformedPolicyDocument: Syntax errors in policy.
+  (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument;
+  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)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
+  at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
+  at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
+  at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
+  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)
+  ... 19 more
+```
+
+### <a name="recursive_auth"></a> `IOException`: "AssumedRoleCredentialProvider cannot be in fs.s3a.assumed.role.credentials.provider"
+
+You can't use the Assumed Role Credential Provider as the provider in
+`fs.s3a.assumed.role.credentials.provider`.
+
+```
+java.io.IOException: AssumedRoleCredentialProvider cannot be in fs.s3a.assumed.role.credentials.provider
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:86)
+  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)
+  at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
+```
+
+### <a name="invalid_keypair"></a> `AWSBadRequestException`: "not a valid key=value pair"
+
+
+There's an space or other typo in the `fs.s3a.access.key` or `fs.s3a.secret.key` values used for the
+inner authentication which is breaking signature creation.
+
+```
+ org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider
+  on : 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; Request ID: c4a8841d-f556-11e7-99f9-af005a829416):IncompleteSignature:
+    '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 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)
+  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)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
+  at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
+  at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
+  at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
+  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)
+  ... 25 more
+```
+
+### <a name="invalid_token"></a> `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid"
+
+The credentials used to authenticate with the AWS Simple Token Service are invalid.
+
+```
+[ERROR] Failures:
+[ERROR] java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider on :
+ 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):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)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
+  at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
+  at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
+  at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:116)
+  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)
+  ... 25 more
+```
+
+### <a name="invalid_session"></a> `AWSSecurityTokenServiceExceptiond`: "Member must satisfy regular expression pattern: `[\w+=,.@-]*`"
+
+
+The session name, as set in `fs.s3a.assumed.role.session.name` must match the wildcard `[\w+=,.@-]*`.
+
+If the property is unset, it is extracted from the current username and then sanitized to
+match these constraints.
+If set explicitly, it must be valid.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider on
+    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;
+    Request ID: 7c437acb-f55d-11e7-9ad8-3b5e4f701c20):ValidationError:
+    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 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)
+  at org.apache.hadoop.fs.s3a.ITestAssumeRole.lambda$expectFileSystemFailure$0(ITestAssumeRole.java:70)
+  at org.apache.hadoop.fs.s3a.ITestAssumeRole.lambda$interceptC$1(ITestAssumeRole.java:84)
+  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:491)
+  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:377)
+  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:446)
+  at org.apache.hadoop.fs.s3a.ITestAssumeRole.interceptC(ITestAssumeRole.java:82)
+  at org.apache.hadoop.fs.s3a.ITestAssumeRole.expectFileSystemFailure(ITestAssumeRole.java:68)
+  at org.apache.hadoop.fs.s3a.ITestAssumeRole.testAssumeRoleBadSession(ITestAssumeRole.java:216)
+  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
+  at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
+  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
+  at java.lang.reflect.Method.invoke(Method.java:498)
+  at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
+  at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
+  at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
+  at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
+  at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
+  at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
+  at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
+  at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
+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)
+  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)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.doInvoke(AWSSecurityTokenServiceClient.java:1271)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.invoke(AWSSecurityTokenServiceClient.java:1247)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.executeAssumeRole(AWSSecurityTokenServiceClient.java:454)
+  at com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient.assumeRole(AWSSecurityTokenServiceClient.java:431)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.newSession(STSAssumeRoleSessionCredentialsProvider.java:321)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.access$000(STSAssumeRoleSessionCredentialsProvider.java:37)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:76)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider$1.call(STSAssumeRoleSessionCredentialsProvider.java:73)
+  at com.amazonaws.auth.RefreshableTask.refreshValue(RefreshableTask.java:256)
+  at com.amazonaws.auth.RefreshableTask.blockingRefresh(RefreshableTask.java:212)
+  at com.amazonaws.auth.RefreshableTask.getValue(RefreshableTask.java:153)
+  at com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.getCredentials(STSAssumeRoleSessionCredentialsProvider.java:299)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:135)
+  at org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:124)
+  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)
+  ... 26 more
+```

+ 48 - 7
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -29,6 +29,7 @@ See also:
 * [Troubleshooting](./troubleshooting_s3a.html)
 * [Troubleshooting](./troubleshooting_s3a.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)
 * [Testing](./testing.html)
 * [Testing](./testing.html)
 
 
 ##<a name="overview"></a> Overview
 ##<a name="overview"></a> Overview
@@ -490,16 +491,56 @@ This means that the default S3A authentication chain can be defined as
 </property>
 </property>
 ```
 ```
 
 
-### <a name="auth_security"></a> Protecting the AWS Credentials
+## <a name="auth_security"></a> Protecting the AWS Credentials
 
 
-To protect the access/secret keys from prying eyes, it is recommended that you
-use either IAM role-based authentication (such as EC2 instance profile) or
-the credential provider framework securely storing them and accessing them
-through configuration. The following describes using the latter for AWS
-credentials in the S3A FileSystem.
+It is critical that you never share or leak your AWS credentials.
+Loss of credentials can leak/lose all your data, run up large bills,
+and significantly damage your organisation.
 
 
+1. Never share your secrets.
 
 
-## <a name="credential_providers"></a>Storing secrets with Hadoop Credential Providers
+1. Never commit your secrets into an SCM repository.
+The [git secrets](https://github.com/awslabs/git-secrets) can help here.
+
+1. Avoid using s3a URLs which have key and secret in the URL. This
+is dangerous as the secrets leak into the logs.
+
+1. Never include AWS credentials in bug reports, files attached to them,
+or similar.
+
+1. If you use the `AWS_` environment variables,  your list of environment variables
+is equally sensitive.
+
+1. Never use root credentials.
+Use IAM user accounts, with each user/application having its own set of credentials.
+
+1. Use IAM permissions to restrict the permissions individual users and applications
+have. This is best done through roles, rather than configuring individual users.
+
+1. Avoid passing in secrets to Hadoop applications/commands on the command line.
+The command line of any launched program is visible to all users on a Unix system
+(via `ps`), and preserved in command histories.
+
+1. Explore using [IAM Assumed Roles](assumed_roles.html) for role-based permissions
+management: a specific S3A connection can be made with a different assumed role
+and permissions from the primary user account.
+
+1. Consider a workflow in which usera and applications are issued with short-lived
+session credentials, configuring S3A to use these through
+the `TemporaryAWSCredentialsProvider`.
+
+1. Have a secure process in place for cancelling and re-issuing credentials for
+users and applications. Test it regularly by using it to refresh credentials.
+
+When running in EC2, the IAM EC2 instance credential provider will automatically
+obtain the credentials needed to access AWS services in the role the EC2 VM
+was deployed as.
+This credential provider is enabled in S3A by default.
+
+The safest way to keep the AWS login keys a secret within Hadoop is to use
+Hadoop Credentials.
+
+## <a name="hadoop_credential_providers"></a>Storing secrets with Hadoop Credential Providers
 
 
 The Hadoop Credential Provider Framework allows secure "Credential Providers"
 The Hadoop Credential Provider Framework allows secure "Credential Providers"
 to keep secrets outside Hadoop configuration files, storing them in encrypted
 to keep secrets outside Hadoop configuration files, storing them in encrypted

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

@@ -1033,3 +1033,45 @@ There is an in-memory Metadata Store for testing.
 ```
 ```
 
 
 This is not for use in production.
 This is not for use in production.
+
+##<a name="assumed_roles"></a> Testing Assumed Roles
+
+Tests for the AWS Assumed Role credential provider require an assumed
+role to request.
+
+If this role is not set, the tests which require it will be skipped.
+
+To run the tests in `ITestAssumeRole`, you need:
+
+1. A role in your AWS account with the relevant access rights to
+the S3 buckets used in the tests, and ideally DynamoDB, for S3Guard.
+If your bucket is set up by default to use S3Guard, the role must have access
+to that service.
+
+1.  Your IAM User  to have the permissions to adopt that role.
+
+1. The role ARN must be set in `fs.s3a.assumed.role.arn`.
+
+```xml
+<property>
+  <name>fs.s3a.assumed.role.arn</name>
+  <value>arn:aws:kms:eu-west-1:00000000000:key/0000000-16c9-4832-a1a9-c8bbef25ec8b</value>
+</property>
+```
+
+The tests don't do much other than verify that basic file IO works with the role,
+and trigger various failures.
+
+You can also run the entire test suite in an assumed role, a more
+thorough test, by switching to the credentials provider.
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider</value>
+</property>
+```
+
+The usual credentials needed to log in to the bucket will be used, but now
+the credentials used to interact with S3 and DynamoDB will be temporary
+role credentials, rather than the full credentials.

+ 4 - 1
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

@@ -16,7 +16,7 @@
 
 
 <!-- MACRO{toc|fromDepth=0|toDepth=5} -->
 <!-- MACRO{toc|fromDepth=0|toDepth=5} -->
 
 
-##<a name="introduction"></a>  Introduction
+##<a name="introduction"></a> Introduction
 
 
 Common problems working with S3 are
 Common problems working with S3 are
 
 
@@ -35,6 +35,9 @@ includes `distcp` and the `hadoop fs` command.
 
 
 <!-- MACRO{toc|fromDepth=0|toDepth=2} -->
 <!-- MACRO{toc|fromDepth=0|toDepth=2} -->
 
 
+Troubleshooting IAM Assumed Roles is covered in its
+[specific documentation](assumed_roles.html#troubeshooting).
+
 ## <a name="classpath"></a> Classpath Setup
 ## <a name="classpath"></a> Classpath Setup
 
 
 Note that for security reasons, the S3A client does not provide much detail
 Note that for security reasons, the S3A client does not provide much detail

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

@@ -0,0 +1,52 @@
+/*
+ * 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.contract.s3a;
+
+import org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider;
+
+import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.authenticationContains;
+
+/**
+ * Run DistCP under an assumed role.
+ * This is skipped if the FS is already set to run under an assumed role,
+ * because it would duplicate that of the superclass.
+ */
+public class ITestS3AContractDistCpAssumedRole extends ITestS3AContractDistCp {
+
+  @Override
+  public void setup() throws Exception {
+
+    super.setup();
+    // check for the fs having assumed roles
+    assume("No ARN for role tests", !getAssumedRoleARN().isEmpty());
+    assume("Already running as an assumed role",
+        !authenticationContains(getFileSystem().getConf(),
+            AssumedRoleCredentialProvider.NAME));
+  }
+
+  /**
+   * Probe for an ARN for the test FS.
+   * @return any ARN for the (previous created) filesystem.
+   */
+  private String getAssumedRoleARN() {
+    return getFileSystem().getConf().getTrimmed(ASSUMED_ROLE_ARN, "");
+  }
+}

+ 324 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAssumeRole.java

@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.file.AccessDeniedException;
+import java.util.concurrent.Callable;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Tests use of assumed roles.
+ * Only run if an assumed role is provided.
+ */
+public class ITestAssumeRole extends AbstractS3ATestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestAssumeRole.class);
+
+  private static final String ARN_EXAMPLE
+      = "arn:aws:kms:eu-west-1:00000000000:key/" +
+      "0000000-16c9-4832-a1a9-c8bbef25ec8b";
+
+  private static final String E_BAD_ROLE
+      = "Not authorized to perform sts:AssumeRole";
+
+  /**
+   * This is AWS policy removes read access.
+   */
+  public static final String RESTRICTED_POLICY = "{\n"
+      + "   \"Version\": \"2012-10-17\",\n"
+      + "   \"Statement\": [{\n"
+      + "      \"Effect\": \"Deny\",\n"
+      + "      \"Action\": \"s3:ListObjects\",\n"
+      + "      \"Resource\": \"*\"\n"
+      + "    }\n"
+      + "   ]\n"
+      + "}";
+
+  private void assumeRoleTests() {
+    assume("No ARN for role tests", !getAssumedRoleARN().isEmpty());
+  }
+
+  private String getAssumedRoleARN() {
+    return getContract().getConf().getTrimmed(ASSUMED_ROLE_ARN, "");
+  }
+
+  /**
+   * Expect a filesystem to fail to instantiate.
+   * @param conf config to use
+   * @param clazz class of exception to expect
+   * @param text text in exception
+   * @param <E> type of exception as inferred from clazz
+   * @throws Exception if the exception was the wrong class
+   */
+  private <E extends Throwable> void expectFileSystemFailure(
+      Configuration conf,
+      Class<E> clazz,
+      String text) throws Exception {
+    interceptC(clazz,
+        text,
+        () -> new Path(getFileSystem().getUri()).getFileSystem(conf));
+  }
+
+  /**
+   * Experimental variant of intercept() which closes any Closeable
+   * returned.
+   */
+  private static <E extends Throwable> E interceptC(
+      Class<E> clazz, String text,
+      Callable<Closeable> eval)
+      throws Exception {
+
+    return intercept(clazz, text,
+        () -> {
+          try (Closeable c = eval.call()) {
+            return c.toString();
+          }
+        });
+  }
+
+  @Test
+  public void testCreateCredentialProvider() throws IOException {
+    assumeRoleTests();
+    describe("Create the credential provider");
+
+    String roleARN = getAssumedRoleARN();
+
+    Configuration conf = new Configuration(getContract().getConf());
+    conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
+    conf.set(ASSUMED_ROLE_ARN, roleARN);
+    conf.set(ASSUMED_ROLE_SESSION_NAME, "valid");
+    conf.set(ASSUMED_ROLE_SESSION_DURATION, "45m");
+    conf.set(ASSUMED_ROLE_POLICY, RESTRICTED_POLICY);
+    try (AssumedRoleCredentialProvider provider
+             = new AssumedRoleCredentialProvider(conf)) {
+      LOG.info("Provider is {}", provider);
+      AWSCredentials credentials = provider.getCredentials();
+      assertNotNull("Null credentials from " + provider, credentials);
+    }
+  }
+
+  @Test
+  public void testAssumeRoleCreateFS() throws IOException {
+    assumeRoleTests();
+    describe("Create an FS client with the role and do some basic IO");
+
+    String roleARN = getAssumedRoleARN();
+    Configuration conf = createAssumedRoleConfig(roleARN);
+    conf.set(ASSUMED_ROLE_SESSION_NAME, "valid");
+    conf.set(ASSUMED_ROLE_SESSION_DURATION, "45m");
+    Path path = new Path(getFileSystem().getUri());
+    LOG.info("Creating test FS and user {} with assumed role {}",
+        conf.get(ACCESS_KEY), roleARN);
+
+    try (FileSystem fs = path.getFileSystem(conf)) {
+      fs.getFileStatus(new Path("/"));
+      fs.mkdirs(path("testAssumeRoleFS"));
+    }
+  }
+
+  @Test
+  public void testAssumeRoleRestrictedPolicyFS() throws Exception {
+    assumeRoleTests();
+    describe("Restrict the policy for this session; verify that reads fail");
+
+    String roleARN = getAssumedRoleARN();
+    Configuration conf = createAssumedRoleConfig(roleARN);
+    conf.set(ASSUMED_ROLE_POLICY, RESTRICTED_POLICY);
+    Path path = new Path(getFileSystem().getUri());
+    try (FileSystem fs = path.getFileSystem(conf)) {
+      intercept(AccessDeniedException.class, "getFileStatus",
+          () -> fs.getFileStatus(new Path("/")));
+      intercept(AccessDeniedException.class, "getFileStatus",
+          () -> fs.listStatus(new Path("/")));
+      intercept(AccessDeniedException.class, "getFileStatus",
+          () -> fs.mkdirs(path("testAssumeRoleFS")));
+    }
+  }
+
+  @Test
+  public void testAssumeRoleFSBadARN() throws Exception {
+    assumeRoleTests();
+    describe("Attemnpt to create the FS with an invalid ARN");
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.set(ASSUMED_ROLE_ARN, ARN_EXAMPLE);
+    expectFileSystemFailure(conf, AccessDeniedException.class, E_BAD_ROLE);
+  }
+
+  @Test
+  public void testAssumeRoleNoARN() throws Exception {
+    assumeRoleTests();
+    describe("Attemnpt to create the FS with no ARN");
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.unset(ASSUMED_ROLE_ARN);
+    expectFileSystemFailure(conf,
+        IOException.class,
+        AssumedRoleCredentialProvider.E_NO_ROLE);
+  }
+
+  @Test
+  public void testAssumeRoleFSBadPolicy() throws Exception {
+    assumeRoleTests();
+    describe("Attemnpt to create the FS with malformed JSON");
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    // add some malformed JSON
+    conf.set(ASSUMED_ROLE_POLICY, "}");
+    expectFileSystemFailure(conf,
+        AWSBadRequestException.class,
+        "JSON");
+  }
+
+  @Test
+  public void testAssumeRoleFSBadPolicy2() throws Exception {
+    assumeRoleTests();
+    describe("Attemnpt to create the FS with valid but non-compliant JSON");
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    // add some invalid JSON
+    conf.set(ASSUMED_ROLE_POLICY, "{'json':'but not what AWS wants}");
+    expectFileSystemFailure(conf,
+        AWSBadRequestException.class,
+        "Syntax errors in policy");
+  }
+
+  @Test
+  public void testAssumeRoleCannotAuthAssumedRole() throws Exception {
+    assumeRoleTests();
+    describe("Assert that you can't use assumed roles to auth assumed roles");
+
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
+        AssumedRoleCredentialProvider.NAME);
+    expectFileSystemFailure(conf,
+        IOException.class,
+        AssumedRoleCredentialProvider.E_FORBIDDEN_PROVIDER);
+  }
+
+  @Test
+  public void testAssumeRoleBadInnerAuth() throws Exception {
+    assumeRoleTests();
+    describe("Try to authenticate with a keypair with spaces");
+
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
+        SimpleAWSCredentialsProvider.NAME);
+    conf.set(ACCESS_KEY, "not valid");
+    conf.set(SECRET_KEY, "not secret");
+    expectFileSystemFailure(conf, AWSBadRequestException.class, "not a valid " +
+        "key=value pair (missing equal-sign) in Authorization header");
+  }
+
+  @Test
+  public void testAssumeRoleBadInnerAuth2() throws Exception {
+    assumeRoleTests();
+    describe("Try to authenticate with an invalid keypair");
+
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER,
+        SimpleAWSCredentialsProvider.NAME);
+    conf.set(ACCESS_KEY, "notvalid");
+    conf.set(SECRET_KEY, "notsecret");
+    expectFileSystemFailure(conf, AccessDeniedException.class,
+        "The security token included in the request is invalid");
+  }
+
+  @Test
+  public void testAssumeRoleBadSession() throws Exception {
+    assumeRoleTests();
+    describe("Try to authenticate with an invalid session");
+
+    Configuration conf = createAssumedRoleConfig(getAssumedRoleARN());
+    conf.set(ASSUMED_ROLE_SESSION_NAME,
+        "Session Names cannot Hava Spaces!");
+    expectFileSystemFailure(conf, AWSBadRequestException.class,
+        "Member must satisfy regular expression pattern");
+  }
+
+  /**
+   * Create a config for an assumed role; it also disables FS caching.
+   * @param roleARN ARN of role
+   * @return the configuration
+   */
+  private Configuration createAssumedRoleConfig(String roleARN) {
+    Configuration conf = new Configuration(getContract().getConf());
+    conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
+    conf.set(ASSUMED_ROLE_ARN, roleARN);
+    disableFilesystemCaching(conf);
+    return conf;
+  }
+
+  @Test
+  public void testAssumedRoleCredentialProviderValidation() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ASSUMED_ROLE_ARN, "");
+    interceptC(IOException.class,
+        AssumedRoleCredentialProvider.E_NO_ROLE,
+        () -> new AssumedRoleCredentialProvider(conf));
+  }
+
+  @Test
+  public void testAssumedDuration() throws Throwable {
+    assumeRoleTests();
+    describe("Expect the constructor to fail if the session is to short");
+    Configuration conf = new Configuration();
+    conf.set(ASSUMED_ROLE_SESSION_DURATION, "30s");
+    interceptC(IllegalArgumentException.class, "",
+        () -> new AssumedRoleCredentialProvider(conf));
+  }
+
+  @Test
+  public void testAssumedInvalidRole() throws Throwable {
+    assumeRoleTests();
+    describe("Expect the constructor to fail if the role is invalid");
+    Configuration conf = new Configuration();
+    conf.set(ASSUMED_ROLE_ARN, ARN_EXAMPLE);
+    interceptC(AWSSecurityTokenServiceException.class,
+        E_BAD_ROLE,
+        () -> new AssumedRoleCredentialProvider(conf));
+  }
+
+  /**
+   * This is here to check up on the S3ATestUtils probes themselves.
+   * @see S3ATestUtils#authenticationContains(Configuration, String).
+   */
+  @Test
+  public void testauthenticationContainsProbes() {
+    Configuration conf = new Configuration(false);
+    assertFalse("found AssumedRoleCredentialProvider",
+        authenticationContains(conf, AssumedRoleCredentialProvider.NAME));
+
+    conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
+    assertTrue("didn't find AssumedRoleCredentialProvider",
+        authenticationContains(conf, AssumedRoleCredentialProvider.NAME));
+  }
+}

+ 10 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java

@@ -19,9 +19,10 @@
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Assume;
@@ -37,6 +38,7 @@ import java.net.URI;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
 import java.nio.file.AccessDeniedException;
 import java.nio.file.AccessDeniedException;
 
 
+import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState;
 
 
@@ -120,11 +122,12 @@ public class ITestS3ACredentialsInURL extends Assert {
 
 
   /**
   /**
    * Set up some invalid credentials, verify login is rejected.
    * Set up some invalid credentials, verify login is rejected.
-   * @throws Throwable
    */
    */
   @Test
   @Test
   public void testInvalidCredentialsFail() throws Throwable {
   public void testInvalidCredentialsFail() throws Throwable {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
+    // use the default credential provider chain
+    conf.unset(AWS_CREDENTIALS_PROVIDER);
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
     Assume.assumeNotNull(fsname);
     Assume.assumeNotNull(fsname);
     assumeS3GuardState(false, conf);
     assumeS3GuardState(false, conf);
@@ -132,14 +135,11 @@ public class ITestS3ACredentialsInURL extends Assert {
     URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
     URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
 
 
     conf.set(TEST_FS_S3A_NAME, testURI.toString());
     conf.set(TEST_FS_S3A_NAME, testURI.toString());
-    try {
-      fs = S3ATestUtils.createTestFileSystem(conf);
-      FileStatus status = fs.getFileStatus(new Path("/"));
-      fail("Expected an AccessDeniedException, got " + status);
-    } catch (AccessDeniedException e) {
-      // expected
-    }
-
+    LambdaTestUtils.intercept(AccessDeniedException.class,
+        () -> {
+          fs = S3ATestUtils.createTestFileSystem(conf);
+          return fs.getFileStatus(new Path("/"));
+        });
   }
   }
 
 
   private URI createUriWithEmbeddedSecrets(URI original,
   private URI createUriWithEmbeddedSecrets(URI original,

+ 7 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -186,4 +186,11 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         fs.getFileChecksum(f, HELLO.length * 2));
         fs.getFileChecksum(f, HELLO.length * 2));
   }
   }
 
 
+  @Test
+  public void testS3AToStringUnitialized() throws Throwable {
+    try(S3AFileSystem fs = new S3AFileSystem()) {
+      fs.toString();
+    }
+  }
+
 }
 }

+ 16 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -828,9 +828,24 @@ public final class S3ATestUtils {
    * Skip a test if the FS isn't marked as supporting magic commits.
    * Skip a test if the FS isn't marked as supporting magic commits.
    * @param fs filesystem
    * @param fs filesystem
    */
    */
-  public void assumeMagicCommitEnabled(S3AFileSystem fs) {
+  public static void assumeMagicCommitEnabled(S3AFileSystem fs) {
     assume("Magic commit option disabled on " + fs,
     assume("Magic commit option disabled on " + fs,
         fs.hasCapability(CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER));
         fs.hasCapability(CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER));
   }
   }
 
 
+  /**
+   * Probe for the configuration containing a specific credential provider.
+   * If the list is empty, there will be no match, even if the named provider
+   * is on the default list.
+   *
+   * @param conf configuration
+   * @param providerClassname provider class
+   * @return true if the configuration contains that classname.
+   */
+  public static boolean authenticationContains(Configuration conf,
+      String providerClassname) {
+    return conf.getTrimmedStringCollection(AWS_CREDENTIALS_PROVIDER)
+        .contains(providerClassname);
+  }
+
 }
 }

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

@@ -106,6 +106,8 @@ public class TestS3AAWSCredentialsProvider {
   public void testDefaultChain() throws Exception {
   public void testDefaultChain() throws Exception {
     URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
     URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
+    // use the default credential provider chain
+    conf.unset(AWS_CREDENTIALS_PROVIDER);
     AWSCredentialProviderList list1 = S3AUtils.createAWSCredentialProviderSet(
     AWSCredentialProviderList list1 = S3AUtils.createAWSCredentialProviderSet(
         uri1, conf);
         uri1, conf);
     AWSCredentialProviderList list2 = S3AUtils.createAWSCredentialProviderSet(
     AWSCredentialProviderList list2 = S3AUtils.createAWSCredentialProviderSet(

+ 3 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
 import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.net.ServerSocketUtil;
 
 
+import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
 import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
 import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
 import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory.DefaultDynamoDBClientFactory.getRegion;
 import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory.DefaultDynamoDBClientFactory.getRegion;
 
 
@@ -80,6 +81,8 @@ public class DynamoDBLocalClientFactory extends Configured
     startSingletonServer();
     startSingletonServer();
 
 
     final Configuration conf = getConf();
     final Configuration conf = getConf();
+    // use the default credential provider chain
+    conf.unset(AWS_CREDENTIALS_PROVIDER);
     final AWSCredentialsProvider credentials =
     final AWSCredentialsProvider credentials =
         createAWSCredentialProviderSet(null, conf);
         createAWSCredentialProviderSet(null, conf);
     final ClientConfiguration awsConf =
     final ClientConfiguration awsConf =