HADOOP-18965. ITestS3AHugeFilesEncryption failure (#6261)

Followup to:
HADOOP-18850 Enable dual-layer server-side encryption with AWS KMS keys (DSSE-KMS)

Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2023-11-24 17:24:12 +00:00 committed by GitHub
parent 88e760bfc8
commit 476b90f3e5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.EncryptionTestUtils; import org.apache.hadoop.fs.s3a.EncryptionTestUtils;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem;
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
@ -32,6 +33,7 @@
import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.SSE_KMS; import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.SSE_KMS;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet; import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet;
import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm;
import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey; import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey;
/** /**
@ -61,20 +63,30 @@ protected String getBlockOutputBufferName() {
*/ */
@Override @Override
protected boolean isEncrypted(S3AFileSystem fileSystem) { protected boolean isEncrypted(S3AFileSystem fileSystem) {
Configuration c = new Configuration(); Configuration conf = new Configuration();
return StringUtils.isNotBlank(getS3EncryptionKey(getTestBucketName(c), c)); return StringUtils.isNotBlank(getS3EncryptionKey(getTestBucketName(conf), conf));
} }
/**
* This test suite will run if the algorithm is set to SSE_KMS or DSSE_KMS;
* the assertions validate this.
* @param hugeFile file to validate.
* @throws IOException problems with encryption lookup.
* @throws AssertionError if the encryption is not as expected.
*/
@Override @Override
protected void assertEncrypted(Path hugeFile) throws IOException { protected void assertEncrypted(Path hugeFile) throws IOException {
Configuration c = new Configuration(); Configuration conf = new Configuration();
String kmsKey = getS3EncryptionKey(getTestBucketName(c), c);
if (SSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) { final String bucket = getTestBucketName(conf);
String kmsKey = getS3EncryptionKey(bucket, conf);
final S3AEncryptionMethods algorithm = getEncryptionAlgorithm(bucket, conf);
if (SSE_KMS.equals(algorithm)) {
EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, SSE_KMS, kmsKey); EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, SSE_KMS, kmsKey);
} else if (DSSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) { } else if (DSSE_KMS.equals(algorithm)) {
EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, DSSE_KMS, kmsKey); EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, DSSE_KMS, kmsKey);
} else { } else {
throw new AssertionError("Invalid encryption configured"); throw new AssertionError("Invalid encryption configured: " + algorithm);
} }
} }
} }