HADOOP-14556. S3A to support Delegation Tokens.
authorSteve Loughran <stevel@apache.org>
Mon, 14 Jan 2019 17:59:27 +0000 (17:59 +0000)
committerSteve Loughran <stevel@apache.org>
Mon, 14 Jan 2019 17:59:27 +0000 (17:59 +0000)
Contributed by Steve Loughran and Daryn Sharp.

100 files changed:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
hadoop-project/pom.xml
hadoop-tools/hadoop-aws/pom.xml
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3A.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AEncryptionMethods.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentials.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAwsCredentialsException.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RoleModel.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AWSPolicyProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDTService.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractS3ATokenIdentifier.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationConstants.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenIOException.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecrets.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenBinding.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenIdentifier.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenIdentifier.java [moved from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java with 52% similarity]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADtFetcher.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenIdentifier.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/package-info.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/package-info.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/DurationInfo.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_token_architecture.md [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_tokens.md [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java [deleted file]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestMarshalledCredentials.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationIT.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/CountInvocationsProvider.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/Csvout.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestRoleCredentials.java [moved from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java with 55% similarity]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ILoadTestSessionCredentials.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestDelegatedMRJob.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationInFileystem.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationTokens.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationTokens.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/MiniKerberizedHadoopCluster.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/NanoTimerStats.java [new file with mode: 0644]
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3AMiniYarnCluster.java
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/mapreduce/MockJob.java [new file with mode: 0644]

index d1e50d0..200a2d1 100644 (file)
@@ -396,8 +396,11 @@ public abstract class AbstractFileSystem {
       thatPort = this.getUriDefaultPort();
     }
     if (thisPort != thatPort) {
-      throw new InvalidPathException("Wrong FS: " + path + ", expected: "
-          + this.getUri());
+      throw new InvalidPathException("Wrong FS: " + path
+          + " and port=" + thatPort
+          + ", expected: "
+          + this.getUri()
+          + " with port=" + thisPort);
     }
   }
   
index 5a3d736..74631b5 100644 (file)
@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import java.util.Iterator;
 
@@ -37,9 +38,13 @@ public abstract class StorageStatistics {
    *
    * When adding new common statistic name constants, please make them unique.
    * By convention, they are implicitly unique:
-   *  - the name of the constants are uppercase, words separated by underscores.
-   *  - the value of the constants are lowercase of the constant names.
+   * <ul>
+   *   <li>the name of the constants are uppercase, words separated by
+   *   underscores.</li>
+   *   <li>the value of the constants are lowercase of the constant names.</li>
+   * </ul>
    */
+  @InterfaceStability.Evolving
   public interface CommonStatisticNames {
     // The following names are for file system operation invocations
     String OP_APPEND = "op_append";
@@ -49,6 +54,7 @@ public abstract class StorageStatistics {
     String OP_DELETE = "op_delete";
     String OP_EXISTS = "op_exists";
     String OP_GET_CONTENT_SUMMARY = "op_get_content_summary";
+    String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token";
     String OP_GET_FILE_CHECKSUM = "op_get_file_checksum";
     String OP_GET_FILE_STATUS = "op_get_file_status";
     String OP_GET_STATUS = "op_get_status";
index bda2010..73f2d10 100644 (file)
 
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
+  <value>
+    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
+    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+    com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
+    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
+  </value>
   <description>
     Comma-separated class names of credential provider classes which implement
     com.amazonaws.auth.AWSCredentialsProvider.
 
+    When S3A delegation tokens are not enabled, this list will be used
+    to directly authenticate with S3 and DynamoDB services.
+    When S3A Delegation tokens are enabled, depending upon the delegation
+    token binding it may be used
+    to communicate wih the STS endpoint to request session/role
+    credentials.
+
     These are loaded and queried in sequence for a valid set of credentials.
     Each listed class must implement one of the following means of
     construction, which are attempted in order:
-    1. a public constructor accepting java.net.URI and
+    * a public constructor accepting java.net.URI and
         org.apache.hadoop.conf.Configuration,
-    2. a public static method named getInstance that accepts no
+    * a public constructor accepting org.apache.hadoop.conf.Configuration,
+    * a public static method named getInstance that accepts no
        arguments and returns an instance of
        com.amazonaws.auth.AWSCredentialsProvider, or
-    3. a public default constructor.
+    * a public default constructor.
 
     Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
     anonymous access to a publicly accessible S3 bucket without any credentials.
 
     If unspecified, then the default list of credential provider classes,
     queried in sequence, is:
-    1. org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
+    * org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider: looks
+       for session login secrets in the Hadoop configuration.
+    * org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
        Uses the values of fs.s3a.access.key and fs.s3a.secret.key.
-    2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
+    * com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
         configuration of AWS access key ID and secret access key in
         environment variables named AWS_ACCESS_KEY_ID and
         AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
-    3. com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
+    * com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
         of instance profile credentials if running in an EC2 VM.
   </description>
 </property>
   <value>30m</value>
   <description>
     Duration of assumed roles before a refresh is attempted.
-    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    Used when session tokens are requested.
     Range: 15m to 1h
   </description>
 </property>
     AWS Security Token Service Endpoint.
     If unset, uses the default endpoint.
     Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    Used by the AssumedRoleCredentialProvider and in Session and Role delegation
+    tokens.
   </description>
 </property>
 
 <property>
   <name>fs.s3a.assumed.role.sts.endpoint.region</name>
-  <value>us-west-1</value>
+  <value></value>
   <description>
     AWS Security Token Service Endpoint's region;
     Needed if fs.s3a.assumed.role.sts.endpoint points to an endpoint
     other than the default one and the v4 signature is used.
-    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    Used by the AssumedRoleCredentialProvider and in Session and Role delegation
+    tokens.
   </description>
 </property>
 
 </property>
 
 <property>
+  <name>fs.s3a.delegation.tokens.enabled</name>
+  <value>false</value>
+  <description></description>
+</property>
+
+<property>
+  <name>fs.s3a.delegation.token.binding</name>
+  <value></value>
+  <description>
+    The name of a class to provide delegation tokens support in S3A.
+    If unset: delegation token support is disabled.
+
+    Note: for job submission to actually collect these tokens,
+    Kerberos must be enabled.
+
+    Options are:
+    org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding
+    org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding
+    and org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding
+  </description>
+</property>
+
+<property>
   <name>fs.s3a.connection.maximum</name>
   <value>15</value>
   <description>Controls the maximum number of simultaneous connections to S3.</description>
index cb706ed..07431d4 100644 (file)
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.contract;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -519,7 +520,8 @@ public abstract class AbstractContractGetFileStatusTest extends
       Path path,
       PathFilter filter) throws IOException {
     FileStatus[] result = getFileSystem().listStatus(path, filter);
-    assertEquals("length of listStatus(" + path + ", " + filter + " )",
+    assertEquals("length of listStatus(" + path + ", " + filter + " ) " +
+        Arrays.toString(result),
         expected, result.length);
     return result;
   }
index cbb5288..67df6da 100644 (file)
@@ -23,8 +23,11 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Optional;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeoutException;
@@ -646,6 +649,48 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Evaluate a closure and return the result, after verifying that it is
+   * not null.
+   * @param message message to use in assertion text if the result is null
+   * @param eval closure to evaluate
+   * @param <T> type of response
+   * @return the evaluated result
+   * @throws Exception on any problem
+   */
+  public static<T> T notNull(String message, Callable<T> eval)
+      throws Exception {
+    T t = eval.call();
+    Assert.assertNotNull(message, t);
+    return t;
+  }
+
+  /**
+   * Execute a closure as the given user.
+   * @param user user to invoke the closure as
+   * @param eval closure to evaluate
+   * @param <T> return type
+   * @return the result of calling the closure under the identity of the user.
+   * @throws IOException IO failure
+   * @throws InterruptedException interrupted operation.
+   */
+  public static<T> T doAs(UserGroupInformation user, Callable<T> eval)
+      throws IOException, InterruptedException {
+    return user.doAs(new PrivilegedOperation<>(eval));
+  }
+
+  /**
+   * Execute a closure as the given user.
+   * @param user user to invoke the closure as
+   * @param eval closure to evaluate
+   * @throws IOException IO failure
+   * @throws InterruptedException interrupted operation.
+   */
+  public static void doAs(UserGroupInformation user, VoidCallable eval)
+      throws IOException, InterruptedException {
+    user.doAs(new PrivilegedVoidOperation(eval));
+  }
+
+  /**
    * Returns {@code TimeoutException} on a timeout. If
    * there was a inner class passed in, includes it as the
    * inner failure.
@@ -812,4 +857,50 @@ public final class LambdaTestUtils {
     }
   }
 
+  /**
+   * A lambda-invoker for doAs use; invokes the callable provided
+   * in the constructor.
+   * @param <T> return type.
+   */
+  public static class PrivilegedOperation<T>
+      implements PrivilegedExceptionAction<T> {
+
+    private final Callable<T> callable;
+
+    /**
+     * Constructor.
+     * @param callable a non-null callable/closure.
+     */
+    public PrivilegedOperation(final Callable<T> callable) {
+      this.callable = Preconditions.checkNotNull(callable);
+    }
+
+    @Override
+    public T run() throws Exception {
+      return callable.call();
+    }
+  }
+
+  /**
+   * VoidCaller variant of {@link PrivilegedOperation}: converts
+   * a void-returning closure to an action which {@code doAs} can call.
+   */
+  public static class PrivilegedVoidOperation
+      implements PrivilegedExceptionAction<Void> {
+
+    private final Callable<Void> callable;
+
+    /**
+     * Constructor.
+     * @param callable a non-null callable/closure.
+     */
+    public PrivilegedVoidOperation(final VoidCallable callable) {
+      this.callable = new VoidCaller(callable);
+    }
+
+    @Override
+    public Void run() throws Exception {
+      return callable.call();
+    }
+  }
 }
index f164b62..31e2057 100644 (file)
@@ -42,6 +42,8 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ReservationId;
+
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -1529,7 +1531,10 @@ public class Job extends JobContextImpl implements JobContext, AutoCloseable {
     return getSharedCacheUploadPolicies(conf, false);
   }
 
-  private synchronized void connect()
+  /** Only for mocking via unit tests. */
+  @Private
+  @VisibleForTesting
+  synchronized void connect()
           throws IOException, InterruptedException, ClassNotFoundException {
     if (cluster == null) {
       cluster = 
@@ -1549,7 +1554,8 @@ public class Job extends JobContextImpl implements JobContext, AutoCloseable {
 
   /** Only for mocking via unit tests. */
   @Private
-  public JobSubmitter getJobSubmitter(FileSystem fs, 
+  @VisibleForTesting
+  JobSubmitter getJobSubmitter(FileSystem fs,
       ClientProtocol submitClient) throws IOException {
     return new JobSubmitter(fs, submitClient);
   }
index e2e288b..6e8481e 100644 (file)
           <version>${snakeyaml.version}</version>
         </dependency>
         <dependency>
+          <groupId>org.hamcrest</groupId>
+          <artifactId>hamcrest-library</artifactId>
+          <version>1.3</version>
+        </dependency>
+        <dependency>
           <groupId>org.assertj</groupId>
           <artifactId>assertj-core</artifactId>
           <version>3.8.0</version>
index f2612e0..017f00f 100644 (file)
       <artifactId>hadoop-minikdc</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
+      <scope>test</scope>
+    </dependency>
     <!-- Used to create SSL certs for a secure Keystore -->
     <dependency>
       <groupId>org.bouncycastle</groupId>
index f9052fa..542e6f4 100644 (file)
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -39,6 +40,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
+import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
 import org.apache.hadoop.io.IOUtils;
 
 /**
@@ -52,7 +54,8 @@ import org.apache.hadoop.io.IOUtils;
  *   an {@link AmazonClientException}, that is rethrown, rather than
  *   swallowed.</li>
  *   <li>Has some more diagnostics.</li>
- *   <li>On failure, the last AmazonClientException raised is rethrown.</li>
+ *   <li>On failure, the last "relevant" AmazonClientException raised is
+ *   rethrown; exceptions other than 'no credentials' have priority.</li>
  *   <li>Special handling of {@link AnonymousAWSCredentials}.</li>
  * </ol>
  */
@@ -79,6 +82,12 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   private final AtomicBoolean closed = new AtomicBoolean(false);
 
   /**
+   * The name, which is empty by default.
+   * Uses in the code assume if non empty there's a trailing space.
+   */
+  private String name = "";
+
+  /**
    * Empty instance. This is not ready to be used.
    */
   public AWSCredentialProviderList() {
@@ -94,6 +103,29 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   }
 
   /**
+   * Create with an initial list of providers.
+   * @param name name for error messages, may be ""
+   * @param providerArgs provider list.
+   */
+  public AWSCredentialProviderList(final String name,
+      final AWSCredentialsProvider... providerArgs) {
+    setName(name);
+    Collections.addAll(providers, providerArgs);
+  }
+
+  /**
+   * Set the name; adds a ": " if needed.
+   * @param name name to add, or "" for no name.
+   */
+  public void setName(final String name) {
+    if (!name.isEmpty() && !name.endsWith(": ")) {
+      this.name = name + ": ";
+    } else {
+      this.name = name;
+    }
+  }
+
+  /**
    * Add a new provider.
    * @param p provider
    */
@@ -102,6 +134,14 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   }
 
   /**
+   * Add all providers from another list to this one.
+   * @param other the other list.
+   */
+  public void addAll(AWSCredentialProviderList other) {
+    providers.addAll(other.providers);
+  }
+
+  /**
    * Refresh all child entries.
    */
   @Override
@@ -123,7 +163,7 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   public AWSCredentials getCredentials() {
     if (isClosed()) {
       LOG.warn(CREDENTIALS_REQUESTED_WHEN_CLOSED);
-      throw new NoAuthWithAWSException(
+      throw new NoAuthWithAWSException(name +
           CREDENTIALS_REQUESTED_WHEN_CLOSED);
     }
     checkNotEmpty();
@@ -135,6 +175,8 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
     for (AWSCredentialsProvider provider : providers) {
       try {
         AWSCredentials credentials = provider.getCredentials();
+        Preconditions.checkNotNull(credentials,
+            "Null credentials returned by %s", provider);
         if ((credentials.getAWSAccessKeyId() != null &&
             credentials.getAWSSecretKey() != null)
             || (credentials instanceof AnonymousAWSCredentials)) {
@@ -142,6 +184,18 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
           LOG.debug("Using credentials from {}", provider);
           return credentials;
         }
+      } catch (NoAwsCredentialsException e) {
+        // don't bother with the stack trace here as it is usually a
+        // minor detail.
+
+        // only update the last exception if it isn't set.
+        // Why so? Stops delegation token issues being lost on the fallback
+        // values.
+        if (lastException == null) {
+          lastException = e;
+        }
+        LOG.debug("No credentials from {}: {}",
+            provider, e.toString());
       } catch (AmazonClientException e) {
         lastException = e;
         LOG.debug("No credentials provided by {}: {}",
@@ -151,12 +205,16 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
 
     // no providers had any credentials. Rethrow the last exception
     // or create a new one.
-    String message = "No AWS Credentials provided by "
+    String message =  name +  "No AWS Credentials provided by "
         + listProviderNames();
     if (lastException != null) {
       message += ": " + lastException;
     }
-    throw new NoAuthWithAWSException(message, lastException);
+    if (lastException instanceof CredentialInitializationException) {
+      throw lastException;
+    } else {
+      throw new NoAuthWithAWSException(message, lastException);
+    }
   }
 
   /**
@@ -175,7 +233,7 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
    */
   public void checkNotEmpty() {
     if (providers.isEmpty()) {
-      throw new NoAuthWithAWSException(NO_AWS_CREDENTIAL_PROVIDERS);
+      throw new NoAuthWithAWSException(name + NO_AWS_CREDENTIAL_PROVIDERS);
     }
   }
 
@@ -198,8 +256,10 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
   @Override
   public String toString() {
     return "AWSCredentialProviderList[" +
+        name +
         "refcount= " + refCount.get() + ": [" +
-        StringUtils.join(providers, ", ") + ']';
+        StringUtils.join(providers, ", ") + ']'
+        + (lastProvider != null ? (" last provider: " + lastProvider) : "");
   }
 
   /**
@@ -265,4 +325,12 @@ public class AWSCredentialProviderList implements AWSCredentialsProvider,
       }
     }
   }
+
+  /**
+   * Get the size of this list.
+   * @return the number of providers in the list.
+   */
+  public int size() {
+    return providers.size();
+  }
 }
index 9a71f32..bdd3add 100644 (file)
@@ -51,7 +51,7 @@ public final class Constants {
   // s3 secret key
   public static final String SECRET_KEY = "fs.s3a.secret.key";
 
-  // aws credentials provider
+  // aws credentials providers
   public static final String AWS_CREDENTIALS_PROVIDER =
       "fs.s3a.aws.credentials.provider";
 
@@ -63,18 +63,20 @@ public final class Constants {
   public static final String S3A_SECURITY_CREDENTIAL_PROVIDER_PATH =
       "fs.s3a.security.credential.provider.path";
 
-  // session token for when using TemporaryAWSCredentialsProvider
+  /**
+   * session token for when using TemporaryAWSCredentialsProvider: : {@value}.
+   */
   public static final String SESSION_TOKEN = "fs.s3a.session.token";
 
   /**
-   * AWS Role to request.
+   * ARN of AWS Role to request: {@value}.
    */
   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.
+   * to the AWS APIs: {@value}.
    * If not set, one is generated from the current Hadoop/Kerberos username.
    */
   public static final String ASSUMED_ROLE_SESSION_NAME =
@@ -86,34 +88,50 @@ public final class Constants {
   public static final String ASSUMED_ROLE_SESSION_DURATION =
       "fs.s3a.assumed.role.session.duration";
 
-  /** Security Token Service Endpoint. If unset, uses the default endpoint. */
+  /**
+   * Security Token Service Endpoint: {@value}.
+   * If unset, uses the default endpoint.
+   */
   public static final String ASSUMED_ROLE_STS_ENDPOINT =
       "fs.s3a.assumed.role.sts.endpoint";
 
   /**
-   * Region for the STS endpoint; only relevant if the endpoint
-   * is set.
+   * Default endpoint for session tokens: {@value}.
+   * This is the central STS endpoint which, for v3 signing, can
+   * issue STS tokens for any region.
+   */
+  public static final String DEFAULT_ASSUMED_ROLE_STS_ENDPOINT = "";
+
+  /**
+   * Region for the STS endpoint; needed if the endpoint
+   * is set to anything other then the central one.: {@value}.
    */
   public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION =
       "fs.s3a.assumed.role.sts.endpoint.region";
 
   /**
    * Default value for the STS endpoint region; needed for
-   * v4 signing.
+   * v4 signing: {@value}.
    */
-  public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT =
-      "us-west-1";
+  public static final String ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT = "";
 
   /**
-   * Default duration of an assumed role.
+   * Default duration of an assumed role: {@value}.
    */
-  public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "30m";
+  public static final String ASSUMED_ROLE_SESSION_DURATION_DEFAULT = "1h";
 
-  /** list of providers to authenticate for the assumed role. */
+  /**
+   * List of providers to authenticate for the assumed role: {@value}.
+   */
   public static final String ASSUMED_ROLE_CREDENTIALS_PROVIDER =
       "fs.s3a.assumed.role.credentials.provider";
 
-  /** JSON policy containing the policy to apply to the role. */
+  /**
+   * JSON policy containing the policy to apply to the role: {@value}.
+   * This is not used for delegation tokens, which generate the policy
+   * automatically, and restrict it to the S3, KMS and S3Guard services
+   * needed.
+   */
   public static final String ASSUMED_ROLE_POLICY =
       "fs.s3a.assumed.role.policy";
 
@@ -320,7 +338,10 @@ public final class Constants {
   /** Prefix for S3A bucket-specific properties: {@value}. */
   public static final String FS_S3A_BUCKET_PREFIX = "fs.s3a.bucket.";
 
-  public static final int S3A_DEFAULT_PORT = -1;
+  /**
+   * Default port for this is 443: HTTPS.
+   */
+  public static final int S3A_DEFAULT_PORT = 443;
 
   public static final String USER_AGENT_PREFIX = "fs.s3a.user.agent.prefix";
 
index ade317f..3e9368d 100644 (file)
@@ -28,6 +28,9 @@ import com.amazonaws.services.s3.AmazonS3Client;
 import com.amazonaws.services.s3.S3ClientOptions;
 import org.slf4j.Logger;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 
@@ -39,6 +42,8 @@ import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
  * This which calls the AWS SDK to configure and create an
  * {@link AmazonS3Client} that communicates with the S3 service.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
 public class DefaultS3ClientFactory extends Configured
     implements S3ClientFactory {
 
@@ -47,9 +52,13 @@ public class DefaultS3ClientFactory extends Configured
   @Override
   public AmazonS3 createS3Client(URI name,
       final String bucket,
-      final AWSCredentialsProvider credentials) throws IOException {
+      final AWSCredentialsProvider credentials,
+      final String userAgentSuffix) throws IOException {
     Configuration conf = getConf();
     final ClientConfiguration awsConf = S3AUtils.createAwsConf(getConf(), bucket);
+    if (!StringUtils.isEmpty(userAgentSuffix)) {
+      awsConf.setUserAgentSuffix(userAgentSuffix);
+    }
     return configureAmazonS3Client(
         newAmazonS3Client(credentials, awsConf), conf);
   }
index 45912a0..68a69f3 100644 (file)
@@ -476,7 +476,7 @@ public class Invoker {
   };
 
   /**
-   * Log summary at info, full stack at debug.
+   * Log retries at debug.
    */
   public static final Retried LOG_EVENT = new Retried() {
     @Override
index d856d80..78643cc 100644 (file)
@@ -42,6 +42,16 @@ public class S3A extends DelegateToFileSystem{
 
   @Override
   public int getUriDefaultPort() {
-    return Constants.S3A_DEFAULT_PORT;
+    // return Constants.S3A_DEFAULT_PORT;
+    return super.getUriDefaultPort();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("S3A{");
+    sb.append("URI =").append(fsImpl.getUri());
+    sb.append("; fsImpl=").append(fsImpl);
+    sb.append('}');
+    return sb.toString();
   }
 }
index e718cd4..85a00b1 100644 (file)
@@ -25,27 +25,44 @@ import org.apache.commons.lang3.StringUtils;
 /**
  * This enum is to centralize the encryption methods and
  * the value required in the configuration.
+ *
+ * There's two enum values for the two client encryption mechanisms the AWS
+ * S3 SDK supports, even though these are not currently supported in S3A.
+ * This is to aid supporting CSE in some form in future, fundamental
+ * issues about file length of encrypted data notwithstanding.
+ *
  */
 public enum S3AEncryptionMethods {
 
-  SSE_S3("AES256"),
-  SSE_KMS("SSE-KMS"),
-  SSE_C("SSE-C"),
-  NONE("");
+  NONE("", false),
+  SSE_S3("AES256", true),
+  SSE_KMS("SSE-KMS", true),
+  SSE_C("SSE-C", true),
+  CSE_KMS("CSE-KMS", false),
+  CSE_CUSTOM("CSE-CUSTOM", false);
 
   static final String UNKNOWN_ALGORITHM
-      = "Unknown Server Side Encryption algorithm ";
+      = "Unknown encryption algorithm ";
 
   private String method;
+  private boolean serverSide;
 
-  S3AEncryptionMethods(String method) {
+  S3AEncryptionMethods(String method, final boolean serverSide) {
     this.method = method;
+    this.serverSide = serverSide;
   }
 
   public String getMethod() {
     return method;
   }
 
+  /**
+   * Flag to indicate this is a server-side encryption option.
+   * @return true if this is server side.
+   */
+  public boolean isServerSide() {
+    return serverSide;
+  }
 
   /**
    * Get the encryption mechanism from the value provided.
@@ -57,16 +74,12 @@ public enum S3AEncryptionMethods {
     if(StringUtils.isBlank(name)) {
       return NONE;
     }
-    switch(name) {
-    case "AES256":
-      return SSE_S3;
-    case "SSE-KMS":
-      return SSE_KMS;
-    case "SSE-C":
-      return SSE_C;
-    default:
-      throw new IOException(UNKNOWN_ALGORITHM + name);
+    for (S3AEncryptionMethods v : values()) {
+      if (v.getMethod().equals(name)) {
+        return v;
+      }
     }
+    throw new IOException(UNKNOWN_ALGORITHM + name);
   }
 
 }
index e6eab8a..eb055dc 100644 (file)
@@ -36,6 +36,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.Objects;
 import java.util.concurrent.ExecutorService;
@@ -101,6 +102,12 @@ import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
+import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations;
+import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
+import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
+import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.PutTracker;
 import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration;
@@ -114,6 +121,7 @@ import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.fs.store.EtagChecksum;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
@@ -122,8 +130,12 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
-import static org.apache.commons.lang3.StringUtils.isNotBlank;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW;
+import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
+import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
+import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 
 /**
  * The core S3A Filesystem implementation.
@@ -140,7 +152,8 @@ import static org.apache.commons.lang3.StringUtils.isNotEmpty;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AFileSystem extends FileSystem implements StreamCapabilities {
+public class S3AFileSystem extends FileSystem implements StreamCapabilities,
+    AWSPolicyProvider {
   /**
    * Default blocksize as used in blocksize and FS status queries.
    */
@@ -183,7 +196,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
   private LocalDirAllocator directoryAllocator;
   private CannedAccessControlList cannedACL;
-  private S3AEncryptionMethods serverSideEncryptionAlgorithm;
+
+  /**
+   * This must never be null; until initialized it just declares that there
+   * is no encryption.
+   */
+  private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
   private S3AInstrumentation instrumentation;
   private final S3AStorageStatistics storageStatistics =
       createStorageStatistics();
@@ -194,6 +212,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private MetadataStore metadataStore;
   private boolean allowAuthoritative;
 
+  /** Delegation token integration; non-empty when DT support is enabled. */
+  private Optional<S3ADelegationTokens> delegationTokens = Optional.empty();
+
+  /** Principal who created the FS; recorded during initialization. */
+  private UserGroupInformation owner;
+
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
   private String blockOutputBuffer;
@@ -234,32 +258,40 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   public void initialize(URI name, Configuration originalConf)
       throws IOException {
-    setUri(name);
     // get the host; this is guaranteed to be non-null, non-empty
     bucket = name.getHost();
     LOG.debug("Initializing S3AFileSystem for {}", bucket);
     // clone the configuration into one with propagated bucket options
     Configuration conf = propagateBucketOptions(originalConf, bucket);
+    // patch the Hadoop security providers
     patchSecurityCredentialProviders(conf);
-    super.initialize(name, conf);
+    // look for delegation token support early.
+    boolean delegationTokensEnabled = hasDelegationTokenBinding(conf);
+    if (delegationTokensEnabled) {
+      LOG.debug("Using delegation tokens");
+    }
+    // set the URI, this will do any fixup of the URI to remove secrets,
+    // canonicalize.
+    setUri(name, delegationTokensEnabled);
+    super.initialize(uri, conf);
     setConf(conf);
     try {
-      instrumentation = new S3AInstrumentation(name);
+
+      // look for encryption data
+      // DT Bindings may override this
+      setEncryptionSecrets(new EncryptionSecrets(
+          getEncryptionAlgorithm(bucket, conf),
+          getServerSideEncryptionKey(bucket, getConf())));
+
+      invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
+      instrumentation = new S3AInstrumentation(uri);
 
       // Username is the current user at the time the FS was instantiated.
-      username = UserGroupInformation.getCurrentUser().getShortUserName();
+      owner = UserGroupInformation.getCurrentUser();
+      username = owner.getShortUserName();
       workingDir = new Path("/user", username)
           .makeQualified(this.uri, this.getWorkingDirectory());
 
-
-      Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
-          S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
-          S3ClientFactory.class);
-
-      credentials = createAWSCredentialProviderSet(name, conf);
-      s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
-          .createS3Client(name, bucket, credentials);
-      invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
       s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
           onRetry);
       writeHelper = new WriteOperationHelper(this, getConf());
@@ -306,13 +338,18 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       }
       useListV1 = (listVersion == 1);
 
+      // creates the AWS client, including overriding auth chain if
+      // the FS came with a DT
+      // this may do some patching of the configuration (e.g. setting
+      // the encryption algorithms)
+      bindAWSClient(name, delegationTokensEnabled);
+
       initTransferManager();
 
       initCannedAcls(conf);
 
       verifyBucketExists();
 
-      serverSideEncryptionAlgorithm = getEncryptionAlgorithm(bucket, conf);
       inputPolicy = S3AInputPolicy.getPolicy(
           conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
       LOG.debug("Input fadvise policy = {}", inputPolicy);
@@ -391,6 +428,80 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     return instrumentation;
   }
 
+  /**
+   * Set up the client bindings.
+   * If delegation tokens are enabled, the FS first looks for a DT
+   * ahead of any other bindings;.
+   * If there is a DT it uses that to do the auth
+   * and switches to the DT authenticator automatically (and exclusively)
+   * @param name URI of the FS
+   * @param dtEnabled are delegation tokens enabled?
+   * @throws IOException failure.
+   */
+  private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
+    Configuration conf = getConf();
+    credentials = null;
+    String uaSuffix = "";
+
+    if (dtEnabled) {
+      // Delegation support.
+      // Create and start the DT integration.
+      // Then look for an existing DT for this bucket, switch to authenticating
+      // with it if so.
+
+      LOG.debug("Using delegation tokens");
+      S3ADelegationTokens tokens = new S3ADelegationTokens();
+      this.delegationTokens = Optional.of(tokens);
+      tokens.bindToFileSystem(getCanonicalUri(), this);
+      tokens.init(conf);
+      tokens.start();
+      // switch to the DT provider and bypass all other configured
+      // providers.
+      if (tokens.isBoundToDT()) {
+        // A DT was retrieved.
+        LOG.debug("Using existing delegation token");
+        // and use the encryption settings from that client, whatever they were
+      } else {
+        LOG.debug("No delegation token for this instance");
+      }
+      // Get new credential chain
+      credentials = tokens.getCredentialProviders();
+      // and any encryption secrets which came from a DT
+      tokens.getEncryptionSecrets()
+          .ifPresent(this::setEncryptionSecrets);
+      // and update the UA field with any diagnostics provided by
+      // the DT binding.
+      uaSuffix = tokens.getUserAgentField();
+    } else {
+      // DT support is disabled, so create the normal credential chain
+      credentials = createAWSCredentialProviderSet(name, conf);
+    }
+    LOG.debug("Using credential provider {}", credentials);
+    Class<? extends S3ClientFactory> s3ClientFactoryClass = conf.getClass(
+        S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
+        S3ClientFactory.class);
+
+    s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
+        .createS3Client(getUri(), bucket, credentials, uaSuffix);
+  }
+
+  /**
+   * Set the encryption secrets for requests.
+   * @param secrets secrets
+   */
+  protected void setEncryptionSecrets(final EncryptionSecrets secrets) {
+    this.encryptionSecrets = secrets;
+  }
+
+  /**
+   * Get the encryption secrets.
+   * This potentially sensitive information and must be treated with care.
+   * @return the current encryption secrets.
+   */
+  public EncryptionSecrets getEncryptionSecrets() {
+    return encryptionSecrets;
+  }
+
   private void initTransferManager() {
     TransferManagerConfiguration transferConfiguration =
         new TransferManagerConfiguration();
@@ -466,18 +577,30 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
-   * Set the URI field through {@link S3xLoginHelper}.
+   * Set the URI field through {@link S3xLoginHelper} and
+   * optionally {@link #canonicalizeUri(URI)}
    * Exported for testing.
-   * @param uri filesystem URI.
+   * @param fsUri filesystem URI.
+   * @param canonicalize true if the URI should be canonicalized.
    */
   @VisibleForTesting
-  protected void setUri(URI uri) {
-    this.uri = S3xLoginHelper.buildFSURI(uri);
+  protected void setUri(URI fsUri, boolean canonicalize) {
+    URI u = S3xLoginHelper.buildFSURI(fsUri);
+    this.uri = canonicalize ? u : canonicalizeUri(u);
   }
 
+  /**
+   * Get the canonical URI.
+   * @return the canonical URI of this FS.
+   */
+  public URI getCanonicalUri() {
+    return uri;
+  }
+
+  @VisibleForTesting
   @Override
   public int getDefaultPort() {
-    return Constants.S3A_DEFAULT_PORT;
+    return 0;
   }
 
   /**
@@ -558,7 +681,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @return the encryption algorithm.
    */
   public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
-    return serverSideEncryptionAlgorithm;
+    return encryptionSecrets.getEncryptionMethod();
   }
 
   /**
@@ -690,6 +813,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
   }
 
+  /**
+   * Override the base canonicalization logic and relay to
+   * {@link S3xLoginHelper#canonicalizeUri(URI, int)}.
+   * This allows for the option of changing this logic for better DT handling.
+   * @param rawUri raw URI.
+   * @return the canonical URI to use in delegation tokens and file context.
+   */
   @Override
   protected URI canonicalizeUri(URI rawUri) {
     return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
@@ -719,8 +849,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
             fileStatus),
             new S3ObjectAttributes(bucket,
                 pathToKey(f),
-                serverSideEncryptionAlgorithm,
-                getServerSideEncryptionKey(bucket, getConf())),
+                getServerSideEncryptionAlgorithm(),
+                encryptionSecrets.getEncryptionKey()),
             fileStatus.getLen(),
             s3,
             readAhead,
@@ -1092,9 +1222,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @throws IOException IO and object access problems.
    */
   @VisibleForTesting
-  @Retries.RetryRaw
+  @Retries.RetryTranslated
   public ObjectMetadata getObjectMetadata(Path path) throws IOException {
-    return getObjectMetadata(pathToKey(path));
+    return once("getObjectMetadata", path.toString(),
+        () ->
+          // this always does a full HEAD to the object
+          getObjectMetadata(pathToKey(path)));
+  }
+
+  /**
+   * Get all the headers of the object of a path, if the object exists.
+   * @param path path to probe
+   * @return an immutable map of object headers.
+   * @throws IOException failure of the query
+   */
+  @Retries.RetryTranslated
+  public Map<String, Object> getObjectHeaders(Path path) throws IOException {
+    LOG.debug("getObjectHeaders({})", path);
+    checkNotClosed();
+    incrementReadOperations();
+    return getObjectMetadata(path).getRawMetadata();
   }
 
   /**
@@ -1244,10 +1391,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     GetObjectMetadataRequest request =
         new GetObjectMetadataRequest(bucket, key);
     //SSE-C requires to be filled in if enabled for object metadata
-    if(S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
-        isNotBlank(getServerSideEncryptionKey(bucket, getConf()))){
-      request.setSSECustomerKey(generateSSECustomerKey());
-    }
+    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
     ObjectMetadata meta = invoker.retryUntranslated("GET " + key, true,
         () -> {
           incrementStatistic(OBJECT_METADATA_REQUESTS);
@@ -2013,6 +2157,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * Get the owner of this FS: who created it?
+   * @return the owner of the FS.
+   */
+  public UserGroupInformation getOwner() {
+    return owner;
+  }
+
+  /**
    *
    * Make the given path and all non-existent parents into
    * directories. Has the semantics of Unix {@code 'mkdir -p'}.
@@ -2508,6 +2660,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       metadataStore = null;
       instrumentation = null;
       closeAutocloseables(LOG, credentials);
+      cleanupWithLogger(LOG, delegationTokens.orElse(null));
       credentials = null;
     }
   }
@@ -2524,12 +2677,88 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
-   * Override getCanonicalServiceName because we don't support token in S3A.
+   * Get the delegation token support for this filesystem;
+   * not null iff delegation support is enabled.
+   * @return the token support, or an empty option.
+   */
+  @VisibleForTesting
+  public Optional<S3ADelegationTokens> getDelegationTokens() {
+    return delegationTokens;
+  }
+
+  /**
+   * Return a service name iff delegation tokens are enabled and the
+   * token binding is issuing delegation tokens.
+   * @return the canonical service name or null
    */
   @Override
   public String getCanonicalServiceName() {
-    // Does not support Token
-    return null;
+    // this could all be done in map statements, but it'd be harder to
+    // understand and maintain.
+    // Essentially: no DTs, no canonical service name.
+    if (!delegationTokens.isPresent()) {
+      return null;
+    }
+    // DTs present: ask the binding if it is willing to
+    // serve tokens (or fail noisily).
+    S3ADelegationTokens dt = delegationTokens.get();
+    return dt.getTokenIssuingPolicy() != NoTokensAvailable
+        ? dt.getCanonicalServiceName()
+        : null;
+  }
+
+  /**
+   * Get a delegation token if the FS is set up for them.
+   * If the user already has a token, it is returned,
+   * <i>even if it has expired</i>.
+   * @param renewer the account name that is allowed to renew the token.
+   * @return the delegation token or null
+   * @throws IOException IO failure
+   */
+  @Override
+  public Token<AbstractS3ATokenIdentifier> getDelegationToken(String renewer)
+      throws IOException {
+    entryPoint(Statistic.INVOCATION_GET_DELEGATION_TOKEN);
+    LOG.debug("Delegation token requested");
+    if (delegationTokens.isPresent()) {
+      return delegationTokens.get().getBoundOrNewDT(encryptionSecrets);
+    } else {
+      // Delegation token support is not set up
+      LOG.debug("Token support is not enabled");
+      return null;
+    }
+  }
+
+  /**
+   * Build the AWS policy for restricted access to the resources needed
+   * by this bucket.
+   * The policy generated includes S3 access, S3Guard access
+   * if needed, and KMS operations.
+   * @param access access level desired.
+   * @return a policy for use in roles
+   */
+  @Override
+  public List<RoleModel.Statement> listAWSPolicyRules(
+      final Set<AccessLevel> access) {
+    if (access.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RoleModel.Statement> statements = new ArrayList<>(
+        allowS3Operations(bucket,
+            access.contains(AccessLevel.WRITE)
+                || access.contains(AccessLevel.ADMIN)));
+
+    // no attempt is made to qualify KMS access; there's no
+    // way to predict read keys, and not worried about granting
+    // too much encryption access.
+    statements.add(STATEMENT_ALLOW_SSE_KMS_RW);
+
+    // add any metastore policies
+    if (metadataStore instanceof AWSPolicyProvider) {
+      statements.addAll(
+          ((AWSPolicyProvider) metadataStore).listAWSPolicyRules(access));
+    }
+    return statements;
   }
 
   /**
@@ -2581,20 +2810,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
         });
   }
 
+  /**
+   * Set the optional parameters when initiating the request (encryption,
+   * headers, storage, etc).
+   * @param request request to patch.
+   */
   protected void setOptionalMultipartUploadRequestParameters(
-      InitiateMultipartUploadRequest req) {
-    switch (serverSideEncryptionAlgorithm) {
-    case SSE_KMS:
-      req.setSSEAwsKeyManagementParams(generateSSEAwsKeyParams());
-      break;
-    case SSE_C:
-      if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
-        //at the moment, only supports copy using the same key
-        req.setSSECustomerKey(generateSSECustomerKey());
-      }
-      break;
-    default:
-    }
+      InitiateMultipartUploadRequest request) {
+    generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
+    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
   }
 
   /**
@@ -2604,14 +2828,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   protected void setOptionalUploadPartRequestParameters(
       UploadPartRequest request) {
-    switch (serverSideEncryptionAlgorithm) {
-    case SSE_C:
-      if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
-        request.setSSECustomerKey(generateSSECustomerKey());
-      }
-      break;
-    default:
-    }
+    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
   }
 
   /**
@@ -2632,71 +2849,53 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
 
   protected void setOptionalCopyObjectRequestParameters(
       CopyObjectRequest copyObjectRequest) throws IOException {
-    switch (serverSideEncryptionAlgorithm) {
+    switch (getServerSideEncryptionAlgorithm()) {
     case SSE_KMS:
-      copyObjectRequest.setSSEAwsKeyManagementParams(
-          generateSSEAwsKeyParams()
-      );
+      generateSSEAwsKeyParams().ifPresent(
+          copyObjectRequest::setSSEAwsKeyManagementParams);
       break;
     case SSE_C:
-      if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
-        //at the moment, only supports copy using the same key
-        SSECustomerKey customerKey = generateSSECustomerKey();
+      generateSSECustomerKey().ifPresent(customerKey -> {
         copyObjectRequest.setSourceSSECustomerKey(customerKey);
         copyObjectRequest.setDestinationSSECustomerKey(customerKey);
-      }
+      });
       break;
     default:
     }
   }
 
   private void setOptionalPutRequestParameters(PutObjectRequest request) {
-    switch (serverSideEncryptionAlgorithm) {
-    case SSE_KMS:
-      request.setSSEAwsKeyManagementParams(generateSSEAwsKeyParams());
-      break;
-    case SSE_C:
-      if (isNotBlank(getServerSideEncryptionKey(bucket, getConf()))) {
-        request.setSSECustomerKey(generateSSECustomerKey());
-      }
-      break;
-    default:
-    }
+    generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
+    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
   }
 
   private void setOptionalObjectMetadata(ObjectMetadata metadata) {
-    if (S3AEncryptionMethods.SSE_S3.equals(serverSideEncryptionAlgorithm)) {
-      metadata.setSSEAlgorithm(serverSideEncryptionAlgorithm.getMethod());
+    final S3AEncryptionMethods algorithm
+        = getServerSideEncryptionAlgorithm();
+    if (S3AEncryptionMethods.SSE_S3.equals(algorithm)) {
+      metadata.setSSEAlgorithm(algorithm.getMethod());
     }
   }
 
   /**
-   * Create the AWS SDK structure used to configure SSE, based on the
-   * configuration.
-   * @return an instance of the class, which main contain the encryption key
+   * Create the AWS SDK structure used to configure SSE,
+   * if the encryption secrets contain the information/settings for this.
+   * @return an optional set of KMS Key settings
    */
-  @Retries.OnceExceptionsSwallowed
-  private SSEAwsKeyManagementParams generateSSEAwsKeyParams() {
-    //Use specified key, otherwise default to default master aws/s3 key by AWS
-    SSEAwsKeyManagementParams sseAwsKeyManagementParams =
-        new SSEAwsKeyManagementParams();
-    String encryptionKey = getServerSideEncryptionKey(bucket, getConf());
-    if (isNotBlank(encryptionKey)) {
-      sseAwsKeyManagementParams = new SSEAwsKeyManagementParams(encryptionKey);
-    }
-    return sseAwsKeyManagementParams;
+  private Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams() {
+    return EncryptionSecretOperations.createSSEAwsKeyManagementParams(
+        encryptionSecrets);
   }
 
   /**
-   * Create the SSE-C structure for the AWS SDK.
+   * Create the SSE-C structure for the AWS SDK, if the encryption secrets
+   * contain the information/settings for this.
    * This will contain a secret extracted from the bucket/configuration.
-   * @return the customer key.
+   * @return an optional customer key.
    */
-  @Retries.OnceExceptionsSwallowed
-  private SSECustomerKey generateSSECustomerKey() {
-    SSECustomerKey customerKey = new SSECustomerKey(
-        getServerSideEncryptionKey(bucket, getConf()));
-    return customerKey;
+  private Optional<SSECustomerKey> generateSSECustomerKey() {
+    return EncryptionSecretOperations.createSSECustomerKey(
+        encryptionSecrets);
   }
 
   /**
@@ -2902,9 +3101,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       sb.append(", blockSize=").append(getDefaultBlockSize());
     }
     sb.append(", multiPartThreshold=").append(multiPartThreshold);
-    if (serverSideEncryptionAlgorithm != null) {
+    if (getServerSideEncryptionAlgorithm() != null) {
       sb.append(", serverSideEncryptionAlgorithm='")
-          .append(serverSideEncryptionAlgorithm)
+          .append(getServerSideEncryptionAlgorithm())
           .append('\'');
     }
     if (blockFactory != null) {
@@ -2919,6 +3118,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", credentials=").append(credentials);
+    sb.append(", delegation tokens=")
+        .append(delegationTokens.map(Objects::toString).orElse("disabled"));
     sb.append(", statistics {")
         .append(statistics)
         .append("}");
@@ -3056,13 +3257,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
         ETAG_CHECKSUM_ENABLED_DEFAULT)) {
       Path path = qualify(f);
       LOG.debug("getFileChecksum({})", path);
-      return once("getFileChecksum", path.toString(),
-          () -> {
-            // this always does a full HEAD to the object
-            ObjectMetadata headers = getObjectMetadata(path);
-            String eTag = headers.getETag();
-            return eTag != null ? new EtagChecksum(eTag) : null;
-          });
+      ObjectMetadata headers = getObjectMetadata(path);
+      String eTag = headers.getETag();
+      return eTag != null ? new EtagChecksum(eTag) : null;
     } else {
       // disabled
       return null;
index 84f9c9f..78ba47d 100644 (file)
@@ -139,6 +139,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       INVOCATION_CREATE_NON_RECURSIVE,
       INVOCATION_DELETE,
       INVOCATION_EXISTS,
+      INVOCATION_GET_DELEGATION_TOKEN,
       INVOCATION_GET_FILE_CHECKSUM,
       INVOCATION_GET_FILE_STATUS,
       INVOCATION_GLOB_STATUS,
@@ -181,7 +182,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       S3GUARD_METADATASTORE_INITIALIZATION,
       S3GUARD_METADATASTORE_RETRY,
       S3GUARD_METADATASTORE_THROTTLED,
-      STORE_IO_THROTTLED
+      STORE_IO_THROTTLED,
+      DELEGATION_TOKENS_ISSUED
   };
 
   private static final Statistic[] GAUGES_TO_CREATE = {
@@ -1101,6 +1103,30 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
+   * Create a delegation token statistics instance.
+   * @return an instance of delegation token statistics
+   */
+  public DelegationTokenStatistics newDelegationTokenStatistics() {
+    return new DelegationTokenStatistics();
+  }
+
+  /**
+   * Instrumentation exported to S3A Delegation Token support.
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public final class DelegationTokenStatistics {
+
+    private DelegationTokenStatistics() {
+    }
+
+    /** A token has been issued. */
+    public void tokenIssued() {
+      incrementCounter(DELEGATION_TOKENS_ISSUED, 1);
+    }
+  }
+
+    /**
    * Copy all the metrics to a map of (name, long-value).
    * @return a map of the metrics
    */
index 9318a5a..6182b43 100644 (file)
@@ -26,7 +26,6 @@ import com.amazonaws.Protocol;
 import com.amazonaws.SdkBaseException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
-import com.amazonaws.auth.InstanceProfileCredentialsProvider;
 import com.amazonaws.retry.RetryUtils;
 import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
 import com.amazonaws.services.dynamodbv2.model.LimitExceededException;
@@ -36,6 +35,7 @@ import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 import org.apache.commons.lang3.StringUtils;
@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
 import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.net.ConnectTimeoutException;
@@ -71,11 +72,15 @@ import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
@@ -126,6 +131,13 @@ public final class S3AUtils {
 
   private static final String BUCKET_PATTERN = FS_S3A_BUCKET_PREFIX + "%s.%s";
 
+  /**
+   * Error message when the AWS provider list built up contains a forbidden
+   * entry.
+   */
+  @VisibleForTesting
+  public static final String E_FORBIDDEN_AWS_PROVIDER
+      = "AWS provider class cannot be used";
 
   private S3AUtils() {
   }
@@ -168,7 +180,7 @@ public final class S3AUtils {
       SdkBaseException exception) {
     String message = String.format("%s%s: %s",
         operation,
-        path != null ? (" on " + path) : "",
+        StringUtils.isNotEmpty(path)? (" on " + path) : "",
         exception);
     if (!(exception instanceof AmazonServiceException)) {
       Exception innerCause = containsInterruptedException(exception);
@@ -581,35 +593,39 @@ public final class S3AUtils {
   }
 
   /**
+   * The standard AWS provider list for AWS connections.
+   */
+  public static final List<Class<?>>
+      STANDARD_AWS_PROVIDERS = Collections.unmodifiableList(
+      Arrays.asList(
+          TemporaryAWSCredentialsProvider.class,
+          SimpleAWSCredentialsProvider.class,
+          EnvironmentVariableCredentialsProvider.class,
+          IAMInstanceCredentialsProvider.class));
+
+  /**
    * 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;
-   * may be null
+   * @param binding Binding URI -may be null
    * @param conf filesystem configuration
    * @return a credentials provider list
    * @throws IOException Problems loading the providers (including reading
    * secrets from credential files).
    */
   public static AWSCredentialProviderList createAWSCredentialProviderSet(
-      URI binding, Configuration conf) throws IOException {
-    AWSCredentialProviderList credentials = new AWSCredentialProviderList();
-
-    Class<?>[] awsClasses = loadAWSProviderClasses(conf,
-        AWS_CREDENTIALS_PROVIDER);
-    if (awsClasses.length == 0) {
-      credentials.add(new SimpleAWSCredentialsProvider(binding, conf));
-      credentials.add(new EnvironmentVariableCredentialsProvider());
-      credentials.add(InstanceProfileCredentialsProvider.getInstance());
-    } else {
-      for (Class<?> aClass : awsClasses) {
-        credentials.add(createAWSCredentialProvider(conf,
-            aClass,
-            binding));
-      }
-    }
+      @Nullable URI binding,
+      Configuration conf) throws IOException {
+    // this will reject any user:secret entries in the URI
+    S3xLoginHelper.rejectSecretsInURIs(binding);
+    AWSCredentialProviderList credentials =
+        buildAWSProviderList(binding,
+            conf,
+            AWS_CREDENTIALS_PROVIDER,
+            STANDARD_AWS_PROVIDERS,
+            new HashSet<>());
     // make sure the logging message strips out any auth details
     LOG.debug("For URI {}, using credentials {}",
-        S3xLoginHelper.toString(binding), credentials);
+        binding, credentials);
     return credentials;
   }
 
@@ -621,11 +637,11 @@ public final class S3AUtils {
    * @return the list of classes, possibly empty
    * @throws IOException on a failure to load the list.
    */
-  public static Class<?>[] loadAWSProviderClasses(Configuration conf,
+  public static List<Class<?>> loadAWSProviderClasses(Configuration conf,
       String key,
       Class<?>... defaultValue) throws IOException {
     try {
-      return conf.getClasses(key, defaultValue);
+      return Arrays.asList(conf.getClasses(key, defaultValue));
     } catch (RuntimeException e) {
       Throwable c = e.getCause() != null ? e.getCause() : e;
       throw new IOException("From option " + key + ' ' + c, c);
@@ -633,6 +649,49 @@ public final class S3AUtils {
   }
 
   /**
+   * Load list of AWS credential provider/credential provider factory classes;
+   * support a forbidden list to prevent loops, mandate full secrets, etc.
+   * @param binding Binding URI -may be null
+   * @param conf configuration
+   * @param key key
+   * @param forbidden a possibly empty set of forbidden classes.
+   * @param defaultValues list of default providers.
+   * @return the list of classes, possibly empty
+   * @throws IOException on a failure to load the list.
+   */
+  public static AWSCredentialProviderList buildAWSProviderList(
+      @Nullable final URI binding,
+      final Configuration conf,
+      final String key,
+      final List<Class<?>> defaultValues,
+      final Set<Class<?>> forbidden) throws IOException {
+
+    // build up the base provider
+    List<Class<?>> awsClasses = loadAWSProviderClasses(conf,
+        key,
+        defaultValues.toArray(new Class[defaultValues.size()]));
+    // and if the list is empty, switch back to the defaults.
+    // this is to address the issue that configuration.getClasses()
+    // doesn't return the default if the config value is just whitespace.
+    if (awsClasses.isEmpty()) {
+      awsClasses = defaultValues;
+    }
+    // iterate through, checking for blacklists and then instantiating
+    // each provider
+    AWSCredentialProviderList providers = new AWSCredentialProviderList();
+    for (Class<?> aClass : awsClasses) {
+
+      if (forbidden.contains(aClass)) {
+        throw new IOException(E_FORBIDDEN_AWS_PROVIDER
+            + " in option " + key + ": " + aClass);
+      }
+      providers.add(createAWSCredentialProvider(conf,
+          aClass, binding));
+    }
+    return providers;
+  }
+
+  /**
    * Create an AWS credential provider from its class by using reflection.  The
    * class must implement one of the following means of construction, which are
    * attempted in order:
@@ -640,6 +699,8 @@ public final class S3AUtils {
    * <ol>
    * <li>a public constructor accepting java.net.URI and
    *     org.apache.hadoop.conf.Configuration</li>
+   * <li>a public constructor accepting
+   *    org.apache.hadoop.conf.Configuration</li>
    * <li>a public static method named getInstance that accepts no
    *    arguments and returns an instance of
    *    com.amazonaws.auth.AWSCredentialsProvider, or</li>
@@ -652,11 +713,11 @@ public final class S3AUtils {
    * @return the instantiated class
    * @throws IOException on any instantiation failure.
    */
-  public static AWSCredentialsProvider createAWSCredentialProvider(
+  private static AWSCredentialsProvider createAWSCredentialProvider(
       Configuration conf,
       Class<?> credClass,
-      URI uri) throws IOException {
-    AWSCredentialsProvider credentials;
+      @Nullable URI uri) throws IOException {
+    AWSCredentialsProvider credentials = null;
     String className = credClass.getName();
     if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
       throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
@@ -699,9 +760,9 @@ public final class S3AUtils {
       // no supported constructor or factory method found
       throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
           + ".  A class specified in %s must provide a public constructor "
-          + "accepting Configuration, or a public factory method named "
-          + "getInstance that accepts no arguments, or a public default "
-          + "constructor.", className, AWS_CREDENTIALS_PROVIDER));
+          + "of a supported signature, or a public factory method named "
+          + "getInstance that accepts no arguments.",
+          className, AWS_CREDENTIALS_PROVIDER));
     } catch (InvocationTargetException e) {
       Throwable targetException = e.getTargetException();
       if (targetException == null) {
@@ -727,6 +788,24 @@ public final class S3AUtils {
   }
 
   /**
+   * Set a key if the value is non-empty.
+   * @param config config to patch
+   * @param key key to set
+   * @param val value to probe and set
+   * @param origin origin
+   * @return true if the property was set
+   */
+  public static boolean setIfDefined(Configuration config, String key,
+      String val, String origin) {
+    if (StringUtils.isNotEmpty(val)) {
+      config.set(key, val, origin);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
    * Return the access key and secret for S3 API use.
    * or indicated in the UserInfo of the name URI param.
    * @param name the URI for which we need the access keys; may be null
@@ -1399,7 +1478,7 @@ public final class S3AUtils {
    * @return the encryption key or ""
    * @throws IllegalArgumentException bad arguments.
    */
-  static String getServerSideEncryptionKey(String bucket,
+  public static String getServerSideEncryptionKey(String bucket,
       Configuration conf) {
     try {
       return lookupPassword(bucket, conf, SERVER_SIDE_ENCRYPTION_KEY);
@@ -1420,7 +1499,7 @@ public final class S3AUtils {
    * one is set.
    * @throws IOException on any validation problem.
    */
-  static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
+  public static S3AEncryptionMethods getEncryptionAlgorithm(String bucket,
       Configuration conf) throws IOException {
     S3AEncryptionMethods sse = S3AEncryptionMethods.getMethod(
         lookupPassword(bucket, conf,
@@ -1430,6 +1509,7 @@ public final class S3AUtils {
     String diagnostics = passwordDiagnostics(sseKey, "key");
     switch (sse) {
     case SSE_C:
+      LOG.debug("Using SSE-C with {}", diagnostics);
       if (sseKeyLen == 0) {
         throw new IOException(SSE_C_NO_KEY_ERROR);
       }
@@ -1452,7 +1532,6 @@ public final class S3AUtils {
       LOG.debug("Data is unencrypted");
       break;
     }
-    LOG.debug("Using SSE-C with {}", diagnostics);
     return sse;
   }
 
index b237e85..e0a1d78 100644 (file)
@@ -40,11 +40,13 @@ public interface S3ClientFactory {
    * @param name raw input S3A file system URI
    * @param bucket Optional bucket to use to look up per-bucket proxy secrets
    * @param credentialSet credentials to use
+   * @param userAgentSuffix optional suffix for the UA field.
    * @return S3 client
    * @throws IOException IO problem
    */
   AmazonS3 createS3Client(URI name,
-      final String bucket,
-      final AWSCredentialsProvider credentialSet) throws IOException;
+      String bucket,
+      AWSCredentialsProvider credentialSet,
+      String userAgentSuffix) throws IOException;
 
 }
index b31b72a..255d009 100644 (file)
@@ -21,19 +21,18 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
+import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
-import org.apache.hadoop.security.ProviderUtils;
 
 import java.io.IOException;
 import java.net.URI;
 
-import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
-import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
 import static org.apache.hadoop.fs.s3a.S3AUtils.getAWSAccessKeys;
 
 /**
@@ -49,13 +48,29 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
 
   public static final String NAME
       = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider";
-  private String accessKey;
-  private String secretKey;
+  private final String accessKey;
+  private final String secretKey;
 
-  public SimpleAWSCredentialsProvider(URI uri, Configuration conf)
+  /**
+   * Build the credentials from a filesystem URI and configuration.
+   * @param uri FS URI
+   * @param conf configuration containing secrets/references to.
+   * @throws IOException failure
+   */
+  public SimpleAWSCredentialsProvider(final URI uri, final Configuration conf)
       throws IOException {
+      this(getAWSAccessKeys(uri, conf));
+  }
 
-      S3xLoginHelper.Login login = getAWSAccessKeys(uri, conf);
+  /**
+   * Instantiate from a login tuple.
+   * For testing, hence package-scoped.
+   * @param login login secrets
+   * @throws IOException failure
+   */
+  @VisibleForTesting
+  SimpleAWSCredentialsProvider(final S3xLoginHelper.Login login)
+      throws IOException {
       this.accessKey = login.getUser();
       this.secretKey = login.getPassword();
   }
@@ -65,8 +80,8 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
     if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
       return new BasicAWSCredentials(accessKey, secretKey);
     }
-    throw new CredentialInitializationException(
-        "Access key or secret key is unset");
+    throw new NoAwsCredentialsException("SimpleAWSCredentialsProvider",
+        "No AWS credentials in the Hadoop configuration");
   }
 
   @Override
index 799a586..10ae1db 100644 (file)
@@ -57,6 +57,8 @@ public enum Statistic {
       "Calls of delete()"),
   INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
       "Calls of exists()"),
+  INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN,
+      "Calls of getDelegationToken()"),
   INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
       "Calls of getFileChecksum()"),
   INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
@@ -209,7 +211,10 @@ public enum Statistic {
       "s3guard_metadatastore_throttle_rate",
       "S3Guard metadata store throttle rate"),
 
-  STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried");
+  STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"),
+
+  DELEGATION_TOKENS_ISSUED("delegation_tokens_issued",
+      "Number of delegation tokens issued");
 
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);
index d42f68e..f124bd0 100644 (file)
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
 
 package org.apache.hadoop.fs.s3a;
 
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.BasicSessionCredentials;
+import javax.annotation.Nullable;
+import java.io.IOException;
+
 import com.amazonaws.auth.AWSCredentials;
-import org.apache.commons.lang3.StringUtils;
 
-import java.io.IOException;
 import java.net.URI;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.ProviderUtils;
-
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
+import org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
+import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
 
 /**
  * Support session credentials for authenticating with AWS.
@@ -40,50 +40,65 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
  * Please note that users may reference this class name from configuration
  * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
  * would be a backward-incompatible change.
+ *
+ * This credential provider must not fail in creation because that will
+ * break a chain of credential providers.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
+public class TemporaryAWSCredentialsProvider extends
+    AbstractSessionCredentialsProvider {
 
   public static final String NAME
       = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-  private String accessKey;
-  private String secretKey;
-  private String sessionToken;
 
-  public TemporaryAWSCredentialsProvider(Configuration conf)
+  public static final String COMPONENT
+      = "Session credentials in Hadoop configuration";
+
+  /**
+   * Construct from just a configuration.
+   * @param conf configuration.
+   */
+  public TemporaryAWSCredentialsProvider(final Configuration conf)
       throws IOException {
     this(null, conf);
   }
 
-  public TemporaryAWSCredentialsProvider(URI uri, Configuration conf)
+  /**
+   * Constructor: the URI will be null if the provider is inited unbonded
+   * to a filesystem.
+   * @param uri binding to a filesystem URI.
+   * @param conf configuration.
+   */
+  public TemporaryAWSCredentialsProvider(
+      @Nullable final URI uri,
+      final Configuration conf)
       throws IOException {
-
-      // determine the bucket
-      String bucket = uri != null ? uri.getHost():  "";
-      Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
-          conf, S3AFileSystem.class);
-      this.accessKey = lookupPassword(bucket, c, ACCESS_KEY);
-      this.secretKey = lookupPassword(bucket, c, SECRET_KEY);
-      this.sessionToken = lookupPassword(bucket, c, SESSION_TOKEN);
+    super(uri, conf);
   }
 
+  /**
+   * The credentials here must include a session token, else this operation
+   * will raise an exception.
+   * @param config the configuration
+   * @return temporary credentials.
+   * @throws IOException on any failure to load the credentials.
+   * @throws NoAuthWithAWSException validation failure
+   * @throws NoAwsCredentialsException the credentials are actually empty.
+   */
   @Override
-  public AWSCredentials getCredentials() {
-    if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
-        && !StringUtils.isEmpty(sessionToken)) {
-      return new BasicSessionCredentials(accessKey, secretKey, sessionToken);
+  protected AWSCredentials createCredentials(Configuration config)
+      throws IOException {
+    MarshalledCredentials creds = MarshalledCredentialBinding.fromFileSystem(
+        getUri(), config);
+    MarshalledCredentials.CredentialTypeRequired sessionOnly
+        = MarshalledCredentials.CredentialTypeRequired.SessionOnly;
+    // treat only having non-session creds as empty.
+    if (!creds.isValid(sessionOnly)) {
+      throw new NoAwsCredentialsException(COMPONENT);
     }
-    throw new CredentialInitializationException(
-        "Access key, secret key or session token is unset");
-  }
-
-  @Override
-  public void refresh() {}
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName();
+    return MarshalledCredentialBinding.toAWSCredentials(creds,
+        sessionOnly, COMPONENT);
   }
 
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java
new file mode 100644 (file)
index 0000000..1f714b0
--- /dev/null
@@ -0,0 +1,70 @@
+/*
+ * 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.auth;
+
+import javax.annotation.Nullable;
+import java.net.URI;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Base class for AWS credential providers which
+ * take a URI and config in their constructor.
+ */
+public abstract class AbstractAWSCredentialProvider
+    implements AWSCredentialsProvider {
+
+  private final URI binding;
+
+  private final Configuration conf;
+
+  /**
+   * Construct from URI + configuration.
+   * @param uri URI: may be null.
+   * @param conf configuration.
+   */
+  protected AbstractAWSCredentialProvider(
+      @Nullable final URI uri,
+      final Configuration conf) {
+    this.conf = conf;
+    this.binding = uri;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Get the binding URI: may be null.
+   * @return the URI this instance was constructed with,
+   * if any.
+   */
+  public URI getUri() {
+    return binding;
+  }
+
+  /**
+   * Refresh is a no-op by default.
+   */
+  @Override
+  public void refresh() {
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java
new file mode 100644 (file)
index 0000000..7822035
--- /dev/null
@@ -0,0 +1,170 @@
+/*
+ * 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.auth;
+
+import javax.annotation.Nullable;
+import java.net.URI;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.amazonaws.SdkBaseException;
+import com.amazonaws.auth.AWSCredentials;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.CredentialInitializationException;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
+
+/**
+ * Base class for session credential support.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractSessionCredentialsProvider
+    extends AbstractAWSCredentialProvider {
+
+  /** Credentials, created in {@link #init()}. */
+  private AWSCredentials awsCredentials;
+
+  /** Atomic flag for on-demand initialization. */
+  private final AtomicBoolean initialized = new AtomicBoolean(false);
+
+  /**
+   * The (possibly translated) initialization exception.
+   * Used for testing.
+   */
+  private IOException initializationException;
+
+  /**
+   * Constructor.
+   * @param uri possibly null filesystem URI.
+   * @param conf configuration.
+   */
+  public AbstractSessionCredentialsProvider(
+      @Nullable final URI uri,
+      final Configuration conf) {
+    super(uri, conf);
+  }
+
+  /**
+   * Initialize the credentials by calling
+   * {@link #createCredentials(Configuration)} with the current config.
+   */
+  @Retries.OnceTranslated
+  protected void init() throws IOException {
+    // stop re-entrant attempts
+    if (initialized.getAndSet(true)) {
+      return;
+    }
+    try {
+      awsCredentials = Invoker.once("create credentials", "",
+          () -> createCredentials(getConf()));
+    } catch (IOException e) {
+      initializationException = e;
+      throw e;
+    }
+  }
+
+  /**
+   * Has an attempt to initialize the credentials been attempted?
+   * @return true if {@code init()} was called.
+   */
+  public boolean isInitialized() {
+    return initialized.get();
+  }
+
+  /**
+   * Implementation point: whatever the subclass must do to load credentials.
+   * This is called from {@link #init()} and then the credentials are cached,
+   * along with any exception.
+   * @param config the configuration
+   * @return the credentials
+   * @throws IOException on any failure.
+   */
+  protected abstract AWSCredentials createCredentials(Configuration config)
+      throws IOException;
+
+  /**
+   * Get the credentials.
+   * Any exception raised in
+   * {@link #createCredentials(Configuration)}
+   * is thrown here before any attempt to return the credentials
+   * is made.
+   * @return credentials, if set.
+   * @throws SdkBaseException if one was raised during init
+   * @throws CredentialInitializationException on other failures.
+   */
+  public AWSCredentials getCredentials() throws SdkBaseException {
+    // do an on-demand init then raise an AWS SDK exception if
+    // there was a failure.
+    try {
+      if (!isInitialized()) {
+        init();
+      }
+    } catch (IOException e) {
+      if (e.getCause() instanceof SdkBaseException) {
+        throw (SdkBaseException) e.getCause();
+      } else {
+        throw new CredentialInitializationException(e.getMessage(), e);
+      }
+    }
+    if (awsCredentials == null) {
+      throw new CredentialInitializationException(
+          "Provider " + this + " has no credentials");
+    }
+    return awsCredentials;
+  }
+
+  public final boolean hasCredentials() {
+    return awsCredentials == null;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+  /**
+   * Get any IOE raised during initialization.
+   * Null if {@link #init()} hasn't been called, or it actually worked.
+   * @return an exception or null.
+   */
+  @VisibleForTesting
+  public IOException getInitializationException() {
+    return initializationException;
+  }
+
+  /**
+   * A special set of null credentials which are not the anonymous class.
+   * This will be interpreted as "this provider has no credentials to offer",
+   * rather than an explicit error or anonymous access.
+   */
+  protected static final class NoCredentials implements AWSCredentials {
+    @Override
+    public String getAWSAccessKeyId() {
+      return null;
+    }
+
+    @Override
+    public String getAWSSecretKey() {
+      return null;
+    }
+  }
+}
index e5a3639..afad1f8 100644 (file)
 
 package org.apache.hadoop.fs.s3a.auth;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.URI;
+import java.util.Arrays;
 import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 
-import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
 import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +42,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.CredentialInitializationException;
+import org.apache.hadoop.fs.s3a.Retries;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
@@ -46,8 +51,7 @@ import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider;
 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;
+import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
 
 /**
  * Support IAM Assumed roles by instantiating an instance of
@@ -67,10 +71,6 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
   public static final String NAME
       = "org.apache.hadoop.fs.s3a.auth.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;
 
@@ -90,13 +90,13 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
    * Instantiate.
    * This calls {@link #getCredentials()} to fail fast on the inner
    * role credential retrieval.
-   * @param fsUri URI of the filesystem.
+   * @param fsUri possibly null URI of the filesystem.
    * @param conf configuration
    * @throws IOException on IO problems and some parameter checking
    * @throws IllegalArgumentException invalid parameters
    * @throws AWSSecurityTokenServiceException problems getting credentials
    */
-  public AssumedRoleCredentialProvider(URI fsUri, Configuration conf)
+  public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf)
       throws IOException {
 
     arn = conf.getTrimmed(ASSUMED_ROLE_ARN, "");
@@ -105,16 +105,12 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
     }
 
     // build up the base provider
-    Class<?>[] awsClasses = loadAWSProviderClasses(conf,
+    credentialsToSTS = buildAWSProviderList(fsUri, conf,
         ASSUMED_ROLE_CREDENTIALS_PROVIDER,
-        SimpleAWSCredentialsProvider.class);
-    credentialsToSTS = new AWSCredentialProviderList();
-    for (Class<?> aClass : awsClasses) {
-      if (this.getClass().equals(aClass)) {
-        throw new IOException(E_FORBIDDEN_PROVIDER);
-      }
-      credentialsToSTS.add(createAWSCredentialProvider(conf, aClass, fsUri));
-    }
+        Arrays.asList(
+            SimpleAWSCredentialsProvider.class,
+            EnvironmentVariableCredentialsProvider.class),
+        Sets.newHashSet(this.getClass()));
     LOG.debug("Credentials to obtain role credentials: {}", credentialsToSTS);
 
     // then the STS binding
@@ -132,13 +128,13 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
       LOG.debug("Scope down policy {}", policy);
       builder.withScopeDownPolicy(policy);
     }
-    String endpoint = conf.get(ASSUMED_ROLE_STS_ENDPOINT, "");
-    String region = conf.get(ASSUMED_ROLE_STS_ENDPOINT_REGION,
+    String endpoint = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT, "");
+    String region = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,
         ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT);
     AWSSecurityTokenServiceClientBuilder stsbuilder =
         STSClientFactory.builder(
           conf,
-          fsUri.getHost(),
+          fsUri != null ?  fsUri.getHost() : "",
           credentialsToSTS,
           endpoint,
           region);
@@ -164,6 +160,7 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
    * @throws AWSSecurityTokenServiceException if none could be obtained.
    */
   @Override
+  @Retries.RetryRaw
   public AWSCredentials getCredentials() {
     try {
       return invoker.retryUntranslated("getCredentials",
@@ -174,7 +171,7 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
       // its hard to see how this could be raised, but for
       // completeness, it is wrapped as an Amazon Client Exception
       // and rethrown.
-      throw new AmazonClientException(
+      throw new CredentialInitializationException(
           "getCredentials failed: " + e,
           e);
     } catch (AWSSecurityTokenServiceException e) {
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java
new file mode 100644 (file)
index 0000000..7ff4510
--- /dev/null
@@ -0,0 +1,75 @@
+/*
+ * 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.auth;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.InstanceProfileCredentialsProvider;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This is going to be an IAM credential provider which performs
+ * async refresh for lower-latency on IO calls.
+ * Initially it does not do this, simply shares the single IAM instance
+ * across all instances. This makes it less expensive to declare.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IAMInstanceCredentialsProvider
+    implements AWSCredentialsProvider, Closeable {
+
+  private static final InstanceProfileCredentialsProvider INSTANCE =
+      InstanceProfileCredentialsProvider.getInstance();
+
+  public IAMInstanceCredentialsProvider() {
+  }
+
+  /**
+   * Ask for the credentials.
+   * as it invariably means "you aren't running on EC2"
+   * @return the credentials
+   */
+  @Override
+  public AWSCredentials getCredentials() {
+    try {
+      return INSTANCE.getCredentials();
+    } catch (AmazonClientException e) {
+      throw new NoAwsCredentialsException("IAMInstanceCredentialsProvider",
+          e.getMessage(),
+          e);
+    }
+  }
+
+  @Override
+  public void refresh() {
+    INSTANCE.refresh();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // until async, no-op.
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java
new file mode 100644 (file)
index 0000000..58c9c80
--- /dev/null
@@ -0,0 +1,205 @@
+/*
+ * 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.auth;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Date;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSSessionCredentials;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.BasicSessionCredentials;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
+import com.amazonaws.services.securitytoken.model.Credentials;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.security.ProviderUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SESSION_TOKEN;
+import static org.apache.hadoop.fs.s3a.S3AUtils.lookupPassword;
+
+/**
+ * Class to bridge from the serializable/marshallabled
+ * {@link MarshalledCredentialBinding} class to/from AWS classes.
+ * This is to keep that class isolated and not dependent on aws-sdk JARs
+ * to load.
+ */
+public final class MarshalledCredentialBinding {
+
+  private MarshalledCredentialBinding() {
+  }
+
+  /**
+   * Error text on empty credentials: {@value}.
+   */
+  @VisibleForTesting
+  public static final String NO_AWS_CREDENTIALS = "No AWS credentials";
+
+  /**
+   * Create a set of marshalled credentials from a set of credentials
+   * issued by an STS call.
+   * @param credentials AWS-provided session credentials
+   */
+  public static MarshalledCredentials fromSTSCredentials(
+      final Credentials credentials) {
+    MarshalledCredentials marshalled = new MarshalledCredentials(
+        credentials.getAccessKeyId(),
+        credentials.getSecretAccessKey(),
+        credentials.getSessionToken());
+    Date date = credentials.getExpiration();
+    marshalled.setExpiration(date != null ? date.getTime() : 0);
+    return marshalled;
+  }
+
+  /**
+   * Create from a set of AWS credentials.
+   * @param credentials source credential.
+   * @return a set of marshalled credentials.
+   */
+  public static MarshalledCredentials fromAWSCredentials(
+      final AWSSessionCredentials credentials) {
+    return new MarshalledCredentials(
+        credentials.getAWSAccessKeyId(),
+        credentials.getAWSSecretKey(),
+        credentials.getSessionToken());
+  }
+
+  /**
+   * Build a set of credentials from the environment.
+   * @param env environment.
+   * @return a possibly incomplete/invalid set of credentials.
+   */
+  public static MarshalledCredentials fromEnvironment(
+      final Map<String, String> env) {
+    return new MarshalledCredentials(
+      nullToEmptyString(env.get("AWS_ACCESS_KEY")),
+      nullToEmptyString(env.get("AWS_SECRET_KEY")),
+      nullToEmptyString(env.get("AWS_SESSION_TOKEN")));
+  }
+
+  /**
+   * Take a string where a null value is remapped to an empty string.
+   * @param src source string.
+   * @return the value of the string or ""
+   */
+  private static String nullToEmptyString(final String src) {
+    return src == null ? "" : src;
+  }
+
+  /**
+   * Loads the credentials from the owning S3A FS, including
+   * from Hadoop credential providers.
+   * There is no validation.
+   * @param conf configuration to load from
+   * @return the component
+   * @throws IOException on any load failure
+   */
+  public static MarshalledCredentials fromFileSystem(
+      final URI uri,
+      final Configuration conf) throws IOException {
+    // determine the bucket
+    final String bucket = uri != null ? uri.getHost() : "";
+    final Configuration leanConf =
+        ProviderUtils.excludeIncompatibleCredentialProviders(
+            conf, S3AFileSystem.class);
+    return new MarshalledCredentials(
+        lookupPassword(bucket, leanConf, ACCESS_KEY),
+        lookupPassword(bucket, leanConf, SECRET_KEY),
+        lookupPassword(bucket, leanConf, SESSION_TOKEN));
+  }
+
+  /**
+   * Create an AWS credential set from a set of marshalled credentials.
+   *
+   * This code would seem to fit into (@link MarshalledCredentials}, and
+   * while it would from a code-hygiene perspective, to keep all AWS
+   * SDK references out of that class, the logic is implemented here instead,
+   * @param marshalled marshalled credentials
+   * @param typeRequired type of credentials required
+   * @param component component name for exception messages.
+   * @return a new set of credentials
+   * @throws NoAuthWithAWSException validation failure
+   * @throws NoAwsCredentialsException the credentials are actually empty.
+   */
+  public static AWSCredentials toAWSCredentials(
+      final MarshalledCredentials marshalled,
+      final MarshalledCredentials.CredentialTypeRequired typeRequired,
+      final String component)
+      throws NoAuthWithAWSException, NoAwsCredentialsException {
+
+    if (marshalled.isEmpty()) {
+      throw new NoAwsCredentialsException(component, NO_AWS_CREDENTIALS);
+    }
+    if (!marshalled.isValid(typeRequired)) {
+      throw new NoAuthWithAWSException(component + ":" +
+          marshalled.buildInvalidCredentialsError(typeRequired));
+    }
+    final String accessKey = marshalled.getAccessKey();
+    final String secretKey = marshalled.getSecretKey();
+    if (marshalled.hasSessionToken()) {
+      // a session token was supplied, so return session credentials
+      return new BasicSessionCredentials(accessKey, secretKey,
+          marshalled.getSessionToken());
+    } else {
+      // these are full credentials
+      return new BasicAWSCredentials(accessKey, secretKey);
+    }
+  }
+
+  /**
+   * Request a set of credentials from an STS endpoint.
+   * @param parentCredentials the parent credentials needed to talk to STS
+   * @param stsEndpoint an endpoint, use "" for none
+   * @param stsRegion region; use if the endpoint isn't the AWS default.
+   * @param duration duration of the credentials in seconds. Minimum value: 900.
+   * @param invoker invoker to use for retrying the call.
+   * @return the credentials
+   * @throws IOException on a failure of the request
+   */
+  @Retries.RetryTranslated
+  public static MarshalledCredentials requestSessionCredentials(
+      final AWSCredentialsProvider parentCredentials,
+      final ClientConfiguration awsConf,
+      final String stsEndpoint,
+      final String stsRegion,
+      final int duration,
+      final Invoker invoker) throws IOException {
+    final AWSSecurityTokenService tokenService =
+        STSClientFactory.builder(parentCredentials,
+            awsConf,
+            stsEndpoint.isEmpty() ? null : stsEndpoint,
+            stsRegion)
+            .build();
+    return fromSTSCredentials(
+        STSClientFactory.createClientConnection(tokenService, invoker)
+            .requestSessionCredentials(duration, TimeUnit.SECONDS));
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java
new file mode 100644 (file)
index 0000000..03e26e7
--- /dev/null
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth;
+
+import java.io.IOException;
+import java.net.URI;
+
+import com.amazonaws.auth.AWSCredentials;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.CredentialInitializationException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.toAWSCredentials;
+
+/**
+ * AWS credential provider driven from marshalled session/full credentials
+ * (full, simple session or role).
+ * This is <i>not</i> intended for explicit use in job/app configurations,
+ * instead it is returned by Delegation Token Bindings, as needed.
+ * The constructor implicitly prevents explicit use.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class MarshalledCredentialProvider extends
+    AbstractSessionCredentialsProvider {
+
+  /** Name: {@value}. */
+  public static final String NAME
+      = "org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider";
+
+  private final MarshalledCredentials credentials;
+
+  private final MarshalledCredentials.CredentialTypeRequired typeRequired;
+
+  private final String component;
+
+  /**
+   * Constructor.
+   *
+   * @param component component name for exception messages.
+   * @param uri filesystem URI: must not be null.
+   * @param conf configuration.
+   * @param credentials marshalled credentials.
+   * @param typeRequired credential type required.
+   * @throws CredentialInitializationException validation failure
+   * @throws IOException failure
+   */
+  public MarshalledCredentialProvider(
+      final String component,
+      final URI uri,
+      final Configuration conf,
+      final MarshalledCredentials credentials,
+      final MarshalledCredentials.CredentialTypeRequired typeRequired)
+      throws IOException {
+    super(checkNotNull(uri, "No filesystem URI"), conf);
+    this.component = component;
+    this.typeRequired = typeRequired;
+    this.credentials = checkNotNull(credentials);
+  }
+
+  /**
+   * Perform the binding, looking up the DT and parsing it.
+   * @return true if there were some credentials
+   * @throws CredentialInitializationException validation failure
+   * @throws IOException on a failure
+   */
+  @Override
+  protected AWSCredentials createCredentials(final Configuration config)
+      throws IOException {
+    return toAWSCredentials(credentials, typeRequired, component);
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentials.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentials.java
new file mode 100644 (file)
index 0000000..5737dbc
--- /dev/null
@@ -0,0 +1,409 @@
+/*
+ * 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.auth;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Date;
+import java.util.Objects;
+import java.util.Optional;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenIOException;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SESSION_TOKEN;
+
+/**
+ * Stores the credentials for a session or for a full login.
+ * This structure is {@link Writable}, so can be marshalled inside a
+ * delegation token.
+ *
+ * The class is designed so that keys inside are kept non-null; to be
+ * unset just set them to the empty string. This is to simplify marshalling.
+ *
+ * <i>Important: Add no references to any AWS SDK class, to
+ * ensure it can be safely deserialized whenever the relevant token
+ * identifier of a token type declared in this JAR is examined.</i>
+ */
+@InterfaceAudience.Private
+public final class MarshalledCredentials implements Writable, Serializable {
+
+  /**
+   * Error text on invalid non-empty credentials: {@value}.
+   */
+  @VisibleForTesting
+  public static final String INVALID_CREDENTIALS
+      = "Invalid AWS credentials";
+
+  /**
+   * How long can any of the secrets be: {@value}.
+   * This is much longer than the current tokens, but leaves space for
+   * future enhancements.
+   */
+  private static final int MAX_SECRET_LENGTH = 8192;
+
+  private static final long serialVersionUID = 8444610385533920692L;
+
+  /**
+   * Access key of IAM account.
+   */
+  private String accessKey = "";
+
+  /**
+   * Secret key of IAM account.
+   */
+  private String secretKey = "";
+
+  /**
+   * Optional session token.
+   * If non-empty: the credentials can be converted into
+   * session credentials.
+   */
+  private String sessionToken = "";
+
+  /**
+   * ARN of a role. Purely for diagnostics.
+   */
+  private String roleARN = "";
+
+  /**
+   * Expiry time milliseconds in UTC; the {@code Java.Util.Date} value.
+   * 0 means "does not expire/unknown".
+   */
+  private long expiration;
+
+  /**
+   * Constructor.
+   */
+  public MarshalledCredentials() {
+  }
+
+  /**
+   * Create from a set of properties.
+   * No expiry time is expected/known here.
+   * @param accessKey access key
+   * @param secretKey secret key
+   * @param sessionToken session token
+   */
+  public MarshalledCredentials(
+      final String accessKey,
+      final String secretKey,
+      final String sessionToken) {
+    this();
+    this.accessKey = requireNonNull(accessKey);
+    this.secretKey = requireNonNull(secretKey);
+    this.sessionToken = sessionToken == null ? "" : sessionToken;
+  }
+
+  public String getAccessKey() {
+    return accessKey;
+  }
+
+  public String getSecretKey() {
+    return secretKey;
+  }
+
+  public String getSessionToken() {
+    return sessionToken;
+  }
+
+  /**
+   * Expiration; will be 0 for none known.
+   * @return any expiration timestamp
+   */
+  public long getExpiration() {
+    return expiration;
+  }
+
+  public void setExpiration(final long expiration) {
+    this.expiration = expiration;
+  }
+
+  /**
+   * Get a temporal representing the time of expiration, if there
+   * is one.
+   * This is here to wrap up expectations about timestamps and zones.
+   * @return the expiration time.
+   */
+  public Optional<OffsetDateTime> getExpirationDateTime() {
+    return expiration == 0
+        ? Optional.empty()
+        : Optional.of(
+            OffsetDateTime.ofInstant(
+                new Date(expiration).toInstant(),
+                ZoneOffset.UTC));
+  }
+
+  public String getRoleARN() {
+    return roleARN;
+  }
+
+  public void setRoleARN(String roleARN) {
+    this.roleARN = requireNonNull(roleARN);
+  }
+
+  public void setAccessKey(final String accessKey) {
+    this.accessKey = requireNonNull(accessKey, "access key");
+  }
+
+  public void setSecretKey(final String secretKey) {
+    this.secretKey = requireNonNull(secretKey, "secret key");
+  }
+
+  public void setSessionToken(final String sessionToken) {
+    this.sessionToken = requireNonNull(sessionToken, "session token");
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MarshalledCredentials that = (MarshalledCredentials) o;
+    return expiration == that.expiration &&
+        Objects.equals(accessKey, that.accessKey) &&
+        Objects.equals(secretKey, that.secretKey) &&
+        Objects.equals(sessionToken, that.sessionToken) &&
+        Objects.equals(roleARN, that.roleARN);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(accessKey, secretKey, sessionToken, roleARN,
+        expiration);
+  }
+
+  /**
+   * String value MUST NOT include any secrets.
+   * @return a string value for logging.
+   */
+  @Override
+  public String toString() {
+    if (isEmpty()) {
+      return "Empty credentials";
+    }
+
+    String validity = isValid(CredentialTypeRequired.AnyNonEmpty)
+        ? "valid"
+        : "invalid";
+    if (!hasSessionToken()) {
+      // full credentials have the simplest string value.
+      return "full credentials (" + validity + ")";
+    } else {
+      // session/role credentials may have an expiry and role ARN.
+      return String.format("session credentials, expiry %s; %s(%s)",
+          getExpirationDateTime()
+              .map(x -> x.format(DateTimeFormatter.ISO_DATE))
+              .orElse("unknown"),
+          (isNotEmpty(roleARN)
+              ? ("role \"" + roleARN + "\" ")
+              : ""),
+          validity);
+    }
+  }
+
+  /**
+   * Is this empty: does it contain any credentials at all?
+   * This test returns true if either the access key or secret key is empty.
+   * @return true if there are no credentials.
+   */
+  public boolean isEmpty() {
+    return !(isNotEmpty(accessKey) && isNotEmpty(secretKey));
+  }
+
+  /**
+   * Is this a valid set of credentials tokens?
+   * @param required credential type required.
+   * @return true if the requirements are met.
+   */
+  public boolean isValid(final CredentialTypeRequired required) {
+    if (accessKey == null || secretKey == null || sessionToken == null) {
+      // null fields are not permitted, empty is OK for marshalling around.
+      return false;
+    }
+    // now look at whether values are set/unset.
+    boolean hasAccessAndSecretKeys = isNotEmpty(accessKey)
+        && isNotEmpty(secretKey);
+    boolean hasSessionToken = hasSessionToken();
+    switch (required) {
+
+    case AnyIncludingEmpty:
+      // this is simplest.
+      return true;
+
+    case Empty:
+      // empty. ignore session value if the other keys are unset.
+      return !hasAccessAndSecretKeys;
+
+    case AnyNonEmpty:
+      // just look for the access key and secret key being non-empty
+      return hasAccessAndSecretKeys;
+
+    case FullOnly:
+      return hasAccessAndSecretKeys && !hasSessionToken;
+
+    case SessionOnly:
+      return hasAccessAndSecretKeys && hasSessionToken();
+
+      // this is here to keep the IDE quiet
+    default:
+      return false;
+    }
+  }
+
+  /**
+   * Does this set of credentials have a session token.
+   * @return true if there's a session token.
+   */
+  public boolean hasSessionToken() {
+    return isNotEmpty(sessionToken);
+  }
+
+  /**
+   * Write the token.
+   * Only works if valid.
+   * @param out stream to serialize to.
+   * @throws IOException if the serialization failed.
+   */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    validate("Writing " + this + ": ",
+        CredentialTypeRequired.AnyIncludingEmpty);
+    Text.writeString(out, accessKey);
+    Text.writeString(out, secretKey);
+    Text.writeString(out, sessionToken);
+    Text.writeString(out, roleARN);
+    out.writeLong(expiration);
+  }
+
+  /**
+   * Read in the fields.
+   * @throws IOException IO problem
+   */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    accessKey = Text.readString(in, MAX_SECRET_LENGTH);
+    secretKey = Text.readString(in, MAX_SECRET_LENGTH);
+    sessionToken = Text.readString(in, MAX_SECRET_LENGTH);
+    roleARN = Text.readString(in, MAX_SECRET_LENGTH);
+    expiration = in.readLong();
+  }
+
+  /**
+   * Verify that a set of credentials is valid.
+   * @throws DelegationTokenIOException if they aren't
+   * @param message message to prefix errors;
+   * @param typeRequired credential type required.
+   */
+  public void validate(final String message,
+      final CredentialTypeRequired typeRequired) throws IOException {
+    if (!isValid(typeRequired)) {
+      throw new DelegationTokenIOException(message
+          + buildInvalidCredentialsError(typeRequired));
+    }
+  }
+
+  /**
+   * Build an error string for when the credentials do not match
+   * those required.
+   * @param typeRequired credential type required.
+   * @return an error string.
+   */
+  public String buildInvalidCredentialsError(
+      final CredentialTypeRequired typeRequired) {
+    if (isEmpty()) {
+      return " " + MarshalledCredentialBinding.NO_AWS_CREDENTIALS;
+    } else {
+      return " " + INVALID_CREDENTIALS
+          + " in " + toString() + " required: " + typeRequired;
+    }
+  }
+
+  /**
+   * Patch a configuration with the secrets.
+   * This does not set any per-bucket options (it doesn't know the bucket...).
+   * <i>Warning: once done the configuration must be considered sensitive.</i>
+   * @param config configuration to patch
+   */
+  public void setSecretsInConfiguration(Configuration config) {
+    config.set(ACCESS_KEY, accessKey);
+    config.set(SECRET_KEY, secretKey);
+    S3AUtils.setIfDefined(config, SESSION_TOKEN, sessionToken,
+        "session credentials");
+  }
+
+
+  /**
+   * Return a set of empty credentials.
+   * These can be marshalled, but not used for login.
+   * @return a new set of credentials.
+   */
+  public static MarshalledCredentials empty() {
+    return new MarshalledCredentials("", "", "");
+  }
+
+  /**
+   * Enumeration of credential types for use in validation methods.
+   */
+  public enum CredentialTypeRequired {
+    /** No entry at all. */
+    Empty("None"),
+    /** Any credential type including "unset". */
+    AnyIncludingEmpty("Full, Session or None"),
+    /** Any credential type is OK. */
+    AnyNonEmpty("Full or Session"),
+    /** The credentials must be session or role credentials. */
+    SessionOnly("Session"),
+    /** Full credentials are required. */
+    FullOnly("Full");
+
+    private final String text;
+
+    CredentialTypeRequired(final String text) {
+      this.text = text;
+    }
+
+    public String getText() {
+      return text;
+    }
+
+    @Override
+    public String toString() {
+      return getText();
+    }
+  }
+}
index f48e17a..7ec13b0 100644 (file)
 
 package org.apache.hadoop.fs.s3a.auth;
 
-import com.amazonaws.AmazonClientException;
+import org.apache.hadoop.fs.s3a.CredentialInitializationException;
 
 /**
- * A specific subclass of {@code AmazonClientException} which can
- * be used in the retry logic to fail fast when there is any
+ * A specific subclass of {@code AmazonClientException} which is
+ * used in the S3A retry policy to fail fast when there is any
  * authentication problem.
  */
-public class NoAuthWithAWSException extends AmazonClientException {
+public class NoAuthWithAWSException extends CredentialInitializationException {
 
   public NoAuthWithAWSException(final String message, final Throwable t) {
     super(message, t);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAwsCredentialsException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAwsCredentialsException.java
new file mode 100644 (file)
index 0000000..bff5f27
--- /dev/null
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth;
+
+import javax.annotation.Nonnull;
+
+/**
+ * A special exception which declares that no credentials were found;
+ * this can be treated specially in logging, handling, etc.
+ * As it subclasses {@link NoAuthWithAWSException}, the S3A retry handler
+ * knows not to attempt to ask for the credentials again.
+ */
+public class NoAwsCredentialsException extends
+    NoAuthWithAWSException {
+
+  /**
+   * The default error message: {@value}.
+   */
+  public static final String E_NO_AWS_CREDENTIALS = "No AWS Credentials";
+
+  /**
+   * Construct.
+   * @param credentialProvider name of the credential provider.
+   * @param message message.
+   */
+  public NoAwsCredentialsException(
+      @Nonnull final String credentialProvider,
+      @Nonnull final String message) {
+    this(credentialProvider, message, null);
+  }
+
+  /**
+   * Construct with the default message of {@link #E_NO_AWS_CREDENTIALS}.
+   * @param credentialProvider name of the credential provider.
+   */
+  public NoAwsCredentialsException(
+      @Nonnull final String credentialProvider) {
+    this(credentialProvider, E_NO_AWS_CREDENTIALS, null);
+  }
+
+  /**
+   * Construct with exception.
+   * @param credentialProvider name of the credential provider.
+   * @param message message.
+   * @param thrown inner exception
+   */
+  public NoAwsCredentialsException(
+      @Nonnull final String credentialProvider,
+      @Nonnull final String message,
+      final Throwable thrown) {
+    super(credentialProvider + ": " + message, thrown);
+  }
+}
index d4568b0..1082e93 100644 (file)
@@ -19,7 +19,7 @@
 package org.apache.hadoop.fs.s3a.auth;
 
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
@@ -35,8 +35,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.util.JsonSerialization;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
 
 /**
  * Jackson Role Model for Role Properties, for API clients and tests.
@@ -175,6 +175,21 @@ public class RoleModel {
 
   /**
    * Create a statement.
+   * @param allow allow or deny
+   * @param scope scope
+   * @param actions actions
+   * @return the formatted json statement
+   */
+  public static Statement statement(boolean allow,
+      String scope,
+      Collection<String> actions) {
+    return new Statement(RoleModel.effect(allow))
+        .addActions(actions)
+        .addResources(scope);
+  }
+
+  /**
+   * Create a statement.
    * If {@code isDirectory} is true, a "/" is added to the path.
    * This is critical when adding wildcard permissions under
    * a directory, and also needed when locking down dir-as-file
@@ -197,6 +212,29 @@ public class RoleModel {
   }
 
   /**
+   * Create a statement.
+   * If {@code isDirectory} is true, a "/" is added to the path.
+   * This is critical when adding wildcard permissions under
+   * a directory, and also needed when locking down dir-as-file
+   * and dir-as-directory-marker access.
+   * @param allow allow or deny
+   * @param path path
+   * @param isDirectory is this a directory?
+   * @param actions action
+   * @return the formatted json statement
+   */
+  public static Statement statement(
+      final boolean allow,
+      final Path path,
+      final boolean isDirectory,
+      final boolean wildcards,
+      final Collection<String> actions) {
+    return new Statement(RoleModel.effect(allow))
+        .addActions(actions)
+        .addResources(resource(path, isDirectory, wildcards));
+  }
+
+  /**
    * From a set of statements, create a policy.
    * @param statements statements
    * @return the policy
@@ -264,8 +302,8 @@ public class RoleModel {
 
     @Override
     public void validate() {
-      checkNotNull(sid, "Sid");
-      checkNotNull(effect, "Effect");
+      requireNonNull(sid, "Sid");
+      requireNonNull(effect, "Effect");
       checkState(!(action.isEmpty()), "Empty Action");
       checkState(!(resource.isEmpty()), "Empty Resource");
     }
@@ -280,11 +318,25 @@ public class RoleModel {
       return this;
     }
 
+    public Statement addActions(Collection<String> actions) {
+      action.addAll(actions);
+      return this;
+    }
+
     public Statement addResources(String... resources) {
       Collections.addAll(resource, resources);
       return this;
     }
 
+    /**
+     * Add a list of resources.
+     * @param resources resource list
+     * @return this statement.
+     */
+    public Statement addResources(Collection<String> resources) {
+      resource.addAll(resources);
+      return this;
+    }
   }
 
   /**
@@ -298,12 +350,20 @@ public class RoleModel {
     @JsonProperty("Statement")
     public List<Statement> statement;
 
+    /**
+     * Empty constructor: initializes the statements to an empty list.
+     */
+    public Policy() {
+      statement = new ArrayList<>();
+    }
+
     public Policy(final List<RoleModel.Statement> statement) {
       this.statement = statement;
     }
 
     public Policy(RoleModel.Statement... statements) {
-      statement = Arrays.asList(statements);
+      statement = new ArrayList<>(statements.length);
+      Collections.addAll(statement, statements);
     }
 
     /**
@@ -311,11 +371,34 @@ public class RoleModel {
      */
     @Override
     public void validate() {
-      checkNotNull(statement, "Statement");
+      requireNonNull(statement, "Statement");
       checkState(VERSION.equals(version), "Invalid Version: %s", version);
       statement.stream().forEach((a) -> a.validate());
     }
 
+    /**
+     * Add the statements of another policy to this one.
+     * @param other other policy.
+     */
+    public void add(Policy other) {
+      add(other.statement);
+    }
+
+    /**
+     * Add a collection of statements.
+     * @param statements statements to add.
+     */
+    public void add(Collection<Statement> statements) {
+      statement.addAll(statements);
+    }
+
+    /**
+     * Add a single statement.
+     * @param stat new statement.
+     */
+    public void add(Statement stat) {
+      statement.add(stat);
+    }
   }
 
 
index 34ed295..610dbcc 100644 (file)
 
 package org.apache.hadoop.fs.s3a.auth;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
 
 /**
  * Operations, statements and policies covering the operations
  * needed to work with S3 and S3Guard.
  */
+@InterfaceAudience.LimitedPrivate("Tests")
+@InterfaceStability.Unstable
 public final class RolePolicies {
 
   private RolePolicies() {
@@ -88,27 +100,36 @@ public final class RolePolicies {
    */
   public static final String S3_ALL_BUCKETS = "arn:aws:s3:::*";
 
+  /**
+   * All bucket list operations, including
+   * {@link #S3_BUCKET_LIST_BUCKET} and
+   * {@link #S3_BUCKET_LIST_MULTIPART_UPLOADS}.
+   */
+  public static final String S3_BUCKET_ALL_LIST = "s3:ListBucket*";
 
-  public static final String S3_ALL_LIST_OPERATIONS = "s3:List*";
-
-  public static final String S3_ALL_LIST_BUCKET = "s3:ListBucket*";
-
-  public static final String S3_LIST_BUCKET = "s3:ListBucket";
+  /**
+   * List the contents of a bucket.
+   * It applies to a bucket, not to a path in a bucket.
+   */
+  public static final String S3_BUCKET_LIST_BUCKET = "s3:ListBucket";
 
   /**
    * This is used by the abort operation in S3A commit work.
+   * It applies to a bucket, not to a path in a bucket.
    */
-  public static final String S3_LIST_BUCKET_MULTPART_UPLOADS =
+  public static final String S3_BUCKET_LIST_MULTIPART_UPLOADS =
       "s3:ListBucketMultipartUploads";
 
   /**
    * List multipart upload is needed for the S3A Commit protocols.
+   * It applies to a path in a bucket.
    */
   public static final String S3_LIST_MULTIPART_UPLOAD_PARTS
       = "s3:ListMultipartUploadParts";
 
   /**
-   * abort multipart upload is needed for the S3A Commit protocols.
+   * Abort multipart upload is needed for the S3A Commit protocols.
+   * It applies to a path in a bucket.
    */
   public static final String S3_ABORT_MULTIPART_UPLOAD
       = "s3:AbortMultipartUpload";
@@ -181,20 +202,41 @@ public final class RolePolicies {
    * Actions needed to read a file in S3 through S3A, excluding
    * S3Guard and SSE-KMS.
    */
-  public static final String[] S3_PATH_READ_OPERATIONS =
+  private static final String[] S3_PATH_READ_OPERATIONS =
       new String[]{
           S3_GET_OBJECT,
       };
 
   /**
    * Base actions needed to read data from S3 through S3A,
-   * excluding SSE-KMS data and S3Guard-ed buckets.
+   * excluding:
+   * <ol>
+   *   <li>bucket-level operations</li>
+   *   <li>SSE-KMS key operations</li>
+   *   <li>DynamoDB operations for S3Guard.</li>
+   * </ol>
+   * As this excludes the bucket list operations, it is not sufficient
+   * to read from a bucket on its own.
+   */
+  private static final String[] S3_ROOT_READ_OPERATIONS =
+      new String[]{
+          S3_ALL_GET,
+      };
+
+  public static final List<String> S3_ROOT_READ_OPERATIONS_LIST =
+      Collections.unmodifiableList(Arrays.asList(S3_ALL_GET));
+
+  /**
+   * Policies which can be applied to bucket resources for read operations.
+   * <ol>
+   *   <li>SSE-KMS key operations</li>
+   *   <li>DynamoDB operations for S3Guard.</li>
+   * </ol>
    */
-  public static final String[] S3_ROOT_READ_OPERATIONS =
+  public static final String[] S3_BUCKET_READ_OPERATIONS =
       new String[]{
-          S3_LIST_BUCKET,
-          S3_LIST_BUCKET_MULTPART_UPLOADS,
           S3_ALL_GET,
+          S3_BUCKET_ALL_LIST,
       };
 
   /**
@@ -202,43 +244,39 @@ public final class RolePolicies {
    * This includes the appropriate read operations, but
    * not SSE-KMS or S3Guard support.
    */
-  public static final String[] S3_PATH_RW_OPERATIONS =
-      new String[]{
+  public static final List<String> S3_PATH_RW_OPERATIONS =
+      Collections.unmodifiableList(Arrays.asList(new String[]{
           S3_ALL_GET,
           S3_PUT_OBJECT,
           S3_DELETE_OBJECT,
           S3_ABORT_MULTIPART_UPLOAD,
-          S3_LIST_MULTIPART_UPLOAD_PARTS,
-      };
+      }));
 
   /**
    * Actions needed to write data to an S3A Path.
    * This is purely the extra operations needed for writing atop
    * of the read operation set.
    * Deny these and a path is still readable, but not writeable.
-   * Excludes: SSE-KMS and S3Guard permissions.
+   * Excludes: bucket-ARN, SSE-KMS and S3Guard permissions.
    */
-  public static final String[] S3_PATH_WRITE_OPERATIONS =
-      new String[]{
+  public static final List<String> S3_PATH_WRITE_OPERATIONS =
+      Collections.unmodifiableList(Arrays.asList(new String[]{
           S3_PUT_OBJECT,
           S3_DELETE_OBJECT,
           S3_ABORT_MULTIPART_UPLOAD
-      };
+      }));
 
   /**
    * Actions needed for R/W IO from the root of a bucket.
-   * Excludes: SSE-KMS and S3Guard permissions.
+   * Excludes: bucket-ARN, SSE-KMS and S3Guard permissions.
    */
-  public static final String[] S3_ROOT_RW_OPERATIONS =
-      new String[]{
-          S3_LIST_BUCKET,
+  public static final List<String> S3_ROOT_RW_OPERATIONS =
+      Collections.unmodifiableList(Arrays.asList(new String[]{
           S3_ALL_GET,
           S3_PUT_OBJECT,
           S3_DELETE_OBJECT,
           S3_ABORT_MULTIPART_UPLOAD,
-          S3_LIST_MULTIPART_UPLOAD_PARTS,
-          S3_ALL_LIST_BUCKET,
-      };
+      }));
 
   /**
    * All DynamoDB operations: {@value}.
@@ -300,24 +338,15 @@ public final class RolePolicies {
   /**
    * Statement to allow all DDB access.
    */
-  public static final Statement STATEMENT_ALL_DDB = statement(true,
-      ALL_DDB_TABLES, DDB_ALL_OPERATIONS);
+  public static final Statement STATEMENT_ALL_DDB =
+      allowAllDynamoDBOperations(ALL_DDB_TABLES);
 
   /**
    * Statement to allow all client operations needed for S3Guard,
    * but none of the admin operations.
    */
-  public static final Statement STATEMENT_S3GUARD_CLIENT = statement(true,
-      ALL_DDB_TABLES,
-      DDB_BATCH_GET_ITEM,
-      DDB_BATCH_WRITE_ITEM,
-      DDB_DELETE_ITEM,
-      DDB_DESCRIBE_TABLE,
-      DDB_GET_ITEM,
-      DDB_PUT_ITEM,
-      DDB_QUERY,
-      DDB_UPDATE_ITEM
-      );
+  public static final Statement STATEMENT_S3GUARD_CLIENT =
+      allowS3GuardClientOperations(ALL_DDB_TABLES);
 
   /**
    * Allow all S3 Operations.
@@ -328,12 +357,91 @@ public final class RolePolicies {
       S3_ALL_OPERATIONS);
 
   /**
+   * The s3:GetBucketLocation permission is for all buckets, not for
+   * any named bucket, which complicates permissions.
+   */
+  public static final Statement STATEMENT_ALL_S3_GET_BUCKET_LOCATION =
+      statement(true,
+          S3_ALL_BUCKETS,
+          S3_GET_BUCKET_LOCATION);
+
+  /**
    * Policy for all S3 and S3Guard operations, and SSE-KMS.
    */
   public static final Policy ALLOW_S3_AND_SGUARD = policy(
       STATEMENT_ALL_S3,
       STATEMENT_ALL_DDB,
-      STATEMENT_ALLOW_SSE_KMS_RW
+      STATEMENT_ALLOW_SSE_KMS_RW,
+      STATEMENT_ALL_S3_GET_BUCKET_LOCATION
   );
 
+  public static Statement allowS3GuardClientOperations(String tableArn) {
+    return statement(true,
+        tableArn,
+        DDB_BATCH_GET_ITEM,
+        DDB_BATCH_WRITE_ITEM,
+        DDB_DELETE_ITEM,
+        DDB_DESCRIBE_TABLE,
+        DDB_GET_ITEM,
+        DDB_PUT_ITEM,
+        DDB_QUERY,
+        DDB_UPDATE_ITEM
+    );
+  }
+
+  public static Statement allowAllDynamoDBOperations(String tableArn) {
+    return statement(true,
+        tableArn,
+        DDB_ALL_OPERATIONS);
+  }
+
+  /**
+   * From an S3 bucket name, build an ARN to refer to it.
+   * @param bucket bucket name.
+   * @param write are write permissions required
+   * @return return statement granting access.
+   */
+  public static List<Statement> allowS3Operations(String bucket,
+      boolean write) {
+    // add the bucket operations for the specific bucket ARN
+    ArrayList<Statement> statements =
+        Lists.newArrayList(
+            statement(true,
+                bucketToArn(bucket),
+                S3_GET_BUCKET_LOCATION, S3_BUCKET_ALL_LIST));
+    // then add the statements for objects in the buckets
+    if (write) {
+      statements.add(
+          statement(true,
+              bucketObjectsToArn(bucket),
+              S3_ROOT_RW_OPERATIONS));
+    } else {
+      statements.add(
+          statement(true,
+              bucketObjectsToArn(bucket),
+              S3_ROOT_READ_OPERATIONS_LIST));
+    }
+    return statements;
+  }
+
+  /**
+   * From an S3 bucket name, build an ARN to refer to all objects in
+   * it.
+   * @param bucket bucket name.
+   * @return return the ARN to use in statements.
+   */
+  public static String bucketObjectsToArn(String bucket) {
+    return String.format("arn:aws:s3:::%s/*", bucket);
+  }
+
+
+  /**
+   * From an S3 bucket name, build an ARN to refer to it.
+   * @param bucket bucket name.
+   * @return return the ARN to use in statements.
+   */
+  public static String bucketToArn(String bucket) {
+    return String.format("arn:aws:s3:::%s", bucket);
+  }
+
 }
index 10bf88c..74aca50 100644 (file)
 
 package org.apache.hadoop.fs.s3a.auth;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
 import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
+import com.amazonaws.services.securitytoken.model.AssumeRoleRequest;
+import com.amazonaws.services.securitytoken.model.Credentials;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
+
 /**
  * Factory for creating STS Clients.
  */
@@ -48,31 +59,179 @@ public class STSClientFactory {
    * Create the builder ready for any final configuration options.
    * Picks up connection settings from the Hadoop configuration, including
    * proxy secrets.
+   * The endpoint comes from the configuration options
+   * {@link org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants#DELEGATION_TOKEN_ENDPOINT}
+   * and
+   * {@link org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants#DELEGATION_TOKEN_REGION}
+   * @param conf Configuration to act as source of options.
+   * @param bucket Optional bucket to use to look up per-bucket proxy secrets
+   * @param credentials AWS credential chain to use
+   * @return the builder to call {@code build()}
+   * @throws IOException problem reading proxy secrets
+   */
+  public static AWSSecurityTokenServiceClientBuilder builder(
+      final Configuration conf,
+      final String bucket,
+      final AWSCredentialsProvider credentials) throws IOException {
+    final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
+    String endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
+        DEFAULT_DELEGATION_TOKEN_ENDPOINT);
+    String region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
+        DEFAULT_DELEGATION_TOKEN_REGION);
+    return builder(credentials, awsConf, endpoint, region);
+  }
+
+  /**
+   * Create the builder ready for any final configuration options.
+   * Picks up connection settings from the Hadoop configuration, including
+   * proxy secrets.
    * @param conf Configuration to act as source of options.
    * @param bucket Optional bucket to use to look up per-bucket proxy secrets
    * @param credentials AWS credential chain to use
    * @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
-   * @param stsRegion the region, e.g "us-west-1"
+   * @param stsRegion AWS recommend setting the endpoint instead.
    * @return the builder to call {@code build()}
    * @throws IOException problem reading proxy secrets
    */
   public static AWSSecurityTokenServiceClientBuilder builder(
       final Configuration conf,
       final String bucket,
-      final AWSCredentialsProvider credentials, final String stsEndpoint,
+      final AWSCredentialsProvider credentials,
+      final String stsEndpoint,
       final String stsRegion) throws IOException {
-    Preconditions.checkArgument(credentials != null, "No credentials");
+    final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
+    return builder(credentials, awsConf, stsEndpoint, stsRegion);
+  }
+
+  /**
+   * Create the builder ready for any final configuration options.
+   * Picks up connection settings from the Hadoop configuration, including
+   * proxy secrets.
+   * @param awsConf AWS configuration.
+   * @param credentials AWS credential chain to use
+   * @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
+   * @param stsRegion the region, e.g "us-west-1". Must be set if endpoint is.
+   * @return the builder to call {@code build()}
+   */
+  public static AWSSecurityTokenServiceClientBuilder builder(
+      final AWSCredentialsProvider credentials,
+      final ClientConfiguration awsConf,
+      final String stsEndpoint,
+      final String stsRegion) {
     final AWSSecurityTokenServiceClientBuilder builder
         = AWSSecurityTokenServiceClientBuilder.standard();
-    final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket);
+    Preconditions.checkArgument(credentials != null, "No credentials");
     builder.withClientConfiguration(awsConf);
     builder.withCredentials(credentials);
-    if (StringUtils.isNotEmpty(stsEndpoint)) {
-      LOG.debug("STS Endpoint ={}", stsEndpoint);
+    boolean destIsStandardEndpoint = STS_STANDARD.equals(stsEndpoint);
+    if (isNotEmpty(stsEndpoint) && !destIsStandardEndpoint) {
+      Preconditions.checkArgument(
+          isNotEmpty(stsRegion),
+          "STS endpoint is set to %s but no signing region was provided",
+          stsEndpoint);
+      LOG.debug("STS Endpoint={}; region='{}'", stsEndpoint, stsRegion);
       builder.withEndpointConfiguration(
           new AwsClientBuilder.EndpointConfiguration(stsEndpoint, stsRegion));
+    } else {
+      Preconditions.checkArgument(isEmpty(stsRegion),
+          "STS signing region set set to %s but no STS endpoint specified",
+          stsRegion);
     }
     return builder;
   }
 
+  /**
+   * Create an STS Client instance.
+   * @param tokenService STS instance
+   * @param invoker invoker to use
+   * @return an STS client bonded to that interface.
+   * @throws IOException on any failure
+   */
+  public static STSClient createClientConnection(
+      final AWSSecurityTokenService tokenService,
+      final Invoker invoker)
+      throws IOException {
+    return new STSClient(tokenService, invoker);
+  }
+
+  /**
+   * STS client connection with retries.
+   */
+  public static final class STSClient implements Closeable {
+
+    private final AWSSecurityTokenService tokenService;
+
+    private final Invoker invoker;
+
+    private STSClient(final AWSSecurityTokenService tokenService,
+        final Invoker invoker) {
+      this.tokenService = tokenService;
+      this.invoker = invoker;
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        tokenService.shutdown();
+      } catch (UnsupportedOperationException ignored) {
+        // ignore this, as it is what the STS client currently
+        // does.
+      }
+    }
+
+    /**
+     * Request a set of session credentials.
+     *
+     * @param duration duration of the credentials
+     * @param timeUnit time unit of duration
+     * @return the role result
+     * @throws IOException on a failure of the request
+     */
+    @Retries.RetryTranslated
+    public Credentials requestSessionCredentials(
+        final long duration,
+        final TimeUnit timeUnit) throws IOException {
+      int durationSeconds = (int) timeUnit.toSeconds(duration);
+      LOG.debug("Requesting session token of duration {}", duration);
+      final GetSessionTokenRequest request = new GetSessionTokenRequest();
+      request.setDurationSeconds(durationSeconds);
+      return invoker.retry("request session credentials", "",
+          true,
+          () ->{
+            LOG.info("Requesting Amazon STS Session credentials");
+            return tokenService.getSessionToken(request).getCredentials();
+          });
+    }
+
+    /**
+     * Request a set of role credentials.
+     *
+     * @param roleARN ARN to request
+     * @param sessionName name of the session
+     * @param policy optional policy; "" is treated as "none"
+     * @param duration duration of the credentials
+     * @param timeUnit time unit of duration
+     * @return the role result
+     * @throws IOException on a failure of the request
+     */
+    @Retries.RetryTranslated
+    public Credentials requestRole(
+        final String roleARN,
+        final String sessionName,
+        final String policy,
+        final long duration,
+        final TimeUnit timeUnit) throws IOException {
+      LOG.debug("Requesting role {} with duration {}; policy = {}",
+          roleARN, duration, policy);
+      AssumeRoleRequest request = new AssumeRoleRequest();
+      request.setDurationSeconds((int) timeUnit.toSeconds(duration));
+      request.setRoleArn(roleARN);
+      request.setRoleSessionName(sessionName);
+      if (isNotEmpty(policy)) {
+        request.setPolicy(policy);
+      }
+      return invoker.retry("request role credentials", "", true,
+          () -> tokenService.assumeRole(request).getCredentials());
+    }
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AWSPolicyProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AWSPolicyProvider.java
new file mode 100644 (file)
index 0000000..aaca10f
--- /dev/null
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth.delegation;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+
+/**
+ * Interface for providers of AWS policy for accessing data.
+ * This is used when building up the role permissions for a delegation
+ * token.
+ *
+ * The permissions requested are from the perspective of
+ * S3A filesystem operations on the data, <i>not</i> the simpler
+ * model of "permissions on the the remote service".
+ * As an example, to use S3Guard effectively, the client needs full CRUD
+ * access to the table, even for {@link AccessLevel#READ}.
+ */
+public interface AWSPolicyProvider {
+
+  /**
+   * Get the AWS policy statements required for accessing this service.
+   *
+   * @param access access level desired.
+   * @return a possibly empty list of statements to grant access at that
+   * level.
+   */
+  List<RoleModel.Statement> listAWSPolicyRules(Set<AccessLevel> access);
+
+  /**
+   * Access levels.
+   */
+  enum AccessLevel {
+    /** Filesystem data read operations. */
+    READ,
+    /** Data write, encryption, etc. */
+    WRITE,
+    /** Administration of the data, tables, etc. */
+    ADMIN,
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDTService.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDTService.java
new file mode 100644 (file)
index 0000000..dcb83c2
--- /dev/null
@@ -0,0 +1,154 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.net.URI;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This is the base class for both the delegation binding
+ * code and the back end service created; allows for
+ * shared methods across both.
+ *
+ * The lifecycle sequence is as follows
+ * <pre>
+ *   - create
+ *   - bindToFileSystem(uri, ownerFS)
+ *   - init
+ *   - start
+ *   ...api calls...
+ *   - stop
+ * </pre>
+ *
+ * As the S3ADelegation mechanism is all configured during the filesystem
+ * initalize() operation, it is not ready for use through all the start process.
+ */
+public abstract class AbstractDTService
+    extends AbstractService {
+
+  /**
+   * URI of the filesystem.
+   * Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
+   */
+  private URI canonicalUri;
+
+  /**
+   * The owning filesystem.
+   * Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
+   */
+  private S3AFileSystem fileSystem;
+
+  /**
+   * Owner of the filesystem.
+   * Valid after {@link #bindToFileSystem(URI, S3AFileSystem)}.
+   */
+  private UserGroupInformation owner;
+
+  /**
+   * Protected constructor.
+   * @param name service name.
+   */
+  protected AbstractDTService(final String name) {
+    super(name);
+  }
+
+  /**
+   * Bind to the filesystem.
+   * Subclasses can use this to perform their own binding operations -
+   * but they must always call their superclass implementation.
+   * This <i>Must</i> be called before calling {@code init()}.
+   *
+   * <b>Important:</b>
+   * This binding will happen during FileSystem.initialize(); the FS
+   * is not live for actual use and will not yet have interacted with
+   * AWS services.
+   * @param uri the canonical URI of the FS.
+   * @param fs owning FS.
+   * @throws IOException failure.
+   */
+  public void bindToFileSystem(
+      final URI uri,
+      final S3AFileSystem fs) throws IOException {
+    requireServiceState(STATE.NOTINITED);
+    Preconditions.checkState(canonicalUri == null,
+        "bindToFileSystem called twice");
+    this.canonicalUri = requireNonNull(uri);
+    this.fileSystem = requireNonNull(fs);
+    this.owner = fs.getOwner();
+  }
+
+  /**
+   * Get the canonical URI of the filesystem, which is what is
+   * used to identify the tokens.
+   * @return the URI.
+   */
+  public URI getCanonicalUri() {
+    return canonicalUri;
+  }
+
+  /**
+   * Get the owner of the FS.
+   * @return the owner fs
+   */
+  protected S3AFileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  /**
+   * Get the owner of this Service.
+   * @return owner; non-null after binding to an FS.
+   */
+  public UserGroupInformation getOwner() {
+    return owner;
+  }
+
+  /**
+   * Require that the service is in a given state.
+   * @param state desired state.
+   * @throws IllegalStateException if the condition is not met
+   */
+  protected void requireServiceState(final STATE state)
+      throws IllegalStateException {
+    Preconditions.checkState(isInState(state),
+        "Required State: %s; Actual State %s", state, getServiceState());
+  }
+
+  /**
+   * Require the service to be started.
+   * @throws IllegalStateException if it is not.
+   */
+  protected void requireServiceStarted() throws IllegalStateException {
+    requireServiceState(STATE.STARTED);
+  }
+
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    requireNonNull(canonicalUri, "service does not have a canonical URI");
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationTokenBinding.java
new file mode 100644 (file)
index 0000000..73660ea
--- /dev/null
@@ -0,0 +1,305 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.Optional;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DURATION_LOG_AT_INFO;
+
+/**
+ *  An AbstractDelegationTokenBinding implementation is a class which
+ *  handles the binding of its underlying authentication mechanism to the
+ *  Hadoop Delegation token mechanism.
+ *
+ *  See also {@code org.apache.hadoop.fs.azure.security.WasbDelegationTokenManager}
+ *  but note that it assumes Kerberos tokens for which the renewal mechanism
+ *  is the sole plugin point.
+ *  This class is designed to be more generic.
+ *
+ *  <b>Lifecycle</b>
+ *
+ *  It is a Hadoop Service, so has a standard lifecycle: once started
+ *  its lifecycle will follow that of the {@link S3ADelegationTokens}
+ *  instance which created it --which itself follows the lifecycle of the FS.
+ *
+ *  One big difference is that
+ *  {@link #bindToFileSystem(URI, S3AFileSystem)} will be called
+ *  before the {@link #init(Configuration)} operation, this is where
+ *  the owning FS is passed in.
+ *
+ *  Implementations are free to start background operations in their
+ *  {@code serviceStart()} method, provided they are safely stopped in
+ *  {@code serviceStop()}.
+ *
+ *  <b>When to check for the ability to issue tokens</b>
+ *  Implementations MUST start up without actually holding the secrets
+ *  needed to issue tokens (config options, credentials to talk to STS etc)
+ *  as in server-side deployments they are not expected to have these.
+ *
+ *  <b>Retry Policy</b>
+ *
+ *  All methods which talk to AWS services are expected to do translation,
+ *  with retries as they see fit.
+ */
+public abstract class AbstractDelegationTokenBinding extends AbstractDTService {
+
+  /** Token kind: must match that of the token identifiers issued. */
+  private final Text kind;
+
+  private SecretManager<AbstractS3ATokenIdentifier> secretManager;
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractDelegationTokenBinding.class);
+
+  /**
+   * Constructor.
+   *
+   * @param name as passed to superclass for use in log messages.
+   * @param kind token kind.
+   */
+  protected AbstractDelegationTokenBinding(final String name,
+      final Text kind) {
+    super(name);
+    this.kind = requireNonNull(kind);
+  }
+
+  /**
+   * Get the kind of the tokens managed here.
+   * @return the token kind.
+   */
+  public Text getKind() {
+    return kind;
+  }
+
+  /**
+   * Return the name of the owner to be used in tokens.
+   * This may be that of the UGI owner, or it could be related to
+   * the AWS login.
+   * @return a text name of the owner.
+   */
+  public Text getOwnerText() {
+    return new Text(getOwner().getUserName());
+  }
+
+  /**
+   * Predicate: will this binding issue a DT?
+   * That is: should the filesystem declare that it is issuing
+   * delegation tokens? If true
+   * @return a declaration of what will happen when asked for a token.
+   */
+  public S3ADelegationTokens.TokenIssuingPolicy getTokenIssuingPolicy() {
+    return S3ADelegationTokens.TokenIssuingPolicy.RequestNewToken;
+  }
+
+  /**
+   * Create a delegation token for the user.
+   * This will only be called if a new DT is needed, that is: the
+   * filesystem has been deployed unbonded.
+   * @param policy minimum policy to use, if known.
+   * @param encryptionSecrets encryption secrets for the token.
+   * @return the token or null if the back end does not want to issue one.
+   * @throws IOException if one cannot be created
+   */
+  public Token<AbstractS3ATokenIdentifier> createDelegationToken(
+      final Optional<RoleModel.Policy> policy,
+      final EncryptionSecrets encryptionSecrets) throws IOException {
+    requireServiceStarted();
+    final AbstractS3ATokenIdentifier tokenIdentifier =
+            createTokenIdentifier(policy, encryptionSecrets);
+    if (tokenIdentifier != null) {
+      Token<AbstractS3ATokenIdentifier> token =
+          new Token<>(tokenIdentifier, secretManager);
+      token.setKind(getKind());
+      LOG.debug("Created token {} with token identifier {}",
+          token, tokenIdentifier);
+      return token;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Create a token identifier with all the information needed
+   * to be included in a delegation token.
+   * This is where session credentials need to be extracted, etc.
+   * This will only be called if a new DT is needed, that is: the
+   * filesystem has been deployed unbonded.
+   *
+   * If {@link #createDelegationToken(Optional, EncryptionSecrets)}
+   * is overridden, this method can be replaced with a stub.
+   *
+   * @param policy minimum policy to use, if known.
+   * @param encryptionSecrets encryption secrets for the token.
+   * @return the token data to include in the token identifier.
+   * @throws IOException failure creating the token data.
+   */
+  public abstract AbstractS3ATokenIdentifier createTokenIdentifier(
+      Optional<RoleModel.Policy> policy,
+      EncryptionSecrets encryptionSecrets) throws IOException;
+
+  /**
+   * Verify that a token identifier is of a specific class.
+   * This will reject subclasses (i.e. it is stricter than
+   * {@code instanceof}, then cast it to that type.
+   * @param identifier identifier to validate
+   * @param expectedClass class of the expected token identifier.
+   * @throws DelegationTokenIOException If the wrong class was found.
+   */
+  protected <T extends AbstractS3ATokenIdentifier> T convertTokenIdentifier(
+      final AbstractS3ATokenIdentifier identifier,
+      final Class<T> expectedClass) throws DelegationTokenIOException {
+    if (!identifier.getClass().equals(expectedClass)) {
+      throw new DelegationTokenIOException(
+          DelegationTokenIOException.TOKEN_WRONG_CLASS
+              + "; expected a token identifier of type "
+              + expectedClass
+              + " but got "
+              + identifier.getClass()
+              + " and kind " + identifier.getKind());
+    }
+    return (T) identifier;
+  }
+
+  /**
+   * Perform any actions when deploying unbonded, and return a list
+   * of credential providers.
+   * @return non-empty list of AWS credential providers to use for
+   * authenticating this client with AWS services.
+   * @throws IOException any failure.
+   */
+  public abstract AWSCredentialProviderList deployUnbonded()
+      throws IOException;
+
+  /**
+   * Bind to the token identifier, returning the credential providers to use
+   * for the owner to talk to S3, DDB and related AWS Services.
+   * @param retrievedIdentifier the unmarshalled data
+   * @return non-empty list of AWS credential providers to use for
+   * authenticating this client with AWS services.
+   * @throws IOException any failure.
+   */
+  public abstract AWSCredentialProviderList bindToTokenIdentifier(
+      AbstractS3ATokenIdentifier retrievedIdentifier)
+      throws IOException;
+
+  /**
+   * Create a new subclass of {@link AbstractS3ATokenIdentifier}.
+   * This is used in the secret manager.
+   * @return an empty identifier.
+   */
+  public abstract AbstractS3ATokenIdentifier createEmptyIdentifier();
+
+  @Override
+  public String toString() {
+    return super.toString()
+        + " token kind = " + getKind();
+  }
+
+  /**
+   * Service startup: create the secret manager.
+   * @throws Exception failure.
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    secretManager = createSecretMananger();
+  }
+
+  /**
+   * Return a description.
+   * This is logged during after service start and binding:
+   * it should be as informative as possible.
+   * @return a description to log.
+   */
+  public String getDescription() {
+    return "Token binding " + getKind().toString();
+  }
+
+  /**
+   * Create a secret manager.
+   * @return a secret manager.
+   * @throws IOException on failure
+   */
+  protected SecretManager<AbstractS3ATokenIdentifier> createSecretMananger()
+      throws IOException {
+    return new TokenSecretManager();
+  }
+
+  /**
+   * Return a string for use in building up the User-Agent field, so
+   * get into the S3 access logs. Useful for diagnostics.
+   * @return a string for the S3 logs or "" for "nothing to add"
+   */
+  public String getUserAgentField() {
+    return "";
+  }
+
+  /**
+   * Get the password to use in secret managers.
+   * This is a constant; its just recalculated every time to stop findbugs
+   * highlighting security risks of shared mutable byte arrays.
+   * @return a password.
+   */
+  protected static byte[] getSecretManagerPasssword() {
+    return "non-password".getBytes(Charset.forName("UTF-8"));
+  }
+
+  /**
+   * The secret manager always uses the same secret; the
+   * factory for new identifiers is that of the token manager.
+   */
+  protected class TokenSecretManager
+      extends SecretManager<AbstractS3ATokenIdentifier> {
+
+    @Override
+    protected byte[] createPassword(AbstractS3ATokenIdentifier identifier) {
+      return getSecretManagerPasssword();
+    }
+
+    @Override
+    public byte[] retrievePassword(AbstractS3ATokenIdentifier identifier)
+        throws InvalidToken {
+      return getSecretManagerPasssword();
+    }
+
+    @Override
+    public AbstractS3ATokenIdentifier createIdentifier() {
+      try (DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
+          "Creating Delegation Token Identifier")) {
+        return AbstractDelegationTokenBinding.this.createEmptyIdentifier();
+      }
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractS3ATokenIdentifier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractS3ATokenIdentifier.java
new file mode 100644 (file)
index 0000000..7c1c0e3
--- /dev/null
@@ -0,0 +1,305 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Objects;
+import java.util.UUID;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * An S3A Delegation Token Identifier: contains the information needed
+ * to talk to S3A.
+ *
+ * These are loaded via the service loader API an used in a map of
+ * Kind to class, which is then looked up to deserialize token
+ * identifiers of a given class.
+ *
+ * Every non-abstract class must provide
+ * <ol>
+ *   <li>Their unique token kind.</li>
+ *   <li>An empty constructor.</li>
+ *   <li>An entry in the resource file
+ *   {@code /META-INF/services/org.apache.hadoop.security.token.TokenIdentifier}
+ *   </li>
+ * </ol>
+ *
+ * The base implementation contains
+ * <ol>
+ *   <li>The URI of the FS.</li>
+ *   <li>Encryption secrets for use in the destination FS.</li>
+ * </ol>
+ * Subclasses are required to add whatever information is needed to authenticate
+ * the user with the credential provider which their binding class will
+ * provide.
+ *
+ * <i>Important: Add no references to any AWS SDK class, to
+ * ensure it can be safely deserialized whenever the relevant token
+ * identifier of a token type declared in this JAR is examined.</i>
+ */
+public abstract class AbstractS3ATokenIdentifier
+    extends DelegationTokenIdentifier {
+
+  /**
+   * The maximum string length supported for text fields.
+   */
+  protected static final int MAX_TEXT_LENGTH = 8192;
+
+  /** Canonical URI of the bucket. */
+  private URI uri;
+
+  /**
+   * Encryption secrets to also marshall with any credentials.
+   * Set during creation to ensure it is never null.
+   */
+  private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
+
+  /**
+   * Timestamp of creation.
+   * This is set to the current time; it will be overridden when
+   * deserializing data.
+   */
+  private long created = System.currentTimeMillis();
+
+  /**
+   * An origin string for diagnostics.
+   */
+  private String origin = "";
+
+  /**
+   * This marshalled UUID can be used in testing to verify transmission,
+   * and reuse; as it is printed you can see what is happending too.
+   */
+  private String uuid = UUID.randomUUID().toString();
+
+  /**
+   * Constructor.
+   * @param kind token kind.
+   * @param uri filesystem URI.
+   * @param owner token owner
+   * @param origin origin text for diagnostics.
+   * @param encryptionSecrets encryption secrets to set.
+   */
+  protected AbstractS3ATokenIdentifier(
+      final Text kind,
+      final URI uri,
+      final Text owner,
+      final String origin,
+      final EncryptionSecrets encryptionSecrets) {
+    this(kind, owner, new Text(), new Text(), uri);
+    this.origin = requireNonNull(origin);
+    this.encryptionSecrets = requireNonNull(encryptionSecrets);
+  }
+
+  /**
+   * Constructor.
+   * @param kind token kind.
+   * @param owner token owner
+   * @param renewer token renewer
+   * @param realUser token real user
+   * @param uri filesystem URI.
+   */
+  protected AbstractS3ATokenIdentifier(
+      final Text kind,
+      final Text owner,
+      final Text renewer,
+      final Text realUser,
+      final URI uri) {
+    super(kind, owner, renewer, realUser);
+    this.uri = requireNonNull(uri);
+  }
+
+  /**
+   * Build from a token.
+   * This has been written for refresh operations;
+   * if someone implements refresh it will be relevant.
+   * @param token to to build from
+   * @throws IOException failure to build the identifier.
+   */
+  protected AbstractS3ATokenIdentifier(
+      final Text kind,
+      final Token<AbstractS3ATokenIdentifier> token) throws IOException {
+    super(kind);
+    ByteArrayInputStream bais = new ByteArrayInputStream(token.getIdentifier());
+    readFields(new DataInputStream(bais));
+  }
+
+  /**
+   * For subclasses to use in their own empty-constructors.
+   */
+  protected AbstractS3ATokenIdentifier(final Text kind) {
+    super(kind);
+  }
+
+  public String getBucket() {
+    return uri.getHost();
+  }
+
+  public URI getUri() {
+    return uri;
+  }
+
+  public String getOrigin() {
+    return origin;
+  }
+
+  public void setOrigin(final String origin) {
+    this.origin = origin;
+  }
+
+  public long getCreated() {
+    return created;
+  }
+
+  /**
+   * Write state.
+   * {@link org.apache.hadoop.io.Writable#write(DataOutput)}.
+   * @param out destination
+   * @throws IOException failure
+   */
+  @Override
+  public void write(final DataOutput out) throws IOException {
+    super.write(out);
+    Text.writeString(out, uri.toString());
+    Text.writeString(out, origin);
+    Text.writeString(out, uuid);
+    encryptionSecrets.write(out);
+    out.writeLong(created);
+  }
+
+  /**
+   * Read state.
+   * {@link org.apache.hadoop.io.Writable#readFields(DataInput)}.
+   *
+   * Note: this operation gets called in toString() operations on tokens, so
+   * must either always succeed, or throw an IOException to trigger the
+   * catch and downgrade. RuntimeExceptions (e.g. Preconditions checks) are
+   * not to be used here for this reason.)
+   *
+   * @param in input stream
+   * @throws DelegationTokenIOException if the token binding is wrong.
+   * @throws IOException IO problems.
+   */
+  @Override
+  public void readFields(final DataInput in)
+      throws DelegationTokenIOException, IOException {
+    super.readFields(in);
+    uri = URI.create(Text.readString(in, MAX_TEXT_LENGTH));
+    origin = Text.readString(in, MAX_TEXT_LENGTH);
+    uuid = Text.readString(in, MAX_TEXT_LENGTH);
+    encryptionSecrets.readFields(in);
+    created = in.readLong();
+  }
+
+  /**
+   * Validate the token by looking at its fields.
+   * @throws IOException on failure.
+   */
+  public void validate() throws IOException {
+    if (uri == null) {
+      throw new DelegationTokenIOException("No URI in " + this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3ATokenIdentifier{");
+    sb.append(getKind());
+    sb.append("; uri=").append(uri);
+    sb.append("; timestamp=").append(created);
+    sb.append("; encryption=").append(encryptionSecrets.toString());
+    sb.append("; ").append(uuid);
+    sb.append("; ").append(origin);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Equality check is on superclass and UUID only.
+   * @param o other.
+   * @return true if the base class considers them equal and the URIs match.
+   */
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    final AbstractS3ATokenIdentifier that = (AbstractS3ATokenIdentifier) o;
+    return Objects.equals(uuid, that.uuid) &&
+        Objects.equals(uri, that.uri);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), uri);
+  }
+
+  /**
+   * Return the expiry time in seconds since 1970-01-01.
+   * @return the time when the session credential expire.
+   */
+  public long getExpiryTime() {
+    return 0;
+  }
+
+  /**
+   * Get the UUID of this token identifier.
+   * @return a UUID.
+   */
+  public String getUuid() {
+    return uuid;
+  }
+
+  /**
+   * Get the encryption secrets.
+   * @return the encryption secrets within this identifier.
+   */
+  public EncryptionSecrets getEncryptionSecrets() {
+    return encryptionSecrets;
+  }
+
+  /**
+   * Create the default origin text message with local hostname and
+   * timestamp.
+   * @return a string for token diagnostics.
+   */
+  public static String createDefaultOriginMessage() {
+    return String.format("Created on %s at time %s.",
+        NetUtils.getHostname(),
+        java.time.Instant.now());
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationConstants.java
new file mode 100644 (file)
index 0000000..7674c69
--- /dev/null
@@ -0,0 +1,165 @@
+/*
+ * 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.auth.delegation;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.io.Text;
+
+/**
+ * All the constants related to delegation tokens.
+ * Not in the normal S3 constants while unstable.
+ *
+ * Where possible, the existing assumed role properties are used to configure
+ * STS binding, default ARN, etc. This makes documenting everything that
+ * much easier and avoids trying to debug precisely which sts endpoint
+ * property should be set.
+ *
+ * Most settings here are replicated in {@code core-default.xml}; the
+ * values MUST be kept in sync.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class DelegationConstants {
+
+  /**
+   * Endpoint for session tokens, used when building delegation tokens:
+   * {@value}.
+   * @see <a href="https://docs.aws.amazon.com/general/latest/gr/rande.html#sts_region">STS regions</a>
+   */
+  public static final String DELEGATION_TOKEN_ENDPOINT =
+      Constants.ASSUMED_ROLE_STS_ENDPOINT;
+
+  /**
+   * Default endpoint for session tokens: {@value}.
+   */
+  public static final String DEFAULT_DELEGATION_TOKEN_ENDPOINT =
+      Constants.DEFAULT_ASSUMED_ROLE_STS_ENDPOINT;
+
+  /**
+   * Region for DT issuing; must be non-empty if the endpoint is set: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_REGION =
+      Constants.ASSUMED_ROLE_STS_ENDPOINT_REGION;
+
+  /**
+   * Region default: {@value}.
+   */
+  public static final String DEFAULT_DELEGATION_TOKEN_REGION =
+      Constants.ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT;
+
+  /**
+   * Duration of tokens in time: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_DURATION =
+      Constants.ASSUMED_ROLE_SESSION_DURATION;
+
+  /**
+   * Default duration of a delegation token: {@value}.
+   * Must be in the range supported by STS.
+   */
+  public static final String DEFAULT_DELEGATION_TOKEN_DURATION =
+      Constants.ASSUMED_ROLE_SESSION_DURATION_DEFAULT;
+
+  /**
+   * Key to list AWS credential providers for Session/role
+   * credentials: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_CREDENTIALS_PROVIDER =
+      Constants.AWS_CREDENTIALS_PROVIDER;
+
+  /**
+   * ARN of the delegation token: {@value}.
+   * Required for the role token.
+   */
+  public static final String DELEGATION_TOKEN_ROLE_ARN =
+      Constants.ASSUMED_ROLE_ARN;
+
+  /**
+   * Property containing classname for token binding: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_BINDING =
+      "fs.s3a.delegation.token.binding";
+  /**
+   * Session Token binding classname: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_SESSION_BINDING =
+      "org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding";
+
+  /**
+   * Default token binding {@value}.
+   */
+  public static final String DEFAULT_DELEGATION_TOKEN_BINDING = "";
+
+  /**
+   * Token binding to pass full credentials: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_FULL_CREDENTIALS_BINDING =
+      "org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding";
+
+  /**
+   * Role DTs: {@value}.
+   */
+  public static final String DELEGATION_TOKEN_ROLE_BINDING =
+      "org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding";
+
+  /** Prefix for token names: {@value}. */
+  public static final String TOKEN_NAME_PREFIX = "S3ADelegationToken/";
+
+  /** Name of session token: {@value}. */
+  public static final String SESSION_TOKEN_NAME = TOKEN_NAME_PREFIX + "Session";
+
+  /** Kind of the session token; value is {@link #SESSION_TOKEN_NAME}. */
+  public static final Text SESSION_TOKEN_KIND = new Text(SESSION_TOKEN_NAME);
+
+  /** Name of full token: {@value}. */
+  public static final String FULL_TOKEN_NAME = TOKEN_NAME_PREFIX + "Full";
+
+  /** Kind of the full token; value is {@link #FULL_TOKEN_NAME}. */
+  public static final Text FULL_TOKEN_KIND = new Text(FULL_TOKEN_NAME);
+
+  /** Name of role token: {@value}. */
+  public static final String ROLE_TOKEN_NAME = TOKEN_NAME_PREFIX + "Role";
+
+  /** Kind of the role token; value is {@link #ROLE_TOKEN_NAME}. */
+  public static final Text ROLE_TOKEN_KIND = new Text(ROLE_TOKEN_NAME);
+
+  /**
+   * Package-scoped option to control level that duration info on token
+   * binding operations are logged at.
+   * Value: {@value}.
+   */
+  static final boolean DURATION_LOG_AT_INFO = true;
+
+  /**
+   * If the token binding auth chain is only session-level auth, you
+   * can't use the role binding: {@value}.
+   */
+  public static final String E_NO_SESSION_TOKENS_FOR_ROLE_BINDING
+      = "Cannot issue S3A Role Delegation Tokens without full AWS credentials";
+
+  /**
+   * The standard STS server.
+   */
+  public static final String STS_STANDARD = "sts.amazonaws.com";
+
+  private DelegationConstants() {
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenIOException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenIOException.java
new file mode 100644 (file)
index 0000000..32d45bc
--- /dev/null
@@ -0,0 +1,50 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+
+/**
+ * General IOException for Delegation Token issues.
+ * Includes recommended error strings, which can be used in tests when
+ * looking for specific errors.
+ */
+public class DelegationTokenIOException extends IOException {
+
+  private static final long serialVersionUID = 599813827985340023L;
+
+  /** Error: delegation token/token identifier class isn't the right one. */
+  public static final String TOKEN_WRONG_CLASS
+      = "Delegation token is wrong class";
+
+  /**
+   * The far end is expecting a different token kind than
+   * that which the client created.
+   */
+  protected static final String TOKEN_MISMATCH = "Token mismatch";
+
+  public DelegationTokenIOException(final String message) {
+    super(message);
+  }
+
+  public DelegationTokenIOException(final String message,
+      final Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java
new file mode 100644 (file)
index 0000000..6526f9a
--- /dev/null
@@ -0,0 +1,73 @@
+/*
+ * 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.auth.delegation;
+
+import java.util.Optional;
+
+import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
+import com.amazonaws.services.s3.model.SSECustomerKey;
+
+import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
+
+/**
+ * These support operations on {@link EncryptionSecrets} which use the AWS SDK
+ * operations. Isolating them here ensures that that class is not required on
+ * the classpath.
+ */
+public final class EncryptionSecretOperations {
+
+  private EncryptionSecretOperations() {
+  }
+
+  /**
+   * Create SSE-C client side key encryption options on demand.
+   * @return an optional key to attach to a request.
+   * @param secrets source of the encryption secrets.
+   */
+  public static Optional<SSECustomerKey> createSSECustomerKey(
+      final EncryptionSecrets secrets) {
+    if (secrets.hasEncryptionKey() &&
+        secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
+      return Optional.of(new SSECustomerKey(secrets.getEncryptionKey()));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  /**
+   * Create SSE-KMS options for a request, iff the encryption is SSE-KMS.
+   * @return an optional SSE-KMS param to attach to a request.
+   * @param secrets source of the encryption secrets.
+   */
+  public static Optional<SSEAwsKeyManagementParams> createSSEAwsKeyManagementParams(
+      final EncryptionSecrets secrets) {
+
+    //Use specified key, otherwise default to default master aws/s3 key by AWS
+    if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS) {
+      if (secrets.hasEncryptionKey()) {
+        return Optional.of(new SSEAwsKeyManagementParams(
+            secrets.getEncryptionKey()));
+      } else {
+        return Optional.of(new SSEAwsKeyManagementParams());
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecrets.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecrets.java
new file mode 100644 (file)
index 0000000..092653d
--- /dev/null
@@ -0,0 +1,221 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Encryption options in a form which can serialized or marshalled as a hadoop
+ * Writeable.
+ *
+ * Maintainers: For security reasons, don't print any of this.
+ *
+ * Note this design marshalls/unmarshalls its serialVersionUID
+ * in its writable, which is used to compare versions.
+ *
+ * <i>Important.</i>
+ * If the wire format is ever changed incompatibly,
+ * update the serial version UID to ensure that older clients get safely
+ * rejected.
+ *
+ * <i>Important</i>
+ * Do not import any AWS SDK classes, directly or indirectly.
+ * This is to ensure that S3A Token identifiers can be unmarshalled even
+ * without that SDK.
+ */
+public class EncryptionSecrets implements Writable, Serializable {
+
+  public static final int MAX_SECRET_LENGTH = 2048;
+
+  private static final long serialVersionUID = 1208329045511296375L;
+
+  /**
+   * Encryption algorithm to use: must match one in
+   * {@link S3AEncryptionMethods}.
+   */
+  private String encryptionAlgorithm = "";
+
+  /**
+   * Encryption key: possibly sensitive information.
+   */
+  private String encryptionKey = "";
+
+  /**
+   * This field isn't serialized/marshalled; it is rebuilt from the
+   * encryptionAlgorithm field.
+   */
+  private transient S3AEncryptionMethods encryptionMethod =
+      S3AEncryptionMethods.NONE;
+
+  /**
+   * Empty constructor, for use in marshalling.
+   */
+  public EncryptionSecrets() {
+  }
+
+  /**
+   * Create a pair of secrets.
+   * @param encryptionAlgorithm algorithm enumeration.
+   * @param encryptionKey key/key reference.
+   * @throws IOException failure to initialize.
+   */
+  public EncryptionSecrets(final S3AEncryptionMethods encryptionAlgorithm,
+      final String encryptionKey) throws IOException {
+    this(encryptionAlgorithm.getMethod(), encryptionKey);
+  }
+
+  /**
+   * Create a pair of secrets.
+   * @param encryptionAlgorithm algorithm name
+   * @param encryptionKey key/key reference.
+   * @throws IOException failure to initialize.
+   */
+  public EncryptionSecrets(final String encryptionAlgorithm,
+      final String encryptionKey) throws IOException {
+    this.encryptionAlgorithm = encryptionAlgorithm;
+    this.encryptionKey = encryptionKey;
+    init();
+  }
+
+  /**
+   * Write out the encryption secrets.
+   * @param out {@code DataOutput} to serialize this object into.
+   * @throws IOException IO failure
+   */
+  @Override
+  public void write(final DataOutput out) throws IOException {
+    new LongWritable(serialVersionUID).write(out);
+    Text.writeString(out, encryptionAlgorithm);
+    Text.writeString(out, encryptionKey);
+  }
+
+  /**
+   * Read in from the writable stream.
+   * After reading, call {@link #init()}.
+   * @param in {@code DataInput} to deserialize this object from.
+   * @throws IOException failure to read/validate data.
+   */
+  @Override
+  public void readFields(final DataInput in) throws IOException {
+    final LongWritable version = new LongWritable();
+    version.readFields(in);
+    if (version.get() != serialVersionUID) {
+      throw new DelegationTokenIOException(
+          "Incompatible EncryptionSecrets version");
+    }
+    encryptionAlgorithm = Text.readString(in, MAX_SECRET_LENGTH);
+    encryptionKey = Text.readString(in, MAX_SECRET_LENGTH);
+    init();
+  }
+
+  /**
+   * For java serialization: read and then call {@link #init()}.
+   * @param in input
+   * @throws IOException IO problem
+   * @throws ClassNotFoundException problem loading inner class.
+   */
+  private void readObject(ObjectInputStream in)
+      throws IOException, ClassNotFoundException {
+    in.defaultReadObject();
+    init();
+  }
+
+  /**
+   * Init all state, including after any read.
+   * @throws IOException error rebuilding state.
+   */
+  private void init() throws IOException {
+    encryptionMethod = S3AEncryptionMethods.getMethod(
+        encryptionAlgorithm);
+  }
+
+  public String getEncryptionAlgorithm() {
+    return encryptionAlgorithm;
+  }
+
+  public String getEncryptionKey() {
+    return encryptionKey;
+  }
+
+  /**
+   * Does this instance have encryption options?
+   * That is: is the algorithm non-null.
+   * @return true if there's an encryption algorithm.
+   */
+  public boolean hasEncryptionAlgorithm() {
+    return StringUtils.isNotEmpty(encryptionAlgorithm);
+  }
+
+  /**
+   * Does this instance have an encryption key?
+   * @return true if there's an encryption key.
+   */
+  public boolean hasEncryptionKey() {
+    return StringUtils.isNotEmpty(encryptionKey);
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    final EncryptionSecrets that = (EncryptionSecrets) o;
+    return Objects.equals(encryptionAlgorithm, that.encryptionAlgorithm)
+        && Objects.equals(encryptionKey, that.encryptionKey);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(encryptionAlgorithm, encryptionKey);
+  }
+
+  /**
+   * Get the encryption method.
+   * @return the encryption method
+   */
+  public S3AEncryptionMethods getEncryptionMethod() {
+    return encryptionMethod;
+  }
+
+  /**
+   * String function returns the encryption mode but not any other
+   * secrets.
+   * @return a string safe for logging.
+   */
+  @Override
+  public String toString() {
+    return S3AEncryptionMethods.NONE.equals(encryptionMethod)
+        ? "(no encryption)"
+        : encryptionMethod.getMethod();
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenBinding.java
new file mode 100644 (file)
index 0000000..138667b
--- /dev/null
@@ -0,0 +1,172 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Optional;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.FULL_TOKEN_KIND;
+
+/**
+ * Full credentials: they are simply passed as-is, rather than
+ * converted to a session.
+ * These aren't as secure; this class exists to (a) support deployments
+ * where there is not STS service and (b) validate the design of
+ * S3A DT support to support different managers.
+ */
+public class FullCredentialsTokenBinding extends
+    AbstractDelegationTokenBinding {
+
+  /**
+   * Wire name of this binding includes a version marker: {@value}.
+   */
+  private static final String NAME = "FullCredentials/001";
+
+  public static final String FULL_TOKEN = "Full Delegation Token";
+
+  /**
+   * Long-lived AWS credentials.
+   */
+  private MarshalledCredentials awsCredentials;
+
+  /**
+   * Origin of credentials.
+   */
+  private String credentialOrigin;
+
+  /**
+   * Constructor, uses name of {@link #name} and token kind of
+   * {@link DelegationConstants#FULL_TOKEN_KIND}.
+   *
+   */
+  public FullCredentialsTokenBinding() {
+    super(NAME, FULL_TOKEN_KIND);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    loadAWSCredentials();
+  }
+
+  /**
+   * Load the AWS credentials.
+   * @throws IOException failure
+   */
+  private void loadAWSCredentials() throws IOException {
+    credentialOrigin = AbstractS3ATokenIdentifier.createDefaultOriginMessage();
+    Configuration conf = getConfig();
+    URI uri = getCanonicalUri();
+    // look for access keys to FS
+    S3xLoginHelper.Login secrets = S3AUtils.getAWSAccessKeys(uri, conf);
+    if (secrets.hasLogin()) {
+      awsCredentials = new MarshalledCredentials(
+          secrets.getUser(), secrets.getPassword(), "");
+      credentialOrigin += "; source = Hadoop configuration data";
+    } else {
+      // if there are none, look for the environment variables.
+      awsCredentials = MarshalledCredentialBinding.fromEnvironment(
+          System.getenv());
+      if (awsCredentials.isValid(
+          MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty)) {
+        // valid tokens, so mark as origin
+        credentialOrigin += "; source = Environment variables";
+      } else {
+        credentialOrigin = "no credentials in configuration or"
+            + " environment variables";
+      }
+    }
+    awsCredentials.validate(credentialOrigin +": ",
+        MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty);
+  }
+
+  /**
+   * Serve up the credentials retrieved from configuration/environment in
+   * {@link #loadAWSCredentials()}.
+   * @return a credential provider for the unbonded instance.
+   * @throws IOException failure to load
+   */
+  @Override
+  public AWSCredentialProviderList deployUnbonded() throws IOException {
+    requireServiceStarted();
+    return new AWSCredentialProviderList(
+        "Full Credentials Token Binding",
+        new MarshalledCredentialProvider(
+            FULL_TOKEN,
+            getFileSystem().getUri(),
+            getConfig(),
+            awsCredentials,
+            MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty));
+  }
+
+  /**
+   * Create a new delegation token.
+   *
+   * It's slightly inefficient to create a new one every time, but
+   * it avoids concurrency problems with managing any singleton.
+   * @param policy minimum policy to use, if known.
+   * @param encryptionSecrets encryption secrets.
+   * @return a DT identifier
+   * @throws IOException failure
+   */
+  @Override
+  public AbstractS3ATokenIdentifier createTokenIdentifier(
+      final Optional<RoleModel.Policy> policy,
+      final EncryptionSecrets encryptionSecrets) throws IOException {
+    requireServiceStarted();
+
+    return new FullCredentialsTokenIdentifier(getCanonicalUri(),
+        getOwnerText(),
+        awsCredentials,
+        encryptionSecrets,
+        credentialOrigin);
+  }
+
+  @Override
+  public AWSCredentialProviderList bindToTokenIdentifier(
+      final AbstractS3ATokenIdentifier retrievedIdentifier)
+      throws IOException {
+    FullCredentialsTokenIdentifier tokenIdentifier =
+        convertTokenIdentifier(retrievedIdentifier,
+            FullCredentialsTokenIdentifier.class);
+    return new AWSCredentialProviderList(
+        "", new MarshalledCredentialProvider(
+            FULL_TOKEN,
+            getFileSystem().getUri(),
+            getConfig(),
+            tokenIdentifier.getMarshalledCredentials(),
+            MarshalledCredentials.CredentialTypeRequired.AnyNonEmpty));
+  }
+
+  @Override
+  public AbstractS3ATokenIdentifier createEmptyIdentifier() {
+    return new FullCredentialsTokenIdentifier();
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenIdentifier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/FullCredentialsTokenIdentifier.java
new file mode 100644 (file)
index 0000000..95e4a28
--- /dev/null
@@ -0,0 +1,50 @@
+/*
+ * 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.auth.delegation;
+
+import java.net.URI;
+
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.io.Text;
+
+/**
+ * The full credentials payload is the same of that for a session token, but
+ * a different token kind is used.
+ *
+ * Token kind is {@link DelegationConstants#FULL_TOKEN_KIND}.
+ */
+public class FullCredentialsTokenIdentifier extends SessionTokenIdentifier {
+
+  public FullCredentialsTokenIdentifier() {
+    super(DelegationConstants.FULL_TOKEN_KIND);
+  }
+
+  public FullCredentialsTokenIdentifier(final URI uri,
+      final Text owner,
+      final MarshalledCredentials marshalledCredentials,
+      final EncryptionSecrets encryptionSecrets,
+      String origin) {
+    super(DelegationConstants.FULL_TOKEN_KIND,
+        owner,
+        uri,
+        marshalledCredentials,
+        encryptionSecrets,
+        origin);
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java
new file mode 100644 (file)
index 0000000..f436671
--- /dev/null
@@ -0,0 +1,176 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.services.securitytoken.model.Credentials;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
+
+import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_CREDENTIALS_PROVIDER;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_ROLE_ARN;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.E_NO_SESSION_TOKENS_FOR_ROLE_BINDING;
+
+/**
+ * Role Token support requests an explicit role and automatically restricts
+ * that role to the given policy of the binding.
+ * The session is locked down as much as possible.
+ */
+public class RoleTokenBinding extends SessionTokenBinding {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      RoleTokenBinding.class);
+
+  private static final RoleModel MODEL = new RoleModel();
+
+  /**
+   * Wire name of this binding includes a version marker: {@value}.
+   */
+  private static final String NAME = "RoleCredentials/001";
+
+  /**
+   * Error message when there is no Role ARN.
+   */
+  @VisibleForTesting
+  public static final String E_NO_ARN =
+      "No role ARN defined in " + DELEGATION_TOKEN_ROLE_ARN;
+
+  public static final String COMPONENT = "Role Delegation Token";
+
+  /**
+   * Role ARN to use when requesting new tokens.
+   */
+  private String roleArn;
+
+  /**
+   * Constructor.
+   * Name is {@link #name}; token kind is
+   * {@link DelegationConstants#ROLE_TOKEN_KIND}.
+   */
+  public RoleTokenBinding() {
+    super(NAME, DelegationConstants.ROLE_TOKEN_KIND);
+  }
+
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    roleArn = getConfig().getTrimmed(DELEGATION_TOKEN_ROLE_ARN, "");
+  }
+
+  /**
+   * Returns a (wrapped) {@link MarshalledCredentialProvider} which
+   * requires the marshalled credentials to contain session secrets.
+   * @param retrievedIdentifier the incoming identifier.
+   * @return the provider chain.
+   * @throws IOException on failure
+   */
+  @Override
+  public AWSCredentialProviderList bindToTokenIdentifier(
+      final AbstractS3ATokenIdentifier retrievedIdentifier)
+      throws IOException {
+    RoleTokenIdentifier tokenIdentifier =
+        convertTokenIdentifier(retrievedIdentifier,
+            RoleTokenIdentifier.class);
+    setTokenIdentifier(Optional.of(tokenIdentifier));
+    MarshalledCredentials marshalledCredentials
+        = tokenIdentifier.getMarshalledCredentials();
+    setExpirationDateTime(marshalledCredentials.getExpirationDateTime());
+    return new AWSCredentialProviderList(
+        "Role Token Binding",
+        new MarshalledCredentialProvider(
+            COMPONENT, getFileSystem().getUri(),
+            getConfig(),
+            marshalledCredentials,
+            MarshalledCredentials.CredentialTypeRequired.SessionOnly));
+  }
+
+  /**
+   * Create the Token Identifier.
+   * Looks for the option {@link DelegationConstants#DELEGATION_TOKEN_ROLE_ARN}
+   * in the config and fail if it is not set.
+   * @param policy the policy which will be used for the requested token.
+   * @param encryptionSecrets encryption secrets.
+   * @return the token.
+   * @throws IllegalArgumentException if there is no role defined.
+   * @throws IOException any problem acquiring the role.
+   */
+  @Override
+  @Retries.RetryTranslated
+  public RoleTokenIdentifier createTokenIdentifier(
+      final Optional<RoleModel.Policy> policy,
+      final EncryptionSecrets encryptionSecrets) throws IOException {
+    requireServiceStarted();
+    Preconditions.checkState(!roleArn.isEmpty(), E_NO_ARN);
+    String policyJson = policy.isPresent() ?
+        MODEL.toJson(policy.get()) : "";
+    final STSClientFactory.STSClient client = prepareSTSClient()
+        .orElseThrow(() -> {
+          // we've come in on a parent binding, so fail fast
+          LOG.error("Cannot issue delegation tokens because the credential"
+              + " providers listed in " + DELEGATION_TOKEN_CREDENTIALS_PROVIDER
+              + " are returning session tokens");
+          return new DelegationTokenIOException(
+              E_NO_SESSION_TOKENS_FOR_ROLE_BINDING);
+        });
+    Credentials credentials = client
+        .requestRole(roleArn,
+            UUID.randomUUID().toString(),
+            policyJson,
+            getDuration(),
+            TimeUnit.SECONDS);
+    return new RoleTokenIdentifier(
+        getCanonicalUri(),
+        getOwnerText(),
+        fromSTSCredentials(credentials),
+        encryptionSecrets,
+        AbstractS3ATokenIdentifier.createDefaultOriginMessage()
+            + " Role ARN=" + roleArn);
+  }
+
+  @Override
+  public RoleTokenIdentifier createEmptyIdentifier() {
+    return new RoleTokenIdentifier();
+  }
+
+  @Override
+  public String getDescription() {
+    return super.getDescription() + " Role ARN=" +
+        (roleArn.isEmpty() ? "(none)" : ('"' +  roleArn +'"'));
+  }
+
+  @Override
+  protected String bindingName() {
+    return "Role";
+  }
+}
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.fs.s3a.auth.delegation;
 
-import org.apache.hadoop.conf.Configuration;
+import java.net.URI;
+
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.io.Text;
 
 /**
- * Run the encryption tests against the Fast output stream.
- * This verifies that both file writing paths can encrypt their data.
+ * Role token identifier.
+ * Token kind is {@link DelegationConstants#ROLE_TOKEN_KIND}
  */
+public class RoleTokenIdentifier extends SessionTokenIdentifier {
 
-public class ITestS3AEncryptionSSECBlockOutputStream
-    extends AbstractTestS3AEncryption {
-
-  @Override
-  protected Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
-    conf.set(Constants.FAST_UPLOAD_BUFFER,
-        Constants.FAST_UPLOAD_BYTEBUFFER);
-    conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,
-        "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=");
-    return conf;
+  public RoleTokenIdentifier() {
+    super(DelegationConstants.ROLE_TOKEN_KIND);
   }
 
-  @Override
-  protected S3AEncryptionMethods getSSEAlgorithm() {
-    return S3AEncryptionMethods.SSE_C;
+  public RoleTokenIdentifier(final URI uri,
+      final Text owner,
+      final MarshalledCredentials marshalledCredentials,
+      final EncryptionSecrets encryptionSecrets,
+      final String origin) {
+    super(DelegationConstants.ROLE_TOKEN_KIND,
+        owner,
+        uri,
+        marshalledCredentials,
+        encryptionSecrets,
+        origin);
   }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
new file mode 100644 (file)
index 0000000..b8eeca1
--- /dev/null
@@ -0,0 +1,685 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.ServiceOperations;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DURATION_LOG_AT_INFO;
+
+/**
+ * Support for creating a DT from a filesystem.
+ *
+ * Isolated from S3A for control and testability.
+ *
+ * The S3A Delegation Tokens are special in that the tokens are not directly
+ * used to authenticate with the AWS services.
+ * Instead they can session/role  credentials requested off AWS on demand.
+ *
+ * The design is extensible in that different back-end bindings can be used
+ * to switch to different session creation mechanisms, or indeed, to any
+ * other authentication mechanism supported by an S3 service, provided it
+ * ultimately accepts some form of AWS credentials for authentication through
+ * the AWS SDK. That is, if someone wants to wire this up to Kerberos, or
+ * OAuth2, this design should support them.
+ *
+ * URIs processed must be the canonical URIs for the service.
+ */
+@InterfaceAudience.Private
+public class S3ADelegationTokens extends AbstractDTService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3ADelegationTokens.class);
+
+  @VisibleForTesting
+  static final String E_ALREADY_DEPLOYED
+      = "S3A Delegation tokens has already been bound/deployed";
+
+  public static final String E_DELEGATION_TOKENS_DISABLED
+      = "Delegation tokens are not enabled";
+
+  /**
+   * User who owns this FS; fixed at instantiation time, so that
+   * in calls to getDelegationToken() and similar, this user is the one whose
+   * credentials are involved.
+   */
+  private final UserGroupInformation user;
+
+  /**
+   * Count of number of created tokens.
+   * For testing and diagnostics.
+   */
+  private final AtomicInteger creationCount = new AtomicInteger(0);
+
+  /**
+   * Text value of this token service.
+   */
+  private Text service;
+
+  /**
+   * Active Delegation token.
+   */
+  private Optional<Token<AbstractS3ATokenIdentifier>> boundDT
+      = Optional.empty();
+
+  /**
+   * The DT decoded when this instance is created by bonding
+   * to an existing DT.
+   */
+  private Optional<AbstractS3ATokenIdentifier> decodedIdentifier
+      = Optional.empty();
+
+  /**
+   * Dynamically loaded token binding; lifecycle matches this object.
+   */
+  private AbstractDelegationTokenBinding tokenBinding;
+
+  /**
+   * List of cred providers; unset until {@link #bindToDelegationToken(Token)}.
+   */
+  private Optional<AWSCredentialProviderList> credentialProviders
+      = Optional.empty();
+
+  /**
+   * The access policies we want for operations.
+   * There's no attempt to ask for "admin" permissions here, e.g.
+   * those to manipulate S3Guard tables.
+   */
+  protected static final EnumSet<AWSPolicyProvider.AccessLevel> ACCESS_POLICY
+      = EnumSet.of(
+          AWSPolicyProvider.AccessLevel.READ,
+          AWSPolicyProvider.AccessLevel.WRITE);
+
+  /**
+   * Statistics for the owner FS.
+   */
+  private S3AInstrumentation.DelegationTokenStatistics stats;
+
+  /**
+   * Name of the token binding as extracted from token kind; used for
+   * logging.
+   */
+  private String tokenBindingName = "";
+
+  /**
+   * Instantiate.
+   */
+  public S3ADelegationTokens() throws IOException {
+    super("S3ADelegationTokens");
+    user = UserGroupInformation.getCurrentUser();
+  }
+
+  @Override
+  public void bindToFileSystem(final URI uri, final S3AFileSystem fs)
+      throws IOException {
+    super.bindToFileSystem(uri, fs);
+    service = getTokenService(getCanonicalUri());
+    stats = fs.getInstrumentation().newDelegationTokenStatistics();
+  }
+
+  /**
+   * Init the service.
+   * This identifies the token binding class to use and creates, initializes
+   * and starts it.
+   * Will raise an exception if delegation tokens are not enabled.
+   * @param conf configuration
+   * @throws Exception any failure to start up
+   */
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    checkState(hasDelegationTokenBinding(conf),
+        E_DELEGATION_TOKENS_DISABLED);
+    Class<? extends AbstractDelegationTokenBinding> binding = conf.getClass(
+        DelegationConstants.DELEGATION_TOKEN_BINDING,
+        SessionTokenBinding.class,
+        AbstractDelegationTokenBinding.class);
+    tokenBinding = binding.newInstance();
+    tokenBinding.bindToFileSystem(getCanonicalUri(), getFileSystem());
+    tokenBinding.init(conf);
+    tokenBindingName = tokenBinding.getKind().toString();
+    LOG.info("Filesystem {} is using delegation tokens of kind {}",
+        getCanonicalUri(), tokenBindingName);
+  }
+
+  /**
+   * Service startup includes binding to any delegation token, and
+   * deploying unbounded if there is none.
+   * It is after this that token operations can be used.
+   * @throws Exception any failure
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    tokenBinding.start();
+    bindToAnyDelegationToken();
+    LOG.info("S3A Delegation support token {} with {}",
+        identifierToString(),
+        tokenBinding.getDescription());
+  }
+
+  /**
+   * Get the identifier as a string, or "(none)".
+   * @return a string value for logs etc.
+   */
+  private String identifierToString() {
+    return decodedIdentifier.map(Objects::toString)
+        .orElse("(none)");
+  }
+
+  /**
+   * Stop the token binding.
+   * @throws Exception on any failure
+   */
+  @SuppressWarnings("ThrowableNotThrown")
+  @Override
+  protected void serviceStop() throws Exception {
+    LOG.debug("Stopping delegation tokens");
+    try {
+      super.serviceStop();
+    } finally {
+      ServiceOperations.stopQuietly(LOG, tokenBinding);
+    }
+  }
+
+
+  /**
+   * Perform the unbonded deployment operations.
+   * Create the AWS credential provider chain to use
+   * when talking to AWS when there is no delegation token to work with.
+   * authenticating this client with AWS services, and saves it
+   * to {@link #credentialProviders}
+   *
+   * @throws IOException any failure.
+   */
+  private void deployUnbonded()
+      throws IOException {
+    requireServiceStarted();
+    checkState(!isBoundToDT(),
+        "Already Bound to a delegation token");
+    LOG.info("No delegation tokens present: using direct authentication");
+    credentialProviders = Optional.of(tokenBinding.deployUnbonded());
+  }
+
+  /**
+   * Attempt to bind to any existing DT, including unmarshalling its contents
+   * and creating the AWS credential provider used to authenticate
+   * the client.
+   *
+   * If successful:
+   * <ol>
+   *   <li>{@link #boundDT} is set to the retrieved token.</li>
+   *   <li>{@link #decodedIdentifier} is set to the extracted identifier.</li>
+   *   <li>{@link #credentialProviders} is set to the credential
+   *   provider(s) returned by the token binding.</li>
+   * </ol>
+   * If unsuccessful, {@link #deployUnbonded()} is called for the
+   * unbonded codepath instead, which will set
+   * {@link #credentialProviders} to its value.
+   *
+   * This means after this call (and only after) the token operations
+   * can be invoked.
+   *
+   * This method is called from {@link #serviceStart()}, so a check on
+   * the service state can be used to check things; the state model
+   * prevents re-entrant calls.
+   * @throws IOException selection/extraction/validation failure.
+   */
+  private void bindToAnyDelegationToken() throws IOException {
+    checkState(!credentialProviders.isPresent(), E_ALREADY_DEPLOYED);
+    Token<AbstractS3ATokenIdentifier> token = selectTokenFromFSOwner();
+    if (token != null) {
+      bindToDelegationToken(token);
+    } else {
+      deployUnbonded();
+    }
+    if (credentialProviders.get().size() == 0) {
+      throw new DelegationTokenIOException("No AWS credential providers"
+          + " created by Delegation Token Binding "
+          + tokenBinding.getName());
+    }
+  }
+
+  /**
+   * This is a test-only back door which resets the state and binds to
+   * a token again.
+   * This allows an instance of this class to be bonded to a DT after being
+   * started, so avoids the need to have the token in the current user
+   * credentials. It is package scoped so as to only be usable in tests
+   * in the same package.
+   *
+   * Yes, this is ugly, but there is no obvious/easy way to test token
+   * binding without Kerberos getting involved.
+   * @param token token to decode and bind to.
+   * @throws IOException selection/extraction/validation failure.
+   */
+  @VisibleForTesting
+  void resetTokenBindingToDT(final Token<AbstractS3ATokenIdentifier> token)
+      throws IOException{
+    credentialProviders = Optional.empty();
+    bindToDelegationToken(token);
+  }
+
+  /**
+   * Bind to a delegation token retrieved for this filesystem.
+   * Extract the secrets from the token and set internal fields
+   * to the values.
+   * <ol>
+   *   <li>{@link #boundDT} is set to {@code token}.</li>
+   *   <li>{@link #decodedIdentifier} is set to the extracted identifier.</li>
+   *   <li>{@link #credentialProviders} is set to the credential
+   *   provider(s) returned by the token binding.</li>
+   * </ol>
+   * @param token token to decode and bind to.
+   * @throws IOException selection/extraction/validation failure.
+   */
+  @VisibleForTesting
+  public void bindToDelegationToken(
+      final Token<AbstractS3ATokenIdentifier> token)
+      throws IOException {
+    checkState(!credentialProviders.isPresent(), E_ALREADY_DEPLOYED);
+    boundDT = Optional.of(token);
+    AbstractS3ATokenIdentifier dti = extractIdentifier(token);
+    LOG.info("Using delegation token {}", dti);
+    decodedIdentifier = Optional.of(dti);
+    try (DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
+        "Creating Delegation Token")) {
+      // extract the credential providers.
+      credentialProviders = Optional.of(
+          tokenBinding.bindToTokenIdentifier(dti));
+    }
+  }
+
+  /**
+   * Predicate: is there a bound DT?
+   * @return true if there's a value in {@link #boundDT}.
+   */
+  public boolean isBoundToDT() {
+    return boundDT.isPresent();
+  }
+
+  /**
+   * Get any bound DT.
+   * @return a delegation token if this instance was bound to it.
+   */
+  public Optional<Token<AbstractS3ATokenIdentifier>> getBoundDT() {
+    return boundDT;
+  }
+
+  /**
+   * Predicate: will this binding issue a DT if requested
+   * in a call to {@link #getBoundOrNewDT(EncryptionSecrets)}?
+   * That is: should the filesystem declare that it is issuing
+   * delegation tokens?
+   * @return a declaration of what will happen when asked for a token.
+   */
+  public TokenIssuingPolicy getTokenIssuingPolicy() {
+    return isBoundToDT()
+        ? TokenIssuingPolicy.ReturnExistingToken
+        : tokenBinding.getTokenIssuingPolicy();
+  }
+
+  /**
+   * Get any bound DT or create a new one.
+   * @return a delegation token.
+   * @throws IOException if one cannot be created
+   * @param encryptionSecrets encryption secrets for any new token.
+   */
+  @SuppressWarnings("OptionalGetWithoutIsPresent")
+  public Token<AbstractS3ATokenIdentifier> getBoundOrNewDT(
+      final EncryptionSecrets encryptionSecrets)
+      throws IOException {
+    LOG.debug("Delegation token requested");
+    if (isBoundToDT()) {
+      // the FS was created on startup with a token, so return it.
+      LOG.debug("Returning current token");
+      return getBoundDT().get();
+    } else {
+      // not bound to a token, so create a new one.
+      // issued DTs are not cached so that long-lived filesystems can
+      // reliably issue session/role tokens.
+      return createDelegationToken(encryptionSecrets);
+    }
+  }
+
+  /**
+   * How many delegation tokens have been issued?
+   * @return the number times {@link #createDelegationToken(EncryptionSecrets)}
+   * returned a token.
+   */
+  public int getCreationCount() {
+    return creationCount.get();
+  }
+
+  /**
+   * Create a delegation token for the user.
+   * This will only be called if a new DT is needed, that is: the
+   * filesystem has been deployed unbonded.
+   * @param encryptionSecrets encryption secrets for the token.
+   * @return the token
+   * @throws IOException if one cannot be created
+   */
+  @VisibleForTesting
+  public Token<AbstractS3ATokenIdentifier> createDelegationToken(
+      final EncryptionSecrets encryptionSecrets) throws IOException {
+    requireServiceStarted();
+    checkArgument(encryptionSecrets != null,
+        "Null encryption secrets");
+    // this isn't done in in advance as it needs S3Guard initialized in the
+    // filesystem before it can generate complete policies.
+    List<RoleModel.Statement> statements = getFileSystem()
+        .listAWSPolicyRules(ACCESS_POLICY);
+    Optional<RoleModel.Policy> rolePolicy =
+        statements.isEmpty() ?
+            Optional.empty() : Optional.of(new RoleModel.Policy(statements));
+
+    try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
+        "Creating New Delegation Token", tokenBinding.getKind())) {
+      Token<AbstractS3ATokenIdentifier> token
+          = tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets);
+      if (token != null) {
+        token.setService(service);
+        noteTokenCreated(token);
+      }
+      return token;
+    }
+  }
+
+  /**
+   * Note that a token has been created; increment counters and statistics.
+   * @param token token created
+   */
+  private void noteTokenCreated(final Token<AbstractS3ATokenIdentifier> token) {
+    LOG.info("Created S3A Delegation Token: {}", token);
+    creationCount.incrementAndGet();
+    stats.tokenIssued();
+  }
+
+  /**
+   * Get the AWS credential provider.
+   * @return the DT credential provider
+   * @throws IOException failure to parse the DT
+   * @throws IllegalStateException if this instance is not bound to a DT
+   */
+  public AWSCredentialProviderList getCredentialProviders()
+      throws IOException {
+    return credentialProviders.orElseThrow(
+        () -> new DelegationTokenIOException("Not yet bonded"));
+  }
+
+  /**
+   * Get the encryption secrets of the DT.
+   * non-empty iff service is started and was bound to a DT.
+   * @return any encryption settings propagated with the DT.
+   */
+  public Optional<EncryptionSecrets> getEncryptionSecrets() {
+    return decodedIdentifier.map(
+        AbstractS3ATokenIdentifier::getEncryptionSecrets);
+  }
+
+  /**
+   * Get any decoded identifier from the bound DT; empty if not bound.
+   * @return the decoded identifier.
+   */
+  public Optional<AbstractS3ATokenIdentifier> getDecodedIdentifier() {
+    return decodedIdentifier;
+  }
+
+  /**
+   * Get the service identifier of the owning FS.
+   * @return a service identifier to use when registering tokens
+   */
+  public Text getService() {
+    return service;
+  }
+
+  /**
+   * The canonical name of the service.
+   * This can be used as the canonical service name for the FS.
+   * @return the canonicalized FS URI.
+   */
+  public String getCanonicalServiceName() {
+    return getCanonicalUri().toString();
+  }
+
+  /**
+   * Find a token for the FS user and canonical filesystem URI.
+   * @return the token, or null if one cannot be found.
+   * @throws IOException on a failure to unmarshall the token.
+   */
+  @VisibleForTesting
+  public Token<AbstractS3ATokenIdentifier> selectTokenFromFSOwner()
+      throws IOException {
+    return lookupToken(user.getCredentials(),
+        service,
+        tokenBinding.getKind());
+  }
+
+  /**
+   * Get the service identifier of a filesystem.
+   * This must be unique for (S3A, the FS URI)
+   * @param fsURI filesystem URI
+   * @return identifier to use.
+   */
+  private static Text getTokenService(final URI fsURI) {
+    return getTokenService(fsURI.toString());
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3ADelegationTokens{");
+    sb.append("canonicalServiceURI=").append(getCanonicalUri());
+    sb.append("; owner=").append(user.getShortUserName());
+    sb.append("; isBoundToDT=").append(isBoundToDT());
+    sb.append("; token creation count=").append(getCreationCount());
+    sb.append("; tokenManager=").append(tokenBinding);
+    sb.append("; token=").append(identifierToString());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Get the kind of the issued tokens.
+   * @return token kind.
+   */
+  public Text getTokenKind() {
+    return tokenBinding.getKind();
+  }
+
+  /**
+   * Get the service identifier of a filesystem URI.
+   * This must be unique for (S3a, the FS URI)
+   * @param fsURI filesystem URI as a string
+   * @return identifier to use.
+   */
+  @VisibleForTesting
+  static Text getTokenService(final String fsURI) {
+    return new Text(fsURI);
+  }
+
+  /**
+   * From a token, get the session token identifier.
+   * @param token token to process
+   * @return the session token identifier
+   * @throws IOException failure to validate/read data encoded in identifier.
+   * @throws IllegalArgumentException if the token isn't an S3A session token
+   */
+  public AbstractS3ATokenIdentifier extractIdentifier(
+      final Token<? extends AbstractS3ATokenIdentifier> token)
+      throws IOException {
+
+    checkArgument(token != null, "null token");
+    AbstractS3ATokenIdentifier identifier;
+    // harden up decode beyond that Token does itself
+    try {
+      identifier = token.decodeIdentifier();
+    } catch (RuntimeException e) {
+      Throwable cause = e.getCause();
+      if (cause != null) {
+        // its a wrapping around class instantiation.
+        throw new DelegationTokenIOException("Decoding S3A token " + cause,
+            cause);
+      } else {
+        throw e;
+      }
+    }
+    if (identifier == null) {
+      throw new DelegationTokenIOException("Failed to unmarshall token for "
+          + getCanonicalUri());
+    }
+    identifier.validate();
+    return identifier;
+  }
+
+  /**
+   * Return a string for use in building up the User-Agent field, so
+   * get into the S3 access logs. Useful for diagnostics.
+   * Delegates to {{@link AbstractDelegationTokenBinding#getUserAgentField()}}
+   * for the current binding.
+   * @return a string for the S3 logs or "" for "nothing to add"
+   */
+  public String getUserAgentField() {
+    return tokenBinding.getUserAgentField();
+  }
+
+  /**
+   * Look up a token from the credentials, verify it is of the correct
+   * kind.
+   * @param credentials credentials to look up.
+   * @param service service name
+   * @param kind token kind to look for
+   * @return the token or null if no suitable token was found
+   * @throws DelegationTokenIOException wrong token kind found
+   */
+  @VisibleForTesting
+  public static Token<AbstractS3ATokenIdentifier> lookupToken(
+      final Credentials credentials,
+      final Text service,
+      final Text kind)
+      throws DelegationTokenIOException {
+
+    LOG.debug("Looking for token for service {} in credentials", service);
+    Token<?> token = credentials.getToken(service);
+    if (token != null) {
+      Text tokenKind = token.getKind();
+      LOG.debug("Found token of kind {}", tokenKind);
+      if (kind.equals(tokenKind)) {
+        // the Oauth implementation catches and logs here; this one
+        // throws the failure up.
+        return (Token<AbstractS3ATokenIdentifier>) token;
+      } else {
+
+        // there's a token for this URI, but its not the right DT kind
+        throw new DelegationTokenIOException(
+            DelegationTokenIOException.TOKEN_MISMATCH + ": expected token"
+            + " for " + service
+            + " of type " + kind
+            + " but got a token of type " + tokenKind);
+      }
+    }
+    // A token for the service was not found
+    LOG.debug("No token for {} found", service);
+    return null;
+  }
+
+  /**
+   * Look up any token from the service; cast it to one of ours.
+   * @param credentials credentials
+   * @param service service to look up
+   * @return any token found or null if none was
+   * @throws ClassCastException if the token is of a wrong type.
+   */
+  public static Token<AbstractS3ATokenIdentifier> lookupToken(
+      final Credentials credentials,
+      final Text service) {
+    return (Token<AbstractS3ATokenIdentifier>) credentials.getToken(service);
+  }
+
+  /**
+   * Look for any S3A token for the given FS service.
+   * @param credentials credentials to scan.
+   * @param uri the URI of the FS to look for
+   * @return the token or null if none was found
+   */
+  public static Token<AbstractS3ATokenIdentifier> lookupS3ADelegationToken(
+      final Credentials credentials,
+      final URI uri) {
+    return lookupToken(credentials, getTokenService(uri.toString()));
+  }
+
+  /**
+   * Predicate: does this configuration enable delegation tokens?
+   * That is: is there any text in the option
+   * {@link DelegationConstants#DELEGATION_TOKEN_BINDING} ?
+   * @param conf configuration to examine
+   * @return true iff the trimmed configuration option is not empty.
+   */
+  public static boolean hasDelegationTokenBinding(Configuration conf) {
+    return StringUtils.isNotEmpty(
+        conf.getTrimmed(DELEGATION_TOKEN_BINDING,
+            DEFAULT_DELEGATION_TOKEN_BINDING));
+  }
+
+  /**
+   * How will tokens be issued on request?
+   *
+   * The {@link #RequestNewToken} policy does not guarantee that a tokens
+   * can be created, only that an attempt will be made to request one.
+   * It may fail (wrong credential types, wrong role, etc).
+   */
+  public enum TokenIssuingPolicy {
+
+    /** The existing token will be returned. */
+    ReturnExistingToken,
+
+    /** No tokens will be issued. */
+    NoTokensAvailable,
+
+    /** An attempt will be made to request a new DT. */
+    RequestNewToken
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADtFetcher.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADtFetcher.java
new file mode 100644 (file)
index 0000000..8ac07a2
--- /dev/null
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth.delegation;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.DtFetcher;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * A DT fetcher for S3A.
+ * This is a copy-and-paste of
+ * {@code org.apache.hadoop.hdfs.HdfsDtFetcher}.
+ *
+ * It is only needed for the `hadoop dtutil` command.
+ */
+public class S3ADtFetcher implements DtFetcher {
+
+  private static final String SERVICE_NAME = Constants.FS_S3A;
+
+  private static final String FETCH_FAILED =
+      "Filesystem not generating Delegation Tokens";
+
+  /**
+   * Returns the service name for HDFS, which is also a valid URL prefix.
+   */
+  public Text getServiceName() {
+    return new Text(SERVICE_NAME);
+  }
+
+  public boolean isTokenRequired() {
+    return UserGroupInformation.isSecurityEnabled();
+  }
+
+  /**
+   *  Returns Token object via FileSystem, null if bad argument.
+   *  @param conf - a Configuration object used with FileSystem.get()
+   *  @param creds - a Credentials object to which token(s) will be added
+   *  @param renewer  - the renewer to send with the token request
+   *  @param url  - the URL to which the request is sent
+   *  @return a Token, or null if fetch fails.
+   */
+  public Token<?> addDelegationTokens(Configuration conf,
+      Credentials creds,
+      String renewer,
+      String url) throws Exception {
+    if (!url.startsWith(getServiceName().toString())) {
+      url = getServiceName().toString() + "://" + url;
+    }
+    FileSystem fs = FileSystem.get(URI.create(url), conf);
+    Token<?> token = fs.getDelegationToken(renewer);
+    if (token == null) {
+      throw new DelegationTokenIOException(FETCH_FAILED + ": " + url);
+    }
+    creds.addToken(token.getService(), token);
+    return token;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
new file mode 100644 (file)
index 0000000..67933c7
--- /dev/null
@@ -0,0 +1,421 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.IOException;
+import java.net.URI;
+import java.time.OffsetDateTime;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSSessionCredentials;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
+import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+
+import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
+import static org.apache.hadoop.fs.s3a.Invoker.once;
+import static org.apache.hadoop.fs.s3a.S3AUtils.STANDARD_AWS_PROVIDERS;
+import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
+import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromAWSCredentials;
+import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
+
+/**
+ * The session token DT binding: creates an AWS session token
+ * for the DT, extracts and serves it up afterwards.
+ */
+public class SessionTokenBinding extends AbstractDelegationTokenBinding {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      SessionTokenBinding.class);
+
+  /**
+   * Wire name of this binding: {@value}.
+   */
+  private static final String NAME = "SessionTokens/001";
+
+  /**
+   * A message added to the standard origin string when the DT is
+   * built from session credentials passed in.
+   */
+  @VisibleForTesting
+  public static final String CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN
+      = "Existing session credentials converted to Delegation Token";
+
+  public static final String SESSION_TOKEN
+      = "Session Delegation Token";
+
+  /** Invoker for STS calls. */
+  private Invoker invoker;
+
+  /**
+   * Has an attempt to initialize STS been attempted?
+   */
+  private final AtomicBoolean stsInitAttempted = new AtomicBoolean(false);
+
+  /** The STS client; created in startup if the parental credentials permit. */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private Optional<STSClientFactory.STSClient> stsClient = Optional.empty();
+
+  /**
+   * Duration of session in seconds.
+   */
+  private long duration;
+
+  /**
+   * Flag to indicate that the auth chain provides session credentials.
+   * If true it means that STS cannot be used (and stsClient is null).
+   */
+  private boolean hasSessionCreds;
+
+  /**
+   * The auth chain for the parent options.
+   */
+  private AWSCredentialProviderList parentAuthChain;
+
+  /**
+   * Has a log message about forwarding credentials been printed yet?
+   */
+  private final AtomicBoolean forwardMessageLogged = new AtomicBoolean(false);
+
+  /** STS endpoint. */
+  private String endpoint;
+
+  /** STS region. */
+  private String region;
+
+  /**
+   * Expiration date time as passed in from source.
+   * If unset, either we are unbound, or the token which came in does not
+   * know its expiry.
+   */
+  private Optional<OffsetDateTime> expirationDateTime;
+
+  /**
+   * Token identifier bound to.
+   */
+  private Optional<SessionTokenIdentifier> tokenIdentifier = Optional.empty();
+
+  /** Constructor for reflection. */
+  public SessionTokenBinding() {
+    this(NAME, SESSION_TOKEN_KIND);
+  }
+
+  /**
+   * Constructor for subclasses.
+   * @param name binding name.
+   * @param kind token kind.
+   */
+  protected SessionTokenBinding(final String name,
+      final Text kind) {
+    super(name, kind);
+  }
+
+  /**
+   * Service start will read in all configuration options
+   * then build that client.
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    Configuration conf = getConfig();
+    duration = conf.getTimeDuration(DELEGATION_TOKEN_DURATION,
+        DEFAULT_DELEGATION_TOKEN_DURATION,
+        TimeUnit.SECONDS);
+    endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
+        DEFAULT_DELEGATION_TOKEN_ENDPOINT);
+    region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
+        DEFAULT_DELEGATION_TOKEN_REGION);
+
+    // create the provider set for session credentials.
+    parentAuthChain = buildAWSProviderList(
+        getCanonicalUri(),
+        conf,
+        AWS_CREDENTIALS_PROVIDER,
+        STANDARD_AWS_PROVIDERS,
+        new HashSet<>());
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+    // this is here to keep findbugs quiet, even though nothing
+    // can safely invoke stsClient as we are shut down.
+    synchronized (this) {
+      this.stsClient.ifPresent(IOUtils::closeStream);
+      this.stsClient = Optional.empty();
+    }
+  }
+
+  /**
+   * Return an unbonded provider chain.
+   * @return the auth chain built from the assumed role credentials
+   * @throws IOException any failure.
+   */
+  @Override
+  public AWSCredentialProviderList deployUnbonded()
+      throws IOException {
+    requireServiceStarted();
+    return parentAuthChain;
+  }
+
+  /**
+   * Get the invoker for STS calls.
+   * @return the invoker
+   */
+  protected Invoker getInvoker() {
+    return invoker;
+  }
+
+  /**
+   * Sets the field {@link #tokenIdentifier} to the extracted/cast
+   * session token identifier, and {@link #expirationDateTime} to
+   * any expiration passed in.
+   * @param retrievedIdentifier the unmarshalled data
+   * @return the provider list
+   * @throws IOException failure
+   */
+  @Override
+  public AWSCredentialProviderList bindToTokenIdentifier(
+      final AbstractS3ATokenIdentifier retrievedIdentifier)
+      throws IOException {
+    final SessionTokenIdentifier identifier = convertTokenIdentifier(
+        retrievedIdentifier,
+        SessionTokenIdentifier.class);
+    setTokenIdentifier(Optional.of(identifier));
+    MarshalledCredentials marshalledCredentials
+        = identifier.getMarshalledCredentials();
+    setExpirationDateTime(marshalledCredentials.getExpirationDateTime());
+    return new AWSCredentialProviderList(
+        "Session Token Binding",
+        new MarshalledCredentialProvider(
+            SESSION_TOKEN,
+            getFileSystem().getUri(),
+            getConfig(),
+            marshalledCredentials,
+            MarshalledCredentials.CredentialTypeRequired.SessionOnly));
+  }
+
+  @Override
+  public String getDescription() {
+    return String.format(
+            "%s token binding for user %s, " +
+            "with STS endpoint \"%s\", region \"%s\""
+                + " and token duration %d:%02d",
+        bindingName(), getOwner().getShortUserName(), endpoint, region,
+        TimeUnit.SECONDS.toMinutes(duration),
+        duration % 60);
+  }
+
+  /**
+   * Get the role of this token; subclasses should override this
+   * for better logging.
+   * @return the role of this token
+   */
+  protected String bindingName() {
+    return "Session";
+  }
+
+  /**
+   * UA field contains the UUID of the token if present.
+   * @return a string for the S3 logs.
+   */
+  public String getUserAgentField() {
+    if (tokenIdentifier.isPresent()) {
+      return "; session ID " + tokenIdentifier.get().getUuid();
+    } else {
+      return "";
+    }
+  }
+
+  /**
+   * Attempt to init the STS connection, only does it once.
+   * If the AWS credential list to this service return session credentials
+   * then this method will return {@code empty()}; no attempt is
+   * made to connect to STS.
+   * Otherwise, the STS binding info will be looked up and an attempt
+   * made to connect to STS.
+   * Only one attempt will be made.
+   * @return any STS client created.
+   * @throws IOException any failure to bind to STS.
+   */
+  private synchronized Optional<STSClientFactory.STSClient> maybeInitSTS()
+      throws IOException {
+    if (stsInitAttempted.getAndSet(true)) {
+      // whether or not it succeeded, the state of the STS client is what
+      // callers get after the first attempt.
+      return stsClient;
+    }
+
+    Configuration conf = getConfig();
+    URI uri = getCanonicalUri();
+
+    // Ask the owner for any session credentials which it already has
+    // so that it can just propagate them.
+    // this call may fail if there are no credentials on the auth
+    // chain.
+    // As no codepath (session propagation, STS creation) will work,
+    // throw this.
+    final AWSCredentials parentCredentials = once("get credentials",
+        "",
+        () -> parentAuthChain.getCredentials());
+    hasSessionCreds = parentCredentials instanceof AWSSessionCredentials;
+
+    if (!hasSessionCreds) {
+      LOG.info("Creating STS client for {}", getDescription());
+
+      invoker = new Invoker(new S3ARetryPolicy(conf), LOG_EVENT);
+      ClientConfiguration awsConf =
+          S3AUtils.createAwsConf(conf, uri.getHost());
+      AWSSecurityTokenService tokenService =
+          STSClientFactory.builder(parentAuthChain,
+              awsConf,
+              endpoint,
+              region)
+              .build();
+      stsClient = Optional.of(
+          STSClientFactory.createClientConnection(tokenService, invoker));
+    } else {
+      LOG.debug("Parent-provided session credentials will be propagated");
+      stsClient = Optional.empty();
+    }
+    return stsClient;
+  }
+
+  /**
+   * Log retries at debug.
+   */
+  public static final Invoker.Retried LOG_EVENT =
+      (text, exception, retries, idempotent) -> {
+        LOG.info("{}: " + exception, text);
+        if (retries == 1) {
+          // stack on first attempt, to keep noise down
+          LOG.debug("{}: " + exception, text, exception);
+        }
+      };
+
+  /**
+   * Get the client to AWS STS.
+   * @return the STS client, when successfully inited.
+   */
+  protected Optional<STSClientFactory.STSClient> prepareSTSClient()
+      throws IOException {
+    return maybeInitSTS();
+  }
+
+  /**
+   * Duration of sessions.
+   * @return duration in seconds.
+   */
+  public long getDuration() {
+    return duration;
+  }
+
+  @Override
+  @Retries.RetryTranslated
+  public SessionTokenIdentifier createTokenIdentifier(
+      final Optional<RoleModel.Policy> policy,
+      final EncryptionSecrets encryptionSecrets) throws IOException {
+    requireServiceStarted();
+
+    final MarshalledCredentials marshalledCredentials;
+    String origin = AbstractS3ATokenIdentifier.createDefaultOriginMessage();
+    final Optional<STSClientFactory.STSClient> client = prepareSTSClient();
+
+    if (client.isPresent()) {
+      // this is the normal route: ask for a new STS token
+      marshalledCredentials = fromSTSCredentials(
+          client.get()
+              .requestSessionCredentials(duration, TimeUnit.SECONDS));
+    } else {
+      // get a new set of parental session credentials (pick up IAM refresh)
+      if (!forwardMessageLogged.getAndSet(true)) {
+        // warn caller on the first -and only the first- use.
+        LOG.warn("Forwarding existing session credentials to {}"
+            + " -duration unknown", getCanonicalUri());
+      }
+      origin += " " + CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN;
+      final AWSCredentials awsCredentials
+          = parentAuthChain.getCredentials();
+      if (awsCredentials instanceof AWSSessionCredentials) {
+        marshalledCredentials = fromAWSCredentials(
+            (AWSSessionCredentials) awsCredentials);
+      } else {
+        throw new DelegationTokenIOException(
+            "AWS Authentication chain is no longer supplying session secrets");
+      }
+    }
+    return new SessionTokenIdentifier(getKind(),
+        getOwnerText(),
+        getCanonicalUri(),
+        marshalledCredentials,
+        encryptionSecrets,
+        origin);
+  }
+
+  @Override
+  public SessionTokenIdentifier createEmptyIdentifier() {
+    return new SessionTokenIdentifier();
+  }
+
+  /**
+   * Expiration date time as passed in from source.
+   * If unset, either we are unbound, or the token which came in does not
+   * know its expiry.
+   */
+  protected Optional<OffsetDateTime> getExpirationDateTime() {
+    return expirationDateTime;
+  }
+
+  protected void setExpirationDateTime(
+      Optional<OffsetDateTime> expirationDateTime) {
+    this.expirationDateTime = expirationDateTime;
+  }
+
+  /**
+   * Token identifier bound to.
+   */
+  protected Optional<SessionTokenIdentifier> getTokenIdentifier() {
+    return tokenIdentifier;
+  }
+
+  protected void setTokenIdentifier(Optional<SessionTokenIdentifier>
+      tokenIdentifier) {
+    this.tokenIdentifier = tokenIdentifier;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenIdentifier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenIdentifier.java
new file mode 100644 (file)
index 0000000..3928a0d
--- /dev/null
@@ -0,0 +1,146 @@
+/*
+ * 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.auth.delegation;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.io.Text;
+
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.SESSION_TOKEN_KIND;
+
+/**
+ * A token identifier which contains a set of AWS session credentials,
+ * credentials which will be valid until they expire.
+ *
+ * <b>Note 1:</b>
+ * There's a risk here that the reference to {@link MarshalledCredentials}
+ * may trigger a transitive load of AWS classes, a load which will
+ * fail if the aws SDK isn't on the classpath.
+ *
+ * <b>Note 2:</b>
+ * This class does support subclassing, but every subclass MUST declare itself
+ * to be of a different token kind.
+ * Otherwise the process for decoding tokens breaks.
+ */
+public class SessionTokenIdentifier extends
+    AbstractS3ATokenIdentifier {
+
+  /**
+   * Session credentials: initially empty but non-null.
+   */
+  private MarshalledCredentials marshalledCredentials
+      = new MarshalledCredentials();
+
+  /**
+   * Constructor for service loader use.
+   * Created with the kind {@link DelegationConstants#SESSION_TOKEN_KIND}.
+   * Subclasses MUST NOT subclass this; they must provide their own
+   * token kind.
+   */
+  public SessionTokenIdentifier() {
+    super(SESSION_TOKEN_KIND);
+  }
+
+  /**
+   * Constructor for subclasses.
+   * @param kind kind of token identifier, for storage in the
+   * token kind to implementation map.
+   */
+  protected SessionTokenIdentifier(final Text kind) {
+    super(kind);
+  }
+
+  /**
+   * Constructor.
+   * @param kind token kind.
+   * @param owner token owner
+   * @param uri filesystem URI.
+   * @param marshalledCredentials credentials to marshall
+   * @param encryptionSecrets encryption secrets
+   * @param origin origin text for diagnostics.
+   */
+  public SessionTokenIdentifier(
+      final Text kind,
+      final Text owner,
+      final URI uri,
+      final MarshalledCredentials marshalledCredentials,
+      final EncryptionSecrets encryptionSecrets,
+      final String origin) {
+    super(kind, uri, owner, origin, encryptionSecrets);
+    this.marshalledCredentials = marshalledCredentials;
+  }
+
+  /**
+   * Constructor.
+   * @param kind token kind.
+   * @param owner token owner
+   * @param uri filesystem URI.
+   */
+  public SessionTokenIdentifier(final Text kind,
+      final Text owner,
+      final Text renewer,
+      final Text realUser,
+      final URI uri) {
+    super(kind, owner, renewer, realUser, uri);
+  }
+
+  @Override
+  public void write(final DataOutput out) throws IOException {
+    super.write(out);
+    marshalledCredentials.write(out);
+  }
+
+  @Override
+  public void readFields(final DataInput in)
+      throws IOException {
+    super.readFields(in);
+    marshalledCredentials.readFields(in);
+  }
+
+  /**
+   * Return the expiry time in seconds since 1970-01-01.
+   * @return the time when the AWS credentials expire.
+   */
+  @Override
+  public long getExpiryTime() {
+    return marshalledCredentials.getExpiration();
+  }
+
+  /**
+   * Get the marshalled credentials.
+   * @return marshalled AWS credentials.
+   */
+  public MarshalledCredentials getMarshalledCredentials() {
+    return marshalledCredentials;
+  }
+
+  /**
+   * Add the (sanitized) marshalled credentials to the string value.
+   * @return a string value for test assertions and debugging.
+   */
+  @Override
+  public String toString() {
+    return super.toString()
+        + "; " + marshalledCredentials.toString();
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/package-info.java
new file mode 100644 (file)
index 0000000..f7eb6b1
--- /dev/null
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Extensible delegation token support for the S3A connector.
+ *
+ * Goal: support multiple back end token issue/renewal services, from
+ * "pure client side" session tokens to full "Kerberos auth".
+ *
+ * It is intended for internal use only; any external implementation
+ * of {@link org.apache.hadoop.fs.s3a.auth.delegation.AbstractDelegationTokenBinding}
+ * must consider this API unstable and track changes as they happen.
+ */
+@InterfaceAudience.LimitedPrivate("authorization-subsystems")
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.auth.delegation;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
index e34d68e..c3a7ee6 100644 (file)
 
 /**
  * Authentication and permissions support.
+ *
+ * Some of the classes in here are expected to be referred to in configuration
+ * files, so must not change their name. These will be explicitly identified.
  */
-@InterfaceAudience.Private
+
+@InterfaceAudience.LimitedPrivate("Authentication services")
 @InterfaceStability.Unstable
 package org.apache.hadoop.fs.s3a.auth;
 
index c6617f8..69f90cb 100644 (file)
@@ -23,7 +23,8 @@ import org.slf4j.Logger;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
- * A duration with logging of final state at info in the {@code close()} call.
+ * A duration with logging of final state at info or debug
+ * in the {@code close()} call.
  * This allows it to be used in a try-with-resources clause, and have the
  * duration automatically logged.
  */
@@ -35,15 +36,41 @@ public class DurationInfo extends Duration
   private final Logger log;
 
   /**
-   * Create the duration text from a {@code String.format()} code call.
+   * Should the log be at INFO rather than DEBUG.
+   */
+  private final boolean logAtInfo;
+
+  /**
+   * Create the duration text from a {@code String.format()} code call;
+   * log output at info level.
    * @param log log to write to
    * @param format format string
    * @param args list of arguments
    */
   public DurationInfo(Logger log, String format, Object... args) {
+    this(log, true, format, args);
+  }
+
+  /**
+   * Create the duration text from a {@code String.format()} code call
+   * and log either at info or debug.
+   * @param log log to write to
+   * @param logAtInfo should the log be at info, rather than debug
+   * @param format format string
+   * @param args list of arguments
+   */
+  public DurationInfo(Logger log,
+      boolean logAtInfo,
+      String format,
+      Object... args) {
     this.text = String.format(format, args);
     this.log = log;
-    log.info("Starting: {}", text);
+    this.logAtInfo = logAtInfo;
+    if (logAtInfo) {
+      log.info("Starting: {}", text);
+    } else {
+      log.debug("Starting: {}", text);
+    }
   }
 
   @Override
@@ -54,6 +81,10 @@ public class DurationInfo extends Duration
   @Override
   public void close() {
     finished();
-    log.info(this.toString());
+    if (logAtInfo) {
+      log.info("{}", this);
+    } else {
+      log.debug("{}", this);
+    }
   }
 }
index e7f0207..c6f826b 100644 (file)
@@ -273,8 +273,8 @@ public class DirListingMetadata extends ExpirableMetadata {
 
     // If this dir's path has host (and thus scheme), so must its children
     URI parentUri = path.toUri();
+    URI childUri = childPath.toUri();
     if (parentUri.getHost() != null) {
-      URI childUri = childPath.toUri();
       Preconditions.checkNotNull(childUri.getHost(), "Expected non-null URI " +
           "host: %s", childUri);
       Preconditions.checkArgument(
@@ -286,7 +286,8 @@ public class DirListingMetadata extends ExpirableMetadata {
     }
     Preconditions.checkArgument(!childPath.isRoot(),
         "childPath cannot be the root path: %s", childPath);
-    Preconditions.checkArgument(childPath.getParent().equals(path),
+    Preconditions.checkArgument(parentUri.getPath().equals(
+        childPath.getParent().toUri().getPath()),
         "childPath %s must be a child of %s", childPath, path);
   }
 
index e8b3c0c..d2676f7 100644 (file)
@@ -26,6 +26,7 @@ import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -70,6 +71,7 @@ import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import com.amazonaws.waiters.WaiterTimedOutException;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +92,9 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.fs.s3a.auth.RoleModel;
 import org.apache.hadoop.fs.s3a.auth.RolePolicies;
+import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -98,6 +102,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations;
+import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations;
 import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
 
@@ -185,7 +191,8 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class DynamoDBMetadataStore implements MetadataStore {
+public class DynamoDBMetadataStore implements MetadataStore,
+    AWSPolicyProvider {
   public static final Logger LOG = LoggerFactory.getLogger(
       DynamoDBMetadataStore.class);
 
@@ -231,6 +238,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
   private String region;
   private Table table;
   private String tableName;
+  private String tableArn;
   private Configuration conf;
   private String username;
 
@@ -403,6 +411,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
     region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
     Preconditions.checkArgument(!StringUtils.isEmpty(region),
         "No DynamoDB region configured");
+    // there's no URI here, which complicates life: you cannot
+    // create AWS providers here which require one.
     credentials = createAWSCredentialProviderSet(null, conf);
     dynamoDB = createDynamoDB(conf, region, null, credentials);
 
@@ -1122,10 +1132,34 @@ public class DynamoDBMetadataStore implements MetadataStore {
     return getClass().getSimpleName() + '{'
         + "region=" + region
         + ", tableName=" + tableName
+        + ", tableArn=" + tableArn
         + '}';
   }
 
   /**
+   * The administrative policy includes all DDB table operations;
+   * application access is restricted to those operations S3Guard operations
+   * require when working with data in a guarded bucket.
+   * @param access access level desired.
+   * @return a possibly empty list of statements.
+   */
+  @Override
+  public List<RoleModel.Statement> listAWSPolicyRules(
+      final Set<AccessLevel> access) {
+    Preconditions.checkState(tableArn != null, "TableARN not known");
+    if (access.isEmpty()) {
+      return Collections.emptyList();
+    }
+    RoleModel.Statement stat;
+    if (access.contains(AccessLevel.ADMIN)) {
+      stat = allowAllDynamoDBOperations(tableArn);
+    } else {
+      stat = allowS3GuardClientOperations(tableArn);
+    }
+    return Lists.newArrayList(stat);
+  }
+
+  /**
    * Create a table if it does not exist and wait for it to become active.
    *
    * If a table with the intended name already exists, then it uses that table.
@@ -1151,6 +1185,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
         LOG.debug("Binding to table {}", tableName);
         TableDescription description = table.describe();
         LOG.debug("Table state: {}", description);
+        tableArn = description.getTableArn();
         final String status = description.getTableStatus();
         switch (status) {
         case "CREATING":
index 6d1e146..3751fda 100644 (file)
@@ -51,8 +51,10 @@ import org.apache.hadoop.fs.s3a.MultipartUtils;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
@@ -1168,6 +1170,23 @@ public abstract class S3GuardTool extends Configured implements Tool {
               "none");
       printOption(out, "\tInput seek policy", INPUT_FADVISE, INPUT_FADV_NORMAL);
 
+      // look at delegation token support
+      if (fs.getDelegationTokens().isPresent()) {
+        // DT is enabled
+        S3ADelegationTokens dtIntegration = fs.getDelegationTokens().get();
+        println(out, "Delegation Support enabled: token kind = %s",
+            dtIntegration.getTokenKind());
+        UserGroupInformation.AuthenticationMethod authenticationMethod
+            = UserGroupInformation.getCurrentUser().getAuthenticationMethod();
+        println(out, "Hadoop security mode: %s", authenticationMethod);
+        if (UserGroupInformation.isSecurityEnabled()) {
+          println(out,
+              "Warning: security is disabled; tokens will not be collected");
+        }
+      } else {
+        println(out, "Delegation token support is disabled");
+      }
+
       if (usingS3Guard) {
         if (commands.getOpt(UNGUARDED_FLAG)) {
           throw badState("S3Guard is enabled for %s", fsUri);
index 60d4b76..84e4a67 100644 (file)
@@ -136,8 +136,6 @@ public final class S3xLoginHelper {
   /**
    * Canonicalize the given URI.
    *
-   * This strips out login information.
-   *
    * @param uri the URI to canonicalize
    * @param defaultPort default port to use in canonicalized URI if the input
    *     URI has no port and this value is greater than 0
diff --git a/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher b/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher
new file mode 100644 (file)
index 0000000..c1a3bd0
--- /dev/null
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+org.apache.hadoop.fs.s3a.auth.delegation.S3ADtFetcher
diff --git a/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
new file mode 100644 (file)
index 0000000..bfd3def
--- /dev/null
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenIdentifier
+org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenIdentifier
+org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenIdentifier
index 8af0457..f08f40f 100644 (file)
@@ -178,12 +178,14 @@ Here are the full set of configuration options.
 
 <property>
   <name>fs.s3a.assumed.role.credentials.provider</name>
-  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+    com.amazonaws.auth.EnvironmentVariableCredentialsProvider
+  </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".
+    Used by AssumedRoleCredentialProvider and the S3A Session Delegation Token
+    and S3A Role Delegation Token bindings.
   </description>
 </property>
 ```
@@ -468,17 +470,69 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
   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"
+### <a name="invalid_duration"></a> `Member must have value greater than or equal to 900`
 
-The value of `fs.s3a.assumed.role.session.duration` is out of range.
+The value of `fs.s3a.assumed.role.session.duration` is too low.
 
 ```
-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.auth.AssumedRoleCredentialProvider.<init>(AssumedRoleCredentialProvider.java:86)
+org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
+com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+1 validation error detected: Value '20' at 'durationSeconds' failed to satisfy constraint:
+Member must have value greater than or equal to 900 (Service: AWSSecurityTokenService;
+Status Code: 400; Error Code: ValidationError;
+Request ID: b9a82403-d0a7-11e8-98ef-596679ee890d)
+```
+
+Fix: increase.
+
+### <a name="duration_too_high"></a> Error "The requested DurationSeconds exceeds the MaxSessionDuration set for this role"
+
+The value of `fs.s3a.assumed.role.session.duration` is too high.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials:
+ com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+The requested DurationSeconds exceeds the MaxSessionDuration set for this role.
+(Service: AWSSecurityTokenService; Status Code: 400;
+ Error Code: ValidationError; Request ID: 17875165-d0a7-11e8-b85f-d15a599a7f6d)
+```
+
+There are two solutions to this
+
+* Decrease the duration value.
+* Increase the duration of a role in the [AWS IAM Console](https://console.aws.amazon.com/iam/home#/roles).
+
+
+### "Value '345600' at 'durationSeconds' failed to satisfy constraint: Member must have value less than or equal to 43200"
+
+Irrespective of the maximum duration of a role, the AWS role API only permits callers to request
+any role for up to  12h; attempting to use a larger number will fail.
+
+
+```
+Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+1 validation error detected:
+Value '345600' at 'durationSeconds' failed to satisfy constraint:
+Member must have value less than or equal to 43200
+(Service: AWSSecurityTokenService;
+Status Code: 400; Error Code:
+ValidationError;
+Request ID: dec1ca6b-d0aa-11e8-ac8c-4119b3ea9f7f)
+```
+
+For full sessions, the duration limit is 129600 seconds: 36h.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: request session credentials:
+com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+1 validation error detected: Value '345600' at 'durationSeconds' failed to satisfy constraint:
+Member must have value less than or equal to 129600
+(Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError;
+Request ID: a6e73d44-d0aa-11e8-95ed-c5bba29f0635)
 ```
 
+For both these errors, the sole fix is to request a shorter duration
+in `fs.s3a.assumed.role.session.duration`.
 
 ### <a name="malformed_policy"></a> `MalformedPolicyDocumentException` "The policy is not in the valid JSON format"
 
@@ -487,7 +541,7 @@ 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.auth.AssumedRoleCredentialProvider on :
+org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.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):
@@ -508,36 +562,9 @@ Caused by: com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentExc
    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.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
-  at org.apache.hadoop.fs.s3a.auth.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"
+### <a name="policy_syntax_error"></a> `MalformedPolicyDocumentException` "Syntax errors in policy"
 
 The policy set in `fs.s3a.assumed.role.policy` is not valid JSON.
 
@@ -564,31 +591,6 @@ Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on :
   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.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
-  at org.apache.hadoop.fs.s3a.auth.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
 ```
 
@@ -646,34 +648,6 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
     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.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
-  at org.apache.hadoop.fs.s3a.auth.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"
@@ -702,31 +676,6 @@ The security token included in the request is invalid.
   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.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:127)
-  at org.apache.hadoop.fs.s3a.auth.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
 ```
 
@@ -740,7 +689,8 @@ match these constraints.
 If set explicitly, it must be valid.
 
 ```
-org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+ Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on
     com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
     1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName'
     failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]*
@@ -765,33 +715,6 @@ Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceExc
     (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.auth.AssumedRoleCredentialProvider.getCredentials(AssumedRoleCredentialProvider.java:135)
-  at org.apache.hadoop.fs.s3a.auth.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
 ```
 
 
@@ -818,24 +741,6 @@ Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
   S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=
   at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
   at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
-  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.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
-  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
-  at com.amazonaws.services.s3.AmazonS3Client.deleteObject(AmazonS3Client.java:2066)
-  at com.amazonaws.services.s3.AmazonS3Client.deleteObject(AmazonS3Client.java:2052)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$deleteObject$7(S3AFileSystem.java:1338)
-  at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:314)
-  at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:280)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.deleteObject(S3AFileSystem.java:1334)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.removeKeys(S3AFileSystem.java:1657)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerRename(S3AFileSystem.java:1046)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:851)
 ```
 
 This is the policy restriction behaving as intended: the caller is trying to
@@ -882,3 +787,63 @@ or just that this specific permission has been omitted.
 If the role policy requested for the assumed role didn't ask for any DynamoDB
 permissions, this is where all attempts to work with a S3Guarded bucket will
 fail. Check the value of `fs.s3a.assumed.role.policy`
+
+### Error `Unable to execute HTTP request`
+
+This is a low-level networking error. Possible causes include:
+
+* The endpoint set in `fs.s3a.assumed.role.sts.endpoint` is invalid.
+* There are underlying network problems.
+
+```
+org.apache.hadoop.fs.s3a.AWSClientIOException: request session credentials:
+  com.amazonaws.SdkClientException:
+
+  Unable to execute HTTP request: null: Unable to execute HTTP request: null
+at com.amazonaws.thirdparty.apache.http.impl.conn.DefaultRoutePlanner.determineRoute(DefaultRoutePlanner.java:88)
+at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.determineRoute(InternalHttpClient.java:124)
+at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:183)
+at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+```
+
+###  <a name="credential_scope"></a> Error "Credential should be scoped to a valid region"
+
+This is based on conflict between the values of `fs.s3a.assumed.role.sts.endpoint`
+and `fs.s3a.assumed.role.sts.endpoint.region`
+Two variants, "not '''"
+
+Variant 1: `Credential should be scoped to a valid region, not 'us-west-1'` (or other string)
+
+
+```
+java.nio.file.AccessDeniedException: : request session credentials:
+com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+Credential should be scoped to a valid region, not 'us-west-1'.
+(Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch; Request ID: d9065cc4-e2b9-11e8-8b7b-f35cb8d7aea4):SignatureDoesNotMatch
+```
+
+One of:
+
+
+* the value of `fs.s3a.assumed.role.sts.endpoint.region` is not a valid region
+* the value of `fs.s3a.assumed.role.sts.endpoint.region` is not the signing
+region of the endpoint set in `fs.s3a.assumed.role.sts.endpoint`
+
+
+Variant 2: `Credential should be scoped to a valid region, not ''`
+
+```
+java.nio.file.AccessDeniedException: : request session credentials:
+com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException:
+  Credential should be scoped to a valid region, not ''. (
+  Service: AWSSecurityTokenService; Status Code: 403; Error Code: SignatureDoesNotMatch;
+  Request ID: bd3e5121-e2ac-11e8-a566-c1a4d66b6a16):SignatureDoesNotMatch
+```
+
+This should be intercepted earlier: an endpoint has been specified but
+not a region.
+
+There's special handling for the central `sts.amazonaws.com` region; when
+that is declared as the value of `fs.s3a.assumed.role.sts.endpoint.region` then
+there is no need to declare a region: whatever value it has is ignored.
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_token_architecture.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_token_architecture.md
new file mode 100644 (file)
index 0000000..90e4e55
--- /dev/null
@@ -0,0 +1,466 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# S3A Delegation Token Architecture
+
+This is an architecture document to accompany
+[Working with Delegation Tokens](delegation_tokens.html)
+
+## Background: Delegation Tokens
+
+Delegation Tokens, "DTs" are a common feature of Hadoop Services.
+They are opaque byte arrays which can be issued by services like
+HDFS, HBase, YARN, and which can be used to authenticate a request with
+that service.
+
+### Tokens are Issued
+
+In a Kerberized cluster, they are issued by the service after the caller
+has authenticated, and so that principal is trusted to be who they say they are.
+The issued DT can therefore attest that whoever is including that token
+on a request is authorized to act on behalf of that principal —for the
+specific set of operations which the DT grants.
+
+As an example, an HDFS DT can be requested by a user, included in the
+launch context of a YARN application -say DistCp, and that launched application
+can then talk to HDFS as if they were that user.
+
+### Tokens are marshalled
+
+Tokens are opaque byte arrays. They are contained within a `Token<T extends TokenIdentifier>`
+ class which includes an expiry time, the service identifier, and some other details.
+
+`Token<>` instances can be serialized as a Hadoop Writable, or converted saved to/from a protobuf
+format. This is how they are included in YARN application and container requests,
+and elsewhere. They can even be saved to files through the `hadoop dt` command.
+
+### Tokens can be unmarshalled
+
+
+At the far end, tokens can be unmarshalled and converted into instances of
+the java classes. This assumes that all the dependent classes are on the
+classpath, obviously.
+
+### Tokens can be used to authenticate callers
+
+The Hadoop RPC layer and the web SPNEGO layer support tokens.
+
+### Tokens can be renewed
+
+DTs can be renewed by the specific principal declared at creation time as
+"the renewer". In the example above, the YARN Resource Manager's principal
+can be declared as the reviewer. Then, even while a token is attached
+to a queued launch request in the RM, the RM can regularly request of HDFS
+that the token is renewed.
+
+There's an ultimate limit on how long tokens can be renewed for, but its
+generally 72h or similar, so that medium-life jobs can access services
+and data on behalf of a user.
+
+### Tokens can be Revoked
+
+When tokens are no longer needed, the service can be told to revoke a token.
+Continuing the YARN example, after an application finishes the YARN RM
+can revoke every token marshalled into the application launch request.
+At which point there's no risk associated with that token being
+compromised.
+
+
+*This is all how "real" Hadoop tokens work*
+
+The S3A Delegation Tokens are subtly different.
+
+The S3A DTs actually include the AWS credentials within the token
+data marshalled and shared across the cluster. The credentials can be one
+of:
+
+* The Full AWS (`fs.s3a.access.key`, `fs.s3a.secret.key`) login.
+* A set of AWS session credentials
+  (`fs.s3a.access.key`, `fs.s3a.secret.key`, `fs.s3a.session.token`).
+
+These credentials are obtained from the AWS Secure Token Service (STS) when the the token is issued.
+* A set of AWS session credentials binding the user to a specific AWS IAM Role,
+further restricted to only access the S3 bucket and matching S3Guard DynamoDB table.
+Again, these credentials are requested when the token is issued.
+
+
+*Tokens can be issued*
+
+When an S3A Filesystem instance is asked to issue a token it can simply package
+up the login secrets (The "Full" tokens), or talk to the AWS STS service
+to get a set of session/assumed role credentials. These are marshalled within
+the overall token, and then onwards to applications.
+
+*Tokens can be marshalled*
+
+The AWS secrets are held in a subclass of `org.apache.hadoop.security.token.TokenIdentifier`.
+This class gets serialized to a byte array when the whole token is marshalled, and deserialized
+when the token is loaded.
+
+*Tokens can be used to authenticate callers*
+
+The S3A FS does not hand the token to AWS services to authenticate the caller.
+Instead it takes the AWS credentials included in the token identifier
+and uses them to sign the requests.
+
+*Tokens cannot be renewed*
+
+The tokens contain the credentials; you cant use them to ask AWS for more.
+
+For full credentials that is moot, but for the session and role credentials,
+they will expire. At which point the application will be unable to
+talk to the AWS infrastructure.
+
+*Tokens cannot be revoked*
+
+The AWS STS APIs don't let you revoke a single set of session credentials.
+
+## Background: How Tokens are collected in MapReduce jobs
+
+
+### `org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal()`
+
+1. Calls `org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes()`
+for the job submission dir on the cluster FS (i.e. `fs.defaultFS`).
+1. Reads in the property `mapreduce.job.hdfs-servers` and extracts DTs from them,
+1. Tells the `FileInputFormat` and `FileOutputFormat` subclasses of the job
+to collect their source and dest FS tokens.
+
+All token collection is via `TokenCache.obtainTokensForNamenodes()`
+
+### `TokenCache.obtainTokensForNamenodes(Credentials, Path[], Configuration) `
+
+1. Returns immediately if security is off.
+1. Retrieves all the filesystems in the list of paths.
+1. Retrieves a token from each unless it is in the list of filesystems in `mapreduce.job.hdfs-servers.token-renewal.exclude`
+1. Merges in any DTs stored in the file referenced under: `mapreduce.job.credentials.binary`
+1. Calls `FileSystem.collectDelegationTokens()`, which, if there isn't any token already in the credential list, issues and adds a new token. *There is no check to see if that existing credential has expired*.
+
+
+### `FileInputFormat.listStatus(JobConf job): FileStatus[]`
+
+Enumerates source paths in (`mapreduce.input.fileinputformat.inputdir`) ; uses `TokenCache.obtainTokensForNamenodes()`
+to collate a token for all of these paths.
+
+This operation is called by the public interface method `FileInputFormat.getSplits()`.
+
+### `FileOutputFormat.checkOutputSpecs()`
+
+Calls `getOutputPath(job)` and asks for the DTs of that output path FS.
+
+
+## Architecture of the S3A Delegation Token Support
+
+
+
+1. The S3A FS client has the ability to be configured with a delegation
+token binding, the "DT Binding", a class declared in the option `fs.s3a.delegation.token.binding`.
+1. If set, when a filesystem is instantiated it asks the DT binding for its list of AWS credential providers.
+(the list in `fs.s3a.aws.credentials.provider` are only used if the DT binding wishes to).
+1. The DT binding scans for the current principal (`UGI.getCurrentUser()`/"the Owner") to see if they
+have any token in their credential cache whose service name matches the URI of the filesystem.
+1. If one is found, it is unmarshalled and then used to authenticate the caller via
+some AWS Credential provider returned to the S3A FileSystem instance.
+1. If none is found, the Filesystem is considered to have been deployed "Unbonded".
+The DT binding has to return a list of the AWS credential providers to use.
+
+When requests are made of AWS services, the created credential provider(s) are
+used to sign requests.
+
+When the filesystem is asked for a delegation token, the
+DT binding will generate a token identifier containing the marshalled tokens.
+
+If the Filesystem was deployed with a DT, that is, it was deployed "bonded", that existing
+DT is returned.
+
+If it was deployed unbonded, the DT Binding is asked to create a new DT.
+
+It is up to the binding what it includes in the token identifier, and how it obtains them.
+This new token identifier is included in a token which has a "canonical service name" of
+the URI of the filesystem (e.g "s3a://landsat-pds").
+
+The issued/reissued token identifier can be marshalled and reused.
+
+
+### class `org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens`
+
+This joins up the S3A Filesystem with the pluggable DT binding classes.
+
+One is instantiated in the S3A Filesystem instance if a DT Binding class
+has been instantiated. If so, it is invoked for
+
+* Building up the authentication chain during filesystem initialization.
+* Determining if the FS should declare that it has a canonical name
+(in `getCanonicalServiceName()`).
+* When asked for a DT (in `getDelegationToken(String renewer)`).
+
+The `S3ADelegationTokens` has the task of instantiating the actual DT binding,
+which must be a subclass of `AbstractDelegationTokenBinding`.
+
+All the DT bindings, and `S3ADelegationTokens` itself are subclasses of
+`org.apache.hadoop.service.AbstractService`; they follow the YARN service lifecycle
+of: create -> init -> start -> stop. This means that a DT binding, may, if it chooses,
+start worker threads when the service is started (`serviceStart()`); it must
+then stop them in the `serviceStop` method. (Anyone doing this must be aware
+that the owner FS is not fully initialized in serviceStart: they must not
+call into the Filesystem).
+
+The actions of this class are
+
+* Lookup of DTs associated with this S3A FS (scanning credentials, unmarshalling).
+* initiating the DT binding in bound/unbound state.
+* issuing DTs, either serving up the existing one, or requesting the DT Binding for
+a new instance of `AbstractS3ATokenIdentifier` and then wrapping a hadoop token
+around it.
+* General logging, debugging, and metrics. Delegation token metrics are
+collected in (`S3AInstrumentation.DelegationTokenStatistics`)
+
+
+
+
+### class `org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier`
+
+All tokens returned are a subclass of `AbstractS3ATokenIdentifier`.
+
+This class contains the following fields:
+
+```java
+  /** Canonical URI of the bucket. */
+  private URI uri;
+
+  /**
+   * Encryption secrets to also marshall with any credentials.
+   * Set during creation to ensure it is never null.
+   */
+  private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
+
+  /**
+   * Timestamp of creation.
+   * This is set to the current time; it will be overridden when
+   * deserializing data.
+   */
+  private long created = System.currentTimeMillis();
+
+  /**
+   * An origin string for diagnostics.
+   */
+  private String origin = "";
+
+  /**
+   * This marshalled UUID can be used in testing to verify transmission,
+   * and reuse; as it is printed you can see what is happending too.
+   */
+  private String uuid = UUID.randomUUID().toString();
+```
+
+The `uuid` field is used for equality tests and debugging; the `origin` and
+`created` fields are also for diagnostics.
+
+The `encryptionSecrets` structure enumerates the AWS encryption mechanism
+of the filesystem instance, and any declared key. This allows
+the client-side secret for SSE-C encryption to be passed to the filesystem,
+or the key name for SSE-KMS.
+
+*The encryption settings and secrets of the S3A filesystem on the client
+are included in the DT, so can be used to encrypt/decrypt data in the cluster.*
+
+### class `SessionTokenIdentifier` extends `AbstractS3ATokenIdentifier`
+
+This holds session tokens, and it also gets used as a superclass of
+the other token identifiers.
+
+It adds a set of `MarshalledCredentials` containing the session secrets.
+
+Every token/token identifier must have a unique *Kind*; this is how token
+identifier deserializers are looked up. For Session Credentials, it is
+`S3ADelegationToken/Session`. Subclasses *must* have a different token kind,
+else the unmarshalling and binding mechanism will fail.
+
+
+### classes `RoleTokenIdentifier` and `FullCredentialsTokenIdentifier`
+
+These are subclasses of `SessionTokenIdentifier` with different token kinds,
+needed for that token unmarshalling.
+
+Their kinds are `S3ADelegationToken/Role` and `S3ADelegationToken/Full`
+respectively.
+
+Having different possible token bindings raises the risk that a job is submitted
+with one binding and yet the cluster is expecting another binding.
+Provided the configuration option `fs.s3a.delegation.token.binding` is not
+marked as final in the `core-site.xml` file, the value of that binding
+set in the job should propagate with the binding: the choice of provider
+is automatic. A cluster can even mix bindings across jobs.
+However if a core-site XML file declares a specific binding for a single bucket and
+the job only had the generic `fs.s3a.delegation.token.binding`` binding,
+then there will be a mismatch.
+Each binding must be rigorous about checking the Kind of any found delegation
+token and failing meaningfully here.
+
+
+
+### class `MarshalledCredentials`
+
+Can marshall a set of AWS credentials (access key, secret key, session token)
+as a Hadoop Writable.
+
+These can be given to an instance of class `MarshalledCredentialProvider`
+and used to sign AWS RPC/REST API calls.
+
+## DT Binding: `AbstractDelegationTokenBinding`
+
+The plugin point for this design is the DT binding, which must be a subclass
+of `org.apache.hadoop.fs.s3a.auth.delegation.AbstractDelegationTokenBinding`.
+
+
+This class
+
+* Returns the *Kind* of these tokens.
+* declares whether tokens will actually  be issued or not (the TokenIssuingPolicy).
+* can issue a DT in
+
+```java