Skip to content

Commit

Permalink
HADOOP-13887. review comments, docs + test fix
Browse files Browse the repository at this point in the history
  • Loading branch information
Mehakmeet Singh committed Jul 20, 2021
1 parent 7f81092 commit e3d5922
Show file tree
Hide file tree
Showing 6 changed files with 180 additions and 26 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -397,10 +397,9 @@ public void close() throws IOException {
// PUT the final block
if (hasBlock &&
(block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
// send last part and set the value of isLastPart to true in case of
// CSE being enabled, since we are sure it is last part as parts
// are being uploaded serially in CSE.
uploadCurrentBlock(isCSEEnabled);
// send last part and set the value of isLastPart to true.
// Necessary to set this "true" in case of client side encryption.
uploadCurrentBlock(true);
}
// wait for the partial uploads to finish
final List<PartETag> partETags =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
AuditIntegration.stubAuditManager();

/**
* Is this S3AFS instance using S3 client side encryption?
* Is this S3A FS instance using S3 client side encryption?
*/
private boolean isCSEEnabled;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,19 +87,19 @@ Encryption options

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

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

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

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

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

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

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

This makes viewing the encryption headers of a file straightforward
This makes viewing the encryption headers of a file straightforward.

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

Analysis

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

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

And a directory encrypted with S3-SSE only
And a directory encrypted with S3-SSE only:

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

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

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

KMS_KEY_ID:

Identifies the symmetric CMK that encrypts the data key.
To specify a CMK, use its key ID, key ARN, alias name, or alias ARN. When
using an alias name, prefix it with "alias/". To specify a CMK in a
different AWSaccount, you must use the key ARN or alias ARN.

For example:
- Key ID: 1234abcd-12ab-34cd-56ef-1234567890ab
- Key ARN: arn:aws:kms:us-east-2:111122223333:key/1234abcd-12ab-34cd-56ef-1234567890ab
- Alias name: alias/ExampleAlias
- Alias ARN: arn:aws:kms:us-east-2:111122223333:alias/ExampleAlias

*Note:* If `fs.s3a.server-side-encryption-algorithm=CSE-KMS` is set,
`fs.s3a.server-side-encryption.key=<KMS_KEY_ID>` property must be set for
S3-CSE to work.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
```
We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption)
is required for range gets to work.

### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK

If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
able to generate unique data key for encryption.

```
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)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
```

Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to
work.

### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId

If the value in `fs.s3a.server-side-encryption.key` property, does not exist
/valid in AWS KMS CMK(Customer managed keys), then this error would be seen.

```
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)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
```

Check if `fs.s3a.server-side-encryption.key` is set correctly and matches the
same on AWS console.

### com.amazonaws.services.kms.model.AWSKMSException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>

User doesn't have authorisation to the specific AWS KMS Key ID.
```
Caused by: com.amazonaws.services.kms.model.AWSKMSException: User: arn:aws
:iam::152813717728:user/<user> is not authorized to perform: kms
:GenerateDataKey on resource: <key_ID> (Service: AWSKMS; Status Code: 400
; Error Code: AccessDeniedException; Request ID: 4ded9f1f-b245-4213-87fc-16cba7a1c4b9; Proxy: null)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
... 49 more
```

The user trying to use the KMS Key ID should have the right permissions to access
(encrypt/decrypt) using the AWS KMS Key used via `fs.s3a.server-side-encryption.key`.
If not, then add permission(or IAM role) in "Key users" section by selecting the
AWS-KMS CMK Key on AWS console.

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,4 +229,17 @@ public interface S3ATestConstants {
* every test case.
*/
String DIRECTORY_MARKER_AUDIT = "fs.s3a.directory.marker.audit";

/**
* Constant bytes being written when Client side encryption KMS is enabled
* for a test. This bytes written takes into account "EncryptionContext",
* which contains the algo used for eg:
* "aws:x-amz-cek-alg":"AES/GCM/NoPadding" , and "KeySpec", which specifies
* the length of data key. for eg: AES_256 to generate a 256-bit symmetric
* key.
*
* For test using bytesWritten as an assertion this constant value can be
* used.
*/
int KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN = 94;
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@
import org.junit.Before;

import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;

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

/**
* A method to verify the bytes written.
*
* <br>
* NOTE: if Client side encryption is enabled, expected bytes written
* should increase by 16(padding of data) + 130(KMS key generation) in case
* of storage type{@link CryptoStorageMode} as ObjectMetadata(Default). If
* Crypto Storage mode is instruction file then add additional bytes as
* that file is stored separately and would account for bytes written.
* should increase by 16(padding of data) + bytes for the key ID set + 94(KMS
* key generation) in case of storage type{@link CryptoStorageMode} as
* ObjectMetadata(Default). If Crypto Storage mode is instruction file then
* add additional bytes as that file is stored separately and would account
* for bytes written.
*
* @param stats Filesystem statistics.
*/
Expand All @@ -85,8 +88,10 @@ protected void verifyWrittenBytes(FileSystem.Statistics stats) {
long expectedBlockSize = blockSize;
if (conf.get(S3_ENCRYPTION_ALGORITHM, "")
.equals(S3AEncryptionMethods.CSE_KMS.getMethod())) {
String keyId = conf.get(S3_ENCRYPTION_KEY, "");
// Adding padding length and KMS key generation bytes written.
expectedBlockSize += CSE_PADDING_LENGTH + 130;
expectedBlockSize += CSE_PADDING_LENGTH + keyId.getBytes().length +
KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN;
}
Assert.assertEquals("Mismatch in bytes written", expectedBlockSize,
stats.getBytesWritten());
Expand Down

0 comments on commit e3d5922

Please sign in to comment.