浏览代码

HADOOP-16477. S3A delegation token tests fail if fs.s3a.encryption.key set.

Contributed by Steve Loughran.

Change-Id: I843989f32472bbdefbd4fa504b26c7a614ab1cee
Steve Loughran 5 年之前
父节点
当前提交
f6697aa82b
共有 14 个文件被更改,包括 224 次插入69 次删除
  1. 2 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
  2. 3 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java
  3. 57 4
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
  4. 67 13
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractTestS3AEncryption.java
  5. 14 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
  6. 21 25
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
  7. 4 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java
  8. 13 7
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKey.java
  9. 14 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
  10. 5 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  11. 6 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
  12. 2 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/AbstractDelegationIT.java
  13. 11 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
  14. 5 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java

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

@@ -457,7 +457,7 @@ public class WriteOperationHelper {
   @Retries.RetryTranslated
   @Retries.RetryTranslated
   public PutObjectResult putObject(PutObjectRequest putObjectRequest)
   public PutObjectResult putObject(PutObjectRequest putObjectRequest)
       throws IOException {
       throws IOException {
-    return retry("put",
+    return retry("Writing Object",
         putObjectRequest.getKey(), true,
         putObjectRequest.getKey(), true,
         () -> owner.putObjectDirect(putObjectRequest));
         () -> owner.putObjectDirect(putObjectRequest));
   }
   }
@@ -472,7 +472,7 @@ public class WriteOperationHelper {
   public UploadResult uploadObject(PutObjectRequest putObjectRequest)
   public UploadResult uploadObject(PutObjectRequest putObjectRequest)
       throws IOException {
       throws IOException {
     // no retry; rely on xfer manager logic
     // no retry; rely on xfer manager logic
-    return retry("put",
+    return retry("Writing Object",
         putObjectRequest.getKey(), true,
         putObjectRequest.getKey(), true,
         () -> owner.executePut(putObjectRequest, null));
         () -> owner.executePut(putObjectRequest, null));
   }
   }

+ 3 - 3
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java

@@ -55,7 +55,7 @@ public final class RolePolicies {
   /**
   /**
    * Arn for all KMS keys: {@value}.
    * Arn for all KMS keys: {@value}.
    */
    */
-  public static final String KMS_ALL_KEYS = "arn:aws:kms:*";
+  public static final String KMS_ALL_KEYS = "*";
 
 
   /**
   /**
    * This is used by S3 to generate a per-object encryption key and
    * This is used by S3 to generate a per-object encryption key and
@@ -68,7 +68,7 @@ public final class RolePolicies {
    * Actions needed to read and write SSE-KMS data.
    * Actions needed to read and write SSE-KMS data.
    */
    */
   private static final String[] KMS_KEY_RW =
   private static final String[] KMS_KEY_RW =
-      new String[]{KMS_DECRYPT, KMS_GENERATE_DATA_KEY};
+      new String[]{KMS_DECRYPT, KMS_GENERATE_DATA_KEY, KMS_ENCRYPT};
 
 
   /**
   /**
    * Actions needed to read SSE-KMS data.
    * Actions needed to read SSE-KMS data.
@@ -81,7 +81,7 @@ public final class RolePolicies {
    * SSE-KMS.
    * SSE-KMS.
    */
    */
   public static final Statement STATEMENT_ALLOW_SSE_KMS_RW =
   public static final Statement STATEMENT_ALLOW_SSE_KMS_RW =
-      statement(true, KMS_ALL_KEYS, KMS_KEY_RW);
+      statement(true, KMS_ALL_KEYS, KMS_ALL_OPERATIONS);
 
 
   /**
   /**
    * Statement to allow read access to KMS keys, so the ability
    * Statement to allow read access to KMS keys, so the ability

+ 57 - 4
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md

@@ -46,9 +46,25 @@ have access to the appropriate KMS keys.
 Trying to learn how IAM Assumed Roles work by debugging stack traces from
 Trying to learn how IAM Assumed Roles work by debugging stack traces from
 the S3A client is "suboptimal".
 the S3A client is "suboptimal".
 
 
-### <a name="how_it_works"></a> How the S3A connector support IAM Assumed Roles.
+### <a name="how_it_works"></a> How the S3A connector supports IAM Assumed Roles.
 
 
-To use assumed roles, the client must be configured to use the
+
+The S3A connector support IAM Assumed Roles in two ways:
+
+1. Using the full credentials on the client to request credentials for a specific
+    role -credentials which are then used for all the store operations.
+    This can be used to verify that a specific role has the access permissions
+    you need, or to "su" into a role which has permissions that's the full
+    accounts does not directly qualify for -such as access to a KMS key.
+2. Using the full credentials to request role credentials which are then
+    propagated into a launched application as delegation tokens.
+    This extends the previous use as it allows the jobs to be submitted to a
+    shared cluster with the permissions of the requested role, rather than
+    those of the VMs/Containers of the deployed cluster.
+
+For Delegation Token integration, see (Delegation Tokens)[delegation_tokens.html]
+
+To for Assumed Role authentication, the client must be configured to use the
 *Assumed Role Credential Provider*, `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider`,
 *Assumed Role Credential Provider*, `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider`,
 in the configuration option `fs.s3a.aws.credentials.provider`.
 in the configuration option `fs.s3a.aws.credentials.provider`.
 
 
@@ -298,7 +314,7 @@ Without these permissions, tables cannot be created, destroyed or have their IO
 changed through the `s3guard set-capacity` call.
 changed through the `s3guard set-capacity` call.
 The `dynamodb:Scan` permission is needed for `s3guard prune`
 The `dynamodb:Scan` permission is needed for `s3guard prune`
 
 
-The `dynamodb:CreateTable` permission is needed by a client it tries to
+The `dynamodb:CreateTable` permission is needed by a client when it tries to
 create the DynamoDB table on startup, that is
 create the DynamoDB table on startup, that is
 `fs.s3a.s3guard.ddb.table.create` is `true` and the table does not already exist.
 `fs.s3a.s3guard.ddb.table.create` is `true` and the table does not already exist.
 
 
@@ -758,14 +774,51 @@ Make sure that all the read and write permissions are allowed for any bucket/pat
 to which data is being written to, and read permissions for all
 to which data is being written to, and read permissions for all
 buckets read from.
 buckets read from.
 
 
+### <a name="access_denied_kms"></a> `AccessDeniedException` When working with KMS-encrypted data
+
 If the bucket is using SSE-KMS to encrypt data:
 If the bucket is using SSE-KMS to encrypt data:
 
 
 1. The caller must have the `kms:Decrypt` permission to read the data.
 1. The caller must have the `kms:Decrypt` permission to read the data.
-1. The caller needs `kms:Decrypt` and `kms:GenerateDataKey`.
+1. The caller needs `kms:Decrypt` and `kms:GenerateDataKey` to write data.
 
 
 Without permissions, the request fails *and there is no explicit message indicating
 Without permissions, the request fails *and there is no explicit message indicating
 that this is an encryption-key issue*.
 that this is an encryption-key issue*.
 
 
+This problem is most obvious when you fail when writing data in a "Writing Object" operation.
+
+If the client does have write access to the bucket, verify that the caller has
+`kms:GenerateDataKey` permissions for the encryption key in use.
+
+```
+java.nio.file.AccessDeniedException: test/testDTFileSystemClient: Writing Object on test/testDTFileSystemClient:
+  com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; 
+  Error Code: AccessDenied; Request ID: E86544FF1D029857)
+
+    at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:243)
+    at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
+    at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:314)
+    at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:406)
+    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:310)
+    at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:285)
+    at org.apache.hadoop.fs.s3a.WriteOperationHelper.retry(WriteOperationHelper.java:150)
+    at org.apache.hadoop.fs.s3a.WriteOperationHelper.putObject(WriteOperationHelper.java:460)
+    at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.lambda$putObject$0(S3ABlockOutputStream.java:438)
+    at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219)
+    at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219)
+    at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
+    at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:57)
+    at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
+    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
+    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
+    at java.lang.Thread.run(Thread.java:748)
+Caused by:  com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403;
+            Error Code: AccessDenied; Request ID: E86544FF1D029857)
+```
+
+Note: the ability to read encrypted data in the store does not guarantee that the caller can encrypt new data.
+It is a separate permission.
+
+
 ### <a name="dynamodb_exception"></a> `AccessDeniedException` + `AmazonDynamoDBException`
 ### <a name="dynamodb_exception"></a> `AccessDeniedException` + `AmazonDynamoDBException`
 
 
 ```
 ```

+ 67 - 13
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractTestS3AEncryption.java

@@ -18,18 +18,25 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
+import java.io.IOException;
+
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.ObjectMetadata;
+import org.junit.Test;
+
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.net.util.Base64;
 import org.apache.commons.net.util.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Test;
-
-import java.io.IOException;
+import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled;
+import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm;
 
 
 /**
 /**
  * Test whether or not encryption works by turning it on. Some checks
  * Test whether or not encryption works by turning it on. Some checks
@@ -38,11 +45,18 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
  */
  */
 public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
 public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
 
 
+  protected static final String AWS_KMS_SSE_ALGORITHM = "aws:kms";
+
+  protected static final String SSE_C_ALGORITHM = "AES256";
+
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     Configuration conf = super.createConfiguration();
     S3ATestUtils.disableFilesystemCaching(conf);
     S3ATestUtils.disableFilesystemCaching(conf);
-    conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+    removeBaseAndBucketOverrides(conf,
+        SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        SERVER_SIDE_ENCRYPTION_KEY);
+    conf.set(SERVER_SIDE_ENCRYPTION_ALGORITHM,
             getSSEAlgorithm().getMethod());
             getSSEAlgorithm().getMethod());
     return conf;
     return conf;
   }
   }
