Skip to content

Commit fabd530

Browse files
steveloughranjiajunmao
authored andcommitted
HADOOP-18965. ITestS3AHugeFilesEncryption failure (apache#6261)
Followup to: HADOOP-18850 Enable dual-layer server-side encryption with AWS KMS keys (DSSE-KMS) Contributed by Steve Loughran
1 parent 2720e73 commit fabd530

File tree

1 file changed

+19
-7
lines changed

1 file changed

+19
-7
lines changed

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesEncryption.java

Lines changed: 19 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -25,13 +25,15 @@
2525
import org.apache.hadoop.fs.Path;
2626
import org.apache.hadoop.fs.s3a.Constants;
2727
import org.apache.hadoop.fs.s3a.EncryptionTestUtils;
28+
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
2829
import org.apache.hadoop.fs.s3a.S3AFileSystem;
2930

3031
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
3132
import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.DSSE_KMS;
3233
import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.SSE_KMS;
3334
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
3435
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet;
36+
import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm;
3537
import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey;
3638

3739
/**
@@ -61,20 +63,30 @@ protected String getBlockOutputBufferName() {
6163
*/
6264
@Override
6365
protected boolean isEncrypted(S3AFileSystem fileSystem) {
64-
Configuration c = new Configuration();
65-
return StringUtils.isNotBlank(getS3EncryptionKey(getTestBucketName(c), c));
66+
Configuration conf = new Configuration();
67+
return StringUtils.isNotBlank(getS3EncryptionKey(getTestBucketName(conf), conf));
6668
}
6769

70+
/**
71+
* This test suite will run if the algorithm is set to SSE_KMS or DSSE_KMS;
72+
* the assertions validate this.
73+
* @param hugeFile file to validate.
74+
* @throws IOException problems with encryption lookup.
75+
* @throws AssertionError if the encryption is not as expected.
76+
*/
6877
@Override
6978
protected void assertEncrypted(Path hugeFile) throws IOException {
70-
Configuration c = new Configuration();
71-
String kmsKey = getS3EncryptionKey(getTestBucketName(c), c);
72-
if (SSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) {
79+
Configuration conf = new Configuration();
80+
81+
final String bucket = getTestBucketName(conf);
82+
String kmsKey = getS3EncryptionKey(bucket, conf);
83+
final S3AEncryptionMethods algorithm = getEncryptionAlgorithm(bucket, conf);
84+
if (SSE_KMS.equals(algorithm)) {
7385
EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, SSE_KMS, kmsKey);
74-
} else if (DSSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) {
86+
} else if (DSSE_KMS.equals(algorithm)) {
7587
EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, DSSE_KMS, kmsKey);
7688
} else {
77-
throw new AssertionError("Invalid encryption configured");
89+
throw new AssertionError("Invalid encryption configured: " + algorithm);
7890
}
7991
}
8092
}

0 commit comments

Comments
 (0)