Skip to content

Commit e3d5922

Browse files
author
Mehakmeet Singh
committed
HADOOP-13887. review comments, docs + test fix
1 parent 7f81092 commit e3d5922

File tree

6 files changed

+180
-26
lines changed

6 files changed

+180
-26
lines changed

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -397,10 +397,9 @@ public void close() throws IOException {
397397
// PUT the final block
398398
if (hasBlock &&
399399
(block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
400-
// send last part and set the value of isLastPart to true in case of
401-
// CSE being enabled, since we are sure it is last part as parts
402-
// are being uploaded serially in CSE.
403-
uploadCurrentBlock(isCSEEnabled);
400+
// send last part and set the value of isLastPart to true.
401+
// Necessary to set this "true" in case of client side encryption.
402+
uploadCurrentBlock(true);
404403
}
405404
// wait for the partial uploads to finish
406405
final List<PartETag> partETags =

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -360,7 +360,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
360360
AuditIntegration.stubAuditManager();
361361

362362
/**
363-
* Is this S3AFS instance using S3 client side encryption?
363+
* Is this S3A FS instance using S3 client side encryption?
364364
*/
365365
private boolean isCSEEnabled;
366366

hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md

Lines changed: 29 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -87,19 +87,19 @@ Encryption options
8787

8888
With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
8989
encryption channel).
90-
The data is dynamically encrypted and decrypted in the S3 Store, as needed.
90+
The data is encrypted in the S3 store and decrypted when it's being retrieved.
9191

9292
A server side algorithm can be enabled by default for a bucket, so that
9393
whenever data is uploaded unencrypted a default encryption algorithm is added.
9494
When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
9595
downloading the data.
9696
SSE-C is different in that every client must know the secret key needed to decypt the data.
9797

98-
Working with SSE-C data hard because every client must be configured to use the
99-
algorithm and supply the key. In particular, it is very hard to mix SSE-C
100-
encrypted objects in the same S3 bucket with objects encrypted with other
101-
algorithms or unencrypted; The S3A client
102-
(and other applications) get very confused.
98+
Working with SSE-C data is harder because every client must be configured to
99+
use the algorithm and supply the key. In particular, it is very hard to mix
100+
SSE-C encrypted objects in the same S3 bucket with objects encrypted with
101+
other algorithms or unencrypted; The S3A client (and other applications) get
102+
very confused.
103103

104104
KMS-based key encryption is powerful as access to a key can be restricted to
105105
specific users/IAM roles. However, use of the key is billed and can be
@@ -113,15 +113,15 @@ This encrypts the data on the client, before transmitting to S3, where it is
113113
stored encrypted. The data is unencrypted after downloading when it is being
114114
read back.
115115

116-
in CSE-KMS, the ID of an AWS-KMS key is provided to the S3A client;
116+
In CSE-KMS, the ID of an AWS-KMS key is provided to the S3A client;
117117
the client communicates with AWS-KMS to request a new encryption key, which
118118
KMS returns along with the same key encrypted with the KMS key.
119119
The S3 client encrypts the payload *and* attaches the KMS-encrypted version
120120
of the key as a header to the object.
121121

122122
When downloading data, this header is extracted, passed to AWS KMS, and,
123-
if the client has the appropriate permissions, the symmetric key
124-
retrieved and returned.
123+
if the client has the appropriate permissions, the symmetric key is
124+
retrieved.
125125
This key is then used to decode the data.
126126

127127
## <a name="sse-s3"></a> S3 Default Encryption
@@ -479,7 +479,7 @@ them in the "XAttr" list of attributed, prefixed with `header.`.
479479
This makes them retrievable in the `getXAttr()` API calls, which
480480
is available on the command line through the `hadoop fs -getfattr -d` command.
481481

482-
This makes viewing the encryption headers of a file straightforward
482+
This makes viewing the encryption headers of a file straightforward.
483483

484484
Here is an example of the operation invoked on a file where the client is using CSE-KMS:
485485
```
@@ -505,8 +505,9 @@ header.x-amz-wrap-alg="kms+context"
505505

506506
Analysis
507507

508-
1. the WARN commands are the AWS SDK warning that because the S3A client uses an encryption algorithm which seek() requires,
509-
the SDK considers it less secure than the most recent algorithm(s). Ignore.
508+
1. The WARN commands are the AWS SDK warning that because the S3A client uses
509+
an encryption algorithm which seek() requires, the SDK considers it less
510+
secure than the most recent algorithm(s). Ignore.
510511

511512
* `header.x-amz-server-side-encryption="AES256"` : the file has been encrypted with S3-SSE. This is set up as the S3 default encryption,
512513
so even when CSE is enabled, the data is doubly encrypted.
@@ -518,7 +519,7 @@ so even when CSE is enabled, the data is doubly encrypted.
518519
it always removes 16 bytes from non-empty files when declaring the length.
519520
* `header.x-amz-version-id="zXccFCB9eICszFgqv_paG1pzaUKY09Xa"`: the bucket is versioned; this is the version ID.
520521

521-
And a directory encrypted with S3-SSE only
522+
And a directory encrypted with S3-SSE only:
522523

523524
```
524525
bin/hadoop fs -getfattr -d s3a://test-london/user/stevel/target/test/data/sOCOsNgEjv
@@ -532,7 +533,7 @@ header.x-amz-server-side-encryption="AES256"
532533
header.x-amz-version-id="KcDOVmznIagWx3gP1HlDqcZvm1mFWZ2a"
533534
```
534535

535-
A file with no-encryption (on a bucket without versioning but with intelligent tiering)
536+
A file with no-encryption (on a bucket without versioning but with intelligent tiering):
536537

537538
```
538539
bin/hadoop fs -getfattr -d s3a://landsat-pds/scene_list.gz
@@ -603,7 +604,7 @@ clients where S3-CSE has not been enabled.
603604
- Performance will be reduced. All encrypt/decrypt is now being done on the
604605
client.
605606
- Writing files may be slower, as only a single block can be encrypted and
606-
uploaded at a time
607+
uploaded at a time.
607608
- Multipart Uploader API disabled.
608609
- S3 Select is not supported.
609610
- Multipart uploads would be serial, and partSize must be a multiple of 16
@@ -619,6 +620,19 @@ clients where S3-CSE has not been enabled.
619620
- Set `fs.s3a.server-side-encryption-algorithm=CSE-KMS`.
620621
- Set `fs.s3a.server-side-encryption.key=<KMS_KEY_ID>`.
621622

623+
KMS_KEY_ID:
624+
625+
Identifies the symmetric CMK that encrypts the data key.
626+
To specify a CMK, use its key ID, key ARN, alias name, or alias ARN. When
627+
using an alias name, prefix it with "alias/". To specify a CMK in a
628+
different AWSaccount, you must use the key ARN or alias ARN.
629+
630+
For example:
631+
- Key ID: 1234abcd-12ab-34cd-56ef-1234567890ab
632+
- Key ARN: arn:aws:kms:us-east-2:111122223333:key/1234abcd-12ab-34cd-56ef-1234567890ab
633+
- Alias name: alias/ExampleAlias
634+
- Alias ARN: arn:aws:kms:us-east-2:111122223333:alias/ExampleAlias
635+
622636
*Note:* If `fs.s3a.server-side-encryption-algorithm=CSE-KMS` is set,
623637
`fs.s3a.server-side-encryption.key=<KMS_KEY_ID>` property must be set for
624638
S3-CSE to work.

hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

Lines changed: 123 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
13091309
```
13101310
We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption)
13111311
is required for range gets to work.
1312+
1313+
### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK
1314+
1315+
If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
1316+
able to generate unique data key for encryption.
1317+
1318+
```
1319+
Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
1320+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
1321+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
1322+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
1323+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
1324+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
1325+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
1326+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
1327+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
1328+
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
1329+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
1330+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
1331+
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
1332+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
1333+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
1334+
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
1335+
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
1336+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
1337+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
1338+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
1339+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
1340+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
1341+
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
1342+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
1343+
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
1344+
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
1345+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
1346+
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
1347+
... 49 more
1348+
```
1349+
1350+
Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to
1351+
work.
1352+
1353+
### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId
1354+
1355+
If the value in `fs.s3a.server-side-encryption.key` property, does not exist
1356+
/valid in AWS KMS CMK(Customer managed keys), then this error would be seen.
1357+
1358+
```
1359+
Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
1360+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
1361+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
1362+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
1363+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
1364+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
1365+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
1366+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
1367+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
1368+
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
1369+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
1370+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
1371+
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
1372+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
1373+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
1374+
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
1375+
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
1376+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
1377+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
1378+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
1379+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
1380+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
1381+
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
1382+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
1383+
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
1384+
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
1385+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
1386+
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
1387+
... 49 more
1388+
```
1389+
1390+
Check if `fs.s3a.server-side-encryption.key` is set correctly and matches the
1391+
same on AWS console.
1392+
1393+
### com.amazonaws.services.kms.model.AWSKMSException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>
1394+
1395+
User doesn't have authorisation to the specific AWS KMS Key ID.
1396+
```
1397+
Caused by: com.amazonaws.services.kms.model.AWSKMSException: User: arn:aws
1398+
:iam::152813717728:user/<user> is not authorized to perform: kms
1399+
:GenerateDataKey on resource: <key_ID> (Service: AWSKMS; Status Code: 400
1400+
; Error Code: AccessDeniedException; Request ID: 4ded9f1f-b245-4213-87fc-16cba7a1c4b9; Proxy: null)
1401+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
1402+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
1403+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
1404+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
1405+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
1406+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
1407+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
1408+
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
1409+
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
1410+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
1411+
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
1412+
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
1413+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
1414+
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
1415+
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
1416+
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
1417+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
1418+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
1419+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
1420+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
1421+
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
1422+
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
1423+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
1424+
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
1425+
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
1426+
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
1427+
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
1428+
... 49 more
1429+
```
1430+
1431+
The user trying to use the KMS Key ID should have the right permissions to access
1432+
(encrypt/decrypt) using the AWS KMS Key used via `fs.s3a.server-side-encryption.key`.
1433+
If not, then add permission(or IAM role) in "Key users" section by selecting the
1434+
AWS-KMS CMK Key on AWS console.
13121435

13131436
### <a name="not_all_bytes_were_read"></a> Message appears in logs "Not all bytes were read from the S3ObjectInputStream"
13141437

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -229,4 +229,17 @@ public interface S3ATestConstants {
229229
* every test case.
230230
*/
231231
String DIRECTORY_MARKER_AUDIT = "fs.s3a.directory.marker.audit";
232+
233+
/**
234+
* Constant bytes being written when Client side encryption KMS is enabled
235+
* for a test. This bytes written takes into account "EncryptionContext",
236+
* which contains the algo used for eg:
237+
* "aws:x-amz-cek-alg":"AES/GCM/NoPadding" , and "KeySpec", which specifies
238+
* the length of data key. for eg: AES_256 to generate a 256-bit symmetric
239+
* key.
240+
*
241+
* For test using bytesWritten as an assertion this constant value can be
242+
* used.
243+
*/
244+
int KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN = 94;
232245
}

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,8 @@
3333
import org.junit.Before;
3434

3535
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
36+
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY;
37+
import static org.apache.hadoop.fs.s3a.S3ATestConstants.KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN;
3638
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
3739

3840
/**
@@ -70,12 +72,13 @@ protected void verifyReadBytes(FileSystem.Statistics stats) {
7072

7173
/**
7274
* A method to verify the bytes written.
73-
*
75+
* <br>
7476
* NOTE: if Client side encryption is enabled, expected bytes written
75-
* should increase by 16(padding of data) + 130(KMS key generation) in case
76-
* of storage type{@link CryptoStorageMode} as ObjectMetadata(Default). If
77-
* Crypto Storage mode is instruction file then add additional bytes as
78-
* that file is stored separately and would account for bytes written.
77+
* should increase by 16(padding of data) + bytes for the key ID set + 94(KMS
78+
* key generation) in case of storage type{@link CryptoStorageMode} as
79+
* ObjectMetadata(Default). If Crypto Storage mode is instruction file then
80+
* add additional bytes as that file is stored separately and would account
81+
* for bytes written.
7982
*
8083
* @param stats Filesystem statistics.
8184
*/
@@ -85,8 +88,10 @@ protected void verifyWrittenBytes(FileSystem.Statistics stats) {
8588
long expectedBlockSize = blockSize;
8689
if (conf.get(S3_ENCRYPTION_ALGORITHM, "")
8790
.equals(S3AEncryptionMethods.CSE_KMS.getMethod())) {
91+
String keyId = conf.get(S3_ENCRYPTION_KEY, "");
8892
// Adding padding length and KMS key generation bytes written.
89-
expectedBlockSize += CSE_PADDING_LENGTH + 130;
93+
expectedBlockSize += CSE_PADDING_LENGTH + keyId.getBytes().length +
94+
KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN;
9095
}
9196
Assert.assertEquals("Mismatch in bytes written", expectedBlockSize,
9297
stats.getBytesWritten());

0 commit comments

Comments
 (0)