@@ -51,8 +65,34 @@ public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
       0, 1, 2, 3, 4, 5, 254, 255, 256, 257, 2 ^ 12 - 1
       0, 1, 2, 3, 4, 5, 254, 255, 256, 257, 2 ^ 12 - 1
   };
   };
 
 
+  protected void requireEncryptedFileSystem() {
+    skipIfEncryptionTestsDisabled(getFileSystem().getConf());
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    requireEncryptedFileSystem();
+  }
+
+  /**
+   * This examines how encryption settings propagate better.
+   * If the settings are actually in a JCEKS file, then the
+   * test override will fail; this is here to help debug the problem.
+   */
+  @Test
+  public void testEncryptionSettingPropagation() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    S3AEncryptionMethods algorithm = getEncryptionAlgorithm(
+        fs.getBucket(), fs.getConf());
+    assertEquals("Configuration has wrong encryption algorithm",
+        getSSEAlgorithm(), algorithm);
+  }
+
   @Test
   @Test
   public void testEncryption() throws Throwable {
   public void testEncryption() throws Throwable {
+    requireEncryptedFileSystem();
+    validateEncrytionSecrets(getFileSystem().getEncryptionSecrets());
     for (int size: SIZES) {
     for (int size: SIZES) {
       validateEncryptionForFilesize(size);
       validateEncryptionForFilesize(size);
     }
     }
@@ -60,10 +100,11 @@ public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
 
 
   @Test
   @Test
   public void testEncryptionOverRename() throws Throwable {
   public void testEncryptionOverRename() throws Throwable {
-    skipIfEncryptionTestsDisabled(getConfiguration());
     Path src = path(createFilename(1024));
     Path src = path(createFilename(1024));
     byte[] data = dataset(1024, 'a', 'z');
     byte[] data = dataset(1024, 'a', 'z');
     S3AFileSystem fs = getFileSystem();
     S3AFileSystem fs = getFileSystem();
+    EncryptionSecrets secrets = fs.getEncryptionSecrets();
+    validateEncrytionSecrets(secrets);
     writeDataset(fs, src, data, data.length, 1024 * 1024, true);
     writeDataset(fs, src, data, data.length, 1024 * 1024, true);
     ContractTestUtils.verifyFileContents(fs, src, data);
     ContractTestUtils.verifyFileContents(fs, src, data);
     Path dest = path(src.getName() + "-copy");
     Path dest = path(src.getName() + "-copy");
@@ -72,8 +113,19 @@ public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
     assertEncrypted(dest);
     assertEncrypted(dest);
   }
   }
 
 
+  /**
+   * Verify that the filesystem encryption secrets match expected.
+   * This makes sure that the settings have propagated properly.
+   * @param secrets encryption secrets of the filesystem.
+   */
+  protected void validateEncrytionSecrets(final EncryptionSecrets secrets) {
+    assertNotNull("No encryption secrets for filesystem", secrets);
+    S3AEncryptionMethods sseAlgorithm = getSSEAlgorithm();
+    assertEquals("Filesystem has wrong encryption algorithm",
+        sseAlgorithm, secrets.getEncryptionMethod());
+  }
+
   protected void validateEncryptionForFilesize(int len) throws IOException {
   protected void validateEncryptionForFilesize(int len) throws IOException {
-    skipIfEncryptionTestsDisabled(getConfiguration());
     describe("Create an encrypted file of size " + len);
     describe("Create an encrypted file of size " + len);
     String src = createFilename(len);
     String src = createFilename(len);
     Path path = writeThenReadFile(src, len);
     Path path = writeThenReadFile(src, len);
@@ -98,15 +150,17 @@ public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
     ObjectMetadata md = getFileSystem().getObjectMetadata(path);
     ObjectMetadata md = getFileSystem().getObjectMetadata(path);
     switch(getSSEAlgorithm()) {
     switch(getSSEAlgorithm()) {
     case SSE_C:
     case SSE_C:
-      assertEquals("AES256", md.getSSECustomerAlgorithm());
+      assertNull("Metadata algorithm should have been null",
+          md.getSSEAlgorithm());
+      assertEquals("Wrong SSE-C algorithm", SSE_C_ALGORITHM, md.getSSECustomerAlgorithm());
       String md5Key = convertKeyToMd5();
       String md5Key = convertKeyToMd5();
-      assertEquals(md5Key, md.getSSECustomerKeyMd5());
+      assertEquals("getSSECustomerKeyMd5() wrong", md5Key, md.getSSECustomerKeyMd5());
       break;
       break;
     case SSE_KMS:
     case SSE_KMS:
-      assertEquals("aws:kms", md.getSSEAlgorithm());
+      assertEquals(AWS_KMS_SSE_ALGORITHM, md.getSSEAlgorithm());
       //S3 will return full arn of the key, so specify global arn in properties
       //S3 will return full arn of the key, so specify global arn in properties
       assertEquals(this.getConfiguration().
       assertEquals(this.getConfiguration().
-          getTrimmed(Constants.SERVER_SIDE_ENCRYPTION_KEY),
+          getTrimmed(SERVER_SIDE_ENCRYPTION_KEY),
           md.getSSEAwsKmsKeyId());
           md.getSSEAwsKmsKeyId());
       break;
       break;
     default:
     default:
@@ -123,8 +177,8 @@ public abstract class AbstractTestS3AEncryption extends AbstractS3ATestBase {
    * key
    * key
    */
    */
   private String convertKeyToMd5() {
   private String convertKeyToMd5() {
-    String base64Key = getConfiguration().getTrimmed(
-        Constants.SERVER_SIDE_ENCRYPTION_KEY
+    String base64Key = getFileSystem().getConf().getTrimmed(
+        SERVER_SIDE_ENCRYPTION_KEY
     );
     );
     byte[] key = Base64.decodeBase64(base64Key);
     byte[] key = Base64.decodeBase64(base64Key);
     byte[] md5 =  DigestUtils.md5(key);
     byte[] md5 =  DigestUtils.md5(key);

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.file.AccessDeniedException;
 import java.nio.file.AccessDeniedException;
+import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -38,9 +39,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 /**
 /**
@@ -51,11 +53,11 @@ public class ITestS3AAWSCredentialsProvider {
       LoggerFactory.getLogger(ITestS3AAWSCredentialsProvider.class);
       LoggerFactory.getLogger(ITestS3AAWSCredentialsProvider.class);
 
 
   @Rule
   @Rule
-  public Timeout testTimeout = new Timeout(1 * 60 * 1000);
+  public Timeout testTimeout = new Timeout(60_1000, TimeUnit.MILLISECONDS);
 
 
   @Test
   @Test
   public void testBadConfiguration() throws IOException {
   public void testBadConfiguration() throws IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = createConf();
     conf.set(AWS_CREDENTIALS_PROVIDER, "no.such.class");
     conf.set(AWS_CREDENTIALS_PROVIDER, "no.such.class");
     try {
     try {
       createFailingFS(conf);
       createFailingFS(conf);
@@ -93,7 +95,7 @@ public class ITestS3AAWSCredentialsProvider {
 
 
   @Test
   @Test
   public void testBadCredentialsConstructor() throws Exception {
   public void testBadCredentialsConstructor() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = createConf();
     conf.set(AWS_CREDENTIALS_PROVIDER,
     conf.set(AWS_CREDENTIALS_PROVIDER,
         BadCredentialsProviderConstructor.class.getName());
         BadCredentialsProviderConstructor.class.getName());
     try {
     try {
@@ -103,6 +105,14 @@ public class ITestS3AAWSCredentialsProvider {
     }
     }
   }
   }
 
 
+  protected Configuration createConf() {
+    Configuration conf = new Configuration();
+    removeBaseAndBucketOverrides(conf,
+        DELEGATION_TOKEN_BINDING,
+        AWS_CREDENTIALS_PROVIDER);
+    return conf;
+  }
+
   /**
   /**
    * Create a filesystem, expect it to fail by raising an IOException.
    * Create a filesystem, expect it to fail by raising an IOException.
    * Raises an assertion exception if in fact the FS does get instantiated.
    * Raises an assertion exception if in fact the FS does get instantiated.

+ 21 - 25
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java

@@ -67,6 +67,12 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     return conf;
     return conf;
   }
   }
 
 
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    assumeEnabled();
+  }
+
   @Override
   @Override
   public void teardown() throws Exception {
   public void teardown() throws Exception {
     super.teardown();
     super.teardown();
@@ -85,9 +91,6 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   @Test
   @Test
   public void testCreateFileAndReadWithDifferentEncryptionKey() throws
   public void testCreateFileAndReadWithDifferentEncryptionKey() throws
       Exception {
       Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-
     intercept(AccessDeniedException.class,
     intercept(AccessDeniedException.class,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
         () -> {
         () -> {
@@ -117,9 +120,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testCreateSubdirWithDifferentKey() throws Exception {
   public void testCreateSubdirWithDifferentKey() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-    assumeS3GuardState(false, getConfiguration());
+    requireUnguardedFilesystem();
 
 
     intercept(AccessDeniedException.class,
     intercept(AccessDeniedException.class,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
@@ -145,9 +146,6 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testCreateFileThenMoveWithDifferentSSECKey() throws Exception {
   public void testCreateFileThenMoveWithDifferentSSECKey() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-
     intercept(AccessDeniedException.class,
     intercept(AccessDeniedException.class,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
         SERVICE_AMAZON_S3_STATUS_CODE_403,
         () -> {
         () -> {
@@ -169,9 +167,6 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testRenameFile() throws Exception {
   public void testRenameFile() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-
     Path src = path("original-path.txt");
     Path src = path("original-path.txt");
     writeThenReadFile(src, TEST_FILE_LEN);
     writeThenReadFile(src, TEST_FILE_LEN);
     Path newPath = path("different-path.txt");
     Path newPath = path("different-path.txt");
@@ -188,9 +183,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testListEncryptedDir() throws Exception {
   public void testListEncryptedDir() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-    assumeS3GuardState(false, getConfiguration());
+    requireUnguardedFilesystem();
 
 
     Path pathABC = path("testListEncryptedDir/a/b/c/");
     Path pathABC = path("testListEncryptedDir/a/b/c/");
     Path pathAB = pathABC.getParent();
     Path pathAB = pathABC.getParent();
@@ -235,9 +228,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testListStatusEncryptedDir() throws Exception {
   public void testListStatusEncryptedDir() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-    assumeS3GuardState(false, getConfiguration());
+    requireUnguardedFilesystem();
 
 
     Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
     Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
     Path pathAB = pathABC.getParent();
     Path pathAB = pathABC.getParent();
@@ -282,11 +273,9 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testListStatusEncryptedFile() throws Exception {
   public void testListStatusEncryptedFile() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-    assumeS3GuardState(false, getConfiguration());
+    requireUnguardedFilesystem();
     Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
     Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
-    assertTrue(getFileSystem().mkdirs(pathABC));
+    assertTrue("mkdirs failed", getFileSystem().mkdirs(pathABC));
 
 
     Path fileToStat = new Path(pathABC, "fileToStat.txt");
     Path fileToStat = new Path(pathABC, "fileToStat.txt");
     writeThenReadFile(fileToStat, TEST_FILE_LEN);
     writeThenReadFile(fileToStat, TEST_FILE_LEN);
@@ -301,6 +290,15 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
         });
         });
   }
   }
 
 
+  /**
+   * Skip the test case if S3Guard is enabled; generally this is because
+   * list and GetFileStatus calls can succeed even with different keys.
+   */
+  protected void requireUnguardedFilesystem() {
+    assume("Filesystem has a metastore",
+        !getFileSystem().hasMetadataStore());
+  }
+
 
 
   /**
   /**
    * It is possible to delete directories without the proper encryption key and
    * It is possible to delete directories without the proper encryption key and
@@ -310,9 +308,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
    */
   @Test
   @Test
   public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
   public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
-    assumeEnabled();
-    skipIfEncryptionTestsDisabled(getConfiguration());
-    assumeS3GuardState(false, getConfiguration());
+    requireUnguardedFilesystem();
     Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
     Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
 
 
     Path pathAB = pathABC.getParent();
     Path pathAB = pathABC.getParent();

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

@@ -18,14 +18,15 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
-import static org.hamcrest.CoreMatchers.containsString;
-
 import java.io.IOException;
 import java.io.IOException;
 
 
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.ObjectMetadata;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 
 
+import static org.hamcrest.CoreMatchers.containsString;
+
 /**
 /**
  * Concrete class that extends {@link AbstractTestS3AEncryption}
  * Concrete class that extends {@link AbstractTestS3AEncryption}
  * and tests SSE-KMS encryption when no KMS encryption key is provided and AWS
  * and tests SSE-KMS encryption when no KMS encryption key is provided and AWS
@@ -51,7 +52,7 @@ public class ITestS3AEncryptionSSEKMSDefaultKey
   @Override
   @Override
   protected void assertEncrypted(Path path) throws IOException {
   protected void assertEncrypted(Path path) throws IOException {
     ObjectMetadata md = getFileSystem().getObjectMetadata(path);
     ObjectMetadata md = getFileSystem().getObjectMetadata(path);
-    assertEquals("aws:kms", md.getSSEAlgorithm());
+    assertEquals("SSE Algorithm", AWS_KMS_SSE_ALGORITHM, md.getSSEAlgorithm());
     assertThat(md.getSSEAwsKmsKeyId(), containsString("arn:aws:kms:"));
     assertThat(md.getSSEAwsKmsKeyId(), containsString("arn:aws:kms:"));
   }
   }
 }
 }

+ 13 - 7
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKey.java

@@ -18,11 +18,13 @@
 
 
 package org.apache.hadoop.fs.s3a;
 package org.apache.hadoop.fs.s3a;
 
 
-import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.SSE_KMS;
+
 /**
 /**
  * Concrete class that extends {@link AbstractTestS3AEncryption}
  * Concrete class that extends {@link AbstractTestS3AEncryption}
  * and tests SSE-KMS encryption.  This requires the SERVER_SIDE_ENCRYPTION_KEY
  * and tests SSE-KMS encryption.  This requires the SERVER_SIDE_ENCRYPTION_KEY
@@ -33,16 +35,20 @@ public class ITestS3AEncryptionSSEKMSUserDefinedKey
 
 
   @Override
   @Override
   protected Configuration createConfiguration() {
   protected Configuration createConfiguration() {
-    Configuration conf = super.createConfiguration();
-    if(StringUtils.isBlank(conf.get(Constants.SERVER_SIDE_ENCRYPTION_KEY))){
-      skip(Constants.SERVER_SIDE_ENCRYPTION_KEY+ " is not set for " +
-          S3AEncryptionMethods.SSE_KMS.getMethod());
+    // get the KMS key for this test.
+    Configuration c = new Configuration();
+    String kmsKey = c.get(SERVER_SIDE_ENCRYPTION_KEY);
+    if (StringUtils.isBlank(kmsKey)){
+      skip(SERVER_SIDE_ENCRYPTION_KEY+ " is not set for " +
+          SSE_KMS.getMethod());
     }
     }
+    Configuration conf = super.createConfiguration();
+    conf.set(SERVER_SIDE_ENCRYPTION_KEY, kmsKey);
     return conf;
     return conf;
   }
   }
 
 
   @Override
   @Override
   protected S3AEncryptionMethods getSSEAlgorithm() {
   protected S3AEncryptionMethods getSSEAlgorithm() {
-    return S3AEncryptionMethods.SSE_KMS;
+    return SSE_KMS;
   }
   }
 }
 }

+ 14 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -28,6 +28,7 @@ import com.amazonaws.services.s3.model.PutObjectRequest;
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -39,6 +40,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertHasPathCapab
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertLacksPathCapabilities;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertLacksPathCapabilities;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 
 
 /**
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
  * Tests of the S3A FileSystem which don't have a specific home and can share
@@ -56,6 +60,15 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     enableChecksums(true);
     enableChecksums(true);
   }
   }
 
 
+  @Override
+  protected Configuration createConfiguration() {
+    final Configuration conf = super.createConfiguration();
+    removeBaseAndBucketOverrides(conf,
+        SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        SERVER_SIDE_ENCRYPTION_KEY);
+    return conf;
+  }
+
   /**
   /**
    * Turn checksums on.
    * Turn checksums on.
    * Relies on the FS not caching the configuration option
    * Relies on the FS not caching the configuration option
@@ -149,7 +162,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         CommonPathCapabilities.FS_CHECKSUMS);
         CommonPathCapabilities.FS_CHECKSUMS);
     assertNotNull("Null file 1 checksum", checksum1);
     assertNotNull("Null file 1 checksum", checksum1);
     assertNotEquals("file 1 checksum", 0, checksum1.getLength());
     assertNotEquals("file 1 checksum", 0, checksum1.getLength());
-    assertEquals("checksums", checksum1,
+    assertEquals("checksums of empty files", checksum1,
         fs.getFileChecksum(touchFile("file2"), 0));
         fs.getFileChecksum(touchFile("file2"), 0));
   }
   }
 
 

+ 5 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -771,6 +771,11 @@ public final class S3ATestUtils {
         LOG.debug("Removing option {}; was {}", target, v);
         LOG.debug("Removing option {}; was {}", target, v);
         conf.unset(target);
         conf.unset(target);
       }
       }
+      String extended = bucketPrefix + option;
+      if (conf.get(extended) != null) {
+        LOG.debug("Removing option {}", extended);
+        conf.unset(extended);
+      }
     }
     }
   }
   }
 
 

+ 6 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java

@@ -35,13 +35,14 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
@@ -153,11 +154,14 @@ public final class RoleTestUtils {
       final Configuration srcConf,
       final Configuration srcConf,
       final String roleARN) {
       final String roleARN) {
     Configuration conf = new Configuration(srcConf);
     Configuration conf = new Configuration(srcConf);
+    removeBaseAndBucketOverrides(conf,
+        DELEGATION_TOKEN_BINDING,
+        ASSUMED_ROLE_ARN,
+        AWS_CREDENTIALS_PROVIDER);
     conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
     conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME);
     conf.set(ASSUMED_ROLE_ARN, roleARN);
     conf.set(ASSUMED_ROLE_ARN, roleARN);
     conf.set(ASSUMED_ROLE_SESSION_NAME, "test");
     conf.set(ASSUMED_ROLE_SESSION_NAME, "test");
     conf.set(ASSUMED_ROLE_SESSION_DURATION, "15m");
     conf.set(ASSUMED_ROLE_SESSION_DURATION, "15m");
-    conf.unset(DelegationConstants.DELEGATION_TOKEN_BINDING);
     disableFilesystemCaching(conf);
     disableFilesystemCaching(conf);
     return conf;
     return conf;
   }
   }

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

@@ -148,6 +148,8 @@ public abstract class AbstractDelegationIT extends AbstractS3ATestBase {
    * @param binding binding to use
    * @param binding binding to use
    */
    */
   protected void enableDelegationTokens(Configuration conf, String binding) {
   protected void enableDelegationTokens(Configuration conf, String binding) {
+    removeBaseAndBucketOverrides(conf,
+        DELEGATION_TOKEN_BINDING);
     LOG.info("Enabling delegation token support for {}", binding);
     LOG.info("Enabling delegation token support for {}", binding);
     conf.set(DELEGATION_TOKEN_BINDING, binding);
     conf.set(DELEGATION_TOKEN_BINDING, binding);
   }
   }

+ 11 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java

@@ -139,6 +139,10 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
     // disable if assume role opts are off
     // disable if assume role opts are off
     assumeSessionTestsEnabled(conf);
     assumeSessionTestsEnabled(conf);
     disableFilesystemCaching(conf);
     disableFilesystemCaching(conf);
+    removeBaseAndBucketOverrides(conf,
+        DELEGATION_TOKEN_BINDING,
+        SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        SERVER_SIDE_ENCRYPTION_KEY);
     conf.set(HADOOP_SECURITY_AUTHENTICATION,
     conf.set(HADOOP_SECURITY_AUTHENTICATION,
         UserGroupInformation.AuthenticationMethod.KERBEROS.name());
         UserGroupInformation.AuthenticationMethod.KERBEROS.name());
     enableDelegationTokens(conf, getDelegationBinding());
     enableDelegationTokens(conf, getDelegationBinding());
@@ -332,6 +336,7 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
     removeBaseAndBucketOverrides(bucket, conf,
     removeBaseAndBucketOverrides(bucket, conf,
         ACCESS_KEY, SECRET_KEY, SESSION_TOKEN,
         ACCESS_KEY, SECRET_KEY, SESSION_TOKEN,
         SERVER_SIDE_ENCRYPTION_ALGORITHM,
         SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        SERVER_SIDE_ENCRYPTION_KEY,
         DELEGATION_TOKEN_ROLE_ARN,
         DELEGATION_TOKEN_ROLE_ARN,
         DELEGATION_TOKEN_ENDPOINT);
         DELEGATION_TOKEN_ENDPOINT);
     // this is done to make sure you cannot create an STS session no
     // this is done to make sure you cannot create an STS session no
@@ -347,8 +352,10 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
       LOG.info("Delegated filesystem is: {}", delegatedFS);
       LOG.info("Delegated filesystem is: {}", delegatedFS);
       assertBoundToDT(delegatedFS, tokenKind);
       assertBoundToDT(delegatedFS, tokenKind);
       if (encryptionTestEnabled()) {
       if (encryptionTestEnabled()) {
+        assertNotNull("Encryption propagation failed",
+            delegatedFS.getServerSideEncryptionAlgorithm());
         assertEquals("Encryption propagation failed",
         assertEquals("Encryption propagation failed",
-            S3AEncryptionMethods.SSE_S3,
+            fs.getServerSideEncryptionAlgorithm(),
             delegatedFS.getServerSideEncryptionAlgorithm());
             delegatedFS.getServerSideEncryptionAlgorithm());
       }
       }
       verifyRestrictedPermissions(delegatedFS);
       verifyRestrictedPermissions(delegatedFS);
@@ -380,8 +387,10 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
     try (S3AFileSystem secondDelegate = newS3AInstance(uri, conf)) {
     try (S3AFileSystem secondDelegate = newS3AInstance(uri, conf)) {
       assertBoundToDT(secondDelegate, tokenKind);
       assertBoundToDT(secondDelegate, tokenKind);
       if (encryptionTestEnabled()) {
       if (encryptionTestEnabled()) {
+        assertNotNull("Encryption propagation failed",
+            secondDelegate.getServerSideEncryptionAlgorithm());
         assertEquals("Encryption propagation failed",
         assertEquals("Encryption propagation failed",
-            S3AEncryptionMethods.SSE_S3,
+            fs.getServerSideEncryptionAlgorithm(),
             secondDelegate.getServerSideEncryptionAlgorithm());
             secondDelegate.getServerSideEncryptionAlgorithm());
       }
       }
       ContractTestUtils.assertDeleted(secondDelegate, testPath, true);
       ContractTestUtils.assertDeleted(secondDelegate, testPath, true);

+ 5 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java

@@ -66,6 +66,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
+import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
 import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths;
 import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths;
@@ -256,6 +257,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
     assumedRoleConfig = createAssumedRoleConfig();
     assumedRoleConfig = createAssumedRoleConfig();
     bindRolePolicyStatements(assumedRoleConfig,
     bindRolePolicyStatements(assumedRoleConfig,
         STATEMENT_S3GUARD_CLIENT,
         STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
         STATEMENT_ALL_BUCKET_READ_ACCESS,  // root:     r-x
         STATEMENT_ALL_BUCKET_READ_ACCESS,  // root:     r-x
         new Statement(Effects.Allow)       // dest:     rwx
         new Statement(Effects.Allow)       // dest:     rwx
             .addActions(S3_PATH_RW_OPERATIONS)
             .addActions(S3_PATH_RW_OPERATIONS)
@@ -311,9 +313,9 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
   private Configuration createAssumedRoleConfig(String roleARN) {
   private Configuration createAssumedRoleConfig(String roleARN) {
     Configuration conf = newAssumedRoleConfig(getContract().getConf(),
     Configuration conf = newAssumedRoleConfig(getContract().getConf(),
         roleARN);
         roleARN);
-    String bucketName = getTestBucketName(conf);
-
-    removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE);
+    removeBaseAndBucketOverrides(conf,
+        DELEGATION_TOKEN_BINDING,
+        ENABLE_MULTI_DELETE);
     conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete);
     conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete);
     return conf;
     return conf;
   }
   }