Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-19336: S3A: Test failures after CSE support added #7164

Merged
merged 2 commits into from
Nov 22, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -219,6 +219,8 @@ private Keyring createKmsKeyring(S3ClientCreationParameters parameters,
return KmsKeyring.builder()
.kmsClient(kmsClientBuilder.build())
.wrappingKeyId(cseMaterials.getKmsKeyId())
// this is required for backward compatibility with older encryption clients
.enableLegacyWrappingAlgorithms(true)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -808,5 +808,217 @@

## <a name="troubleshooting"></a> Troubleshooting Encryption

The [troubleshooting](./troubleshooting_s3a.html) document covers
stack traces which may surface when working with encrypted data.
The section covers stack traces which may surface when working with encrypted data.

### <a name="encryption"></a> S3 Server Side Encryption

#### `AWSS3IOException` `KMS.NotFoundException` "Invalid arn" when using SSE-KMS

When performing file operations, the user may run into an issue where the KMS
key arn is invalid.

```
org.apache.hadoop.fs.s3a.AWSS3IOException: innerMkdirs on /test:
S3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException;
Request ID: CA89F276B3394565),
S3 Extended Request ID: ncz0LWn8zor1cUO2fQ7gc5eyqOk3YfyQLDn2OQNoe5Zj/GqDLggUYz9QY7JhdZHdBaDTh+TL5ZQ=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
```

Possible causes:

* the KMS key ARN is entered incorrectly, or
* the KMS key referenced by the ARN is in a different region than the S3 bucket
being used.

#### Using SSE-C "Bad Request"

When performing file operations the user may run into an unexpected 400/403
error such as
```
org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/:
S3Exception:
Bad Request (Service: Amazon S3; Status Code: 400;
Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99)
```

This can happen in the cases of not specifying the correct SSE-C encryption key.
Such cases can be as follows:
1. An object is encrypted using SSE-C on S3 and either the wrong encryption type
is used, no encryption is specified, or the SSE-C specified is incorrect.
2. A directory is encrypted with a SSE-C keyA and the user is trying to move a
file using configured SSE-C keyB into that structure.

### <a name="client-side-encryption"></a> S3 Client Side Encryption

#### java.lang.NoClassDefFoundError: software/amazon/encryption/s3/S3EncryptionClient

With the move to the V2 AWS SDK, CSE is implemented via
[amazon-s3-encryption-client-java](https://github.com/aws/amazon-s3-encryption-client-java/tree/v3.1.1)
which is not packaged in AWS SDK V2 bundle jar and needs to be added separately.

Fix: add amazon-s3-encryption-client-java jar version 3.1.1 to the class path.

#### Instruction file not found for S3 object

Reading an unencrypted file would fail when read through CSE enabled client by default.
```
software.amazon.encryption.s3.S3EncryptionClientException: Instruction file not found!
Please ensure the object you are attempting to decrypt has been encrypted
using the S3 Encryption Client.
```
CSE enabled client should read encrypted data only.

Fix: set `fs.s3a.encryption.cse.v1.compatibility.enabled=true`
#### CSE-KMS method requires KMS key ID

KMS key ID is required for CSE-KMS to encrypt data, not providing one leads
to failure.

```
2021-07-07 11:33:04,550 WARN fs.FileSystem: Failed to initialize filesystem
s3a://ap-south-cse/: java.lang.IllegalArgumentException: CSE-KMS
method requires KMS key ID. Use fs.s3a.encryption.key property to set it.
-ls: CSE-KMS method requires KMS key ID. Use fs.s3a.encryption.key property to
set it.
```

set `fs.s3a.encryption.key=<KMS_KEY_ID>` generated through AWS console.

#### `software.amazon.awssdk.services.kms.model.IncorrectKeyException` The key ID in the request does not identify a CMK that can perform this operation.

KMS key ID used to PUT(encrypt) the data, must be the one used to GET the
data.
```
cat: open s3a://ap-south-cse/encryptedData.txt at 0 on
s3a://ap-south-cse/encryptedData.txt:
software.amazon.awssdk.services.kms.model.IncorrectKeyException: The key ID in the
request does not identify a CMK that can perform this operation. (Service: AWSKMS;
Status Code: 400; ErrorCode: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null):IncorrectKeyException:
The key ID in the request does not identify a CMK that can perform this
operation. (Service: AWSKMS ; Status Code: 400; Error Code: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null)
```
Use the same KMS key ID used to upload data to download and read it as well.

#### `software.amazon.awssdk.services.kms.model.NotFoundException` key/<KMS_KEY_ID> does not exist

Using a KMS key ID from a different region than the bucket used to store data
would lead to failure while uploading.

```
mkdir: PUT 0-byte object on testmkdir:
software.amazon.awssdk.services.kms.model.NotFoundException: Key
'arn:aws:kms:ap-south-1:152813717728:key/<KMS_KEY_ID>'
does not exist (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null):NotFoundException:
Key 'arn:aws:kms:ap-south-1:152813717728:key/<KMS_KEY_ID>'
does not exist(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null)
```
If S3 bucket region is different from the KMS key region,
set`fs.s3a.encryption.cse.kms.region=<KMS_REGION>`

#### `software.amazon.encryption.s3.S3EncryptionClientException: Service returned HTTP status code 400` (Service: Kms, Status Code: 400)

An exception may be raised if the Key Management Service (KMS) region is either not specified or does not align with the expected configuration.

Fix: set`fs.s3a.encryption.cse.kms.region=<KMS_REGION>`

#### `software.amazon.encryption.s3.S3EncryptionClientException: Unable to execute HTTP request: Encountered fatal error in publisher: Unable to execute HTTP request: Encountered fatal error in publisher`

When a part upload fails (5xx status error) during a multi-part upload (MPU) with client-side encryption (CSE)
enabled, the partial upload may be retired. Since retrying the multi-part upload is not supported in this encryption scenario,
the entire job must be restarted.

#### `java.lang.ClassNotFoundException: software.amazon.encryption.*`

S3 encryption jars are not bundled into hadoop-aws jar by default. It needs to be added
separately to the class path. Currently, [amazon-s3-encryption-client-java v3.1.1](https://github.com/aws/amazon-s3-encryption-client-java/tree/v3.1.1) is used.

```
software.amazon.encryption.s3.S3EncryptionClientException:
Service returned HTTP status code 400 (Service: Kms, Status Code: 400,
Request ID: XG6CGC5ZH1JQS34S, Extended Request ID: KIyVA/pmbUUGmiqcy/ueyx0iw5ifgpuJMcrs0b4lYYZsXxikuUM2nRCl2lFnya+1TqGCt6YxLnM=):null:
Service returned HTTP status code 400 (Service: Kms, Status Code: 400, Request ID: XG6CGC5ZH1JQS34S, Extended Request ID: KIyVA/pmbUUGmiqcy/ueyx0iw5ifgpuJMcrs0b4lYYZsXxikuUM2nRCl2lFnya+1TqGCt6YxLnM=)
```

Fix: set`fs.s3a.encryption.cse.kms.region=<KMS_REGION>`


#### `software.amazon.awssdk.services.kms.mode.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: software.amazon.awssdk.services.kms.mode.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)
```

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

#### software.amazon.awssdk.services.kms.mode.NotFoundException: Invalid keyId

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

```
Caused by: software.amazon.awssdk.services.kms.model.NotFoundException: Invalid keyId abc
(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID:
9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
```

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

#### software.amazon.awssdk.services.kms.model.KmsException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>

User doesn't have authorization to the specific AWS KMS Key ID.
```
Caused by: software.amazon.awssdk.services.kms.model.KmsException:
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)
```

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.encryption.key`.
If not, then add permission(or IAM role) in "Key users" section by selecting the
AWS-KMS CMK Key on AWS console.

#### `S3EncryptionClientException` "Encountered fatal error in publisher"


```
software.amazon.encryption.s3.S3EncryptionClientException:
Unable to execute HTTP request: Encountered fatal error in publisher:
Unable to execute HTTP request: Encountered fatal error in publisher

...

Check failure on line 1006 in hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

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

blanks: end of line
Caused by: java.lang.IllegalStateException: Must use either different key or iv for GCM encryption
at com.sun.crypto.provider.CipherCore.checkReinit(CipherCore.java:1088)
at com.sun.crypto.provider.CipherCore.update(CipherCore.java:662)
at com.sun.crypto.provider.AESCipher.engineUpdate(AESCipher.java:380)
at javax.crypto.Cipher.update(Cipher.java:1835)
at software.amazon.encryption.s3.internal.CipherSubscriber.onNext(CipherSubscriber.java:52)
at software.amazon.encryption.s3.internal.CipherSubscriber.onNext(CipherSubscriber.java:16)
at software.amazon.awssdk.utils.async.SimplePublisher.doProcessQueue(SimplePublisher.java:267)
at software.amazon.awssdk.utils.async.SimplePublisher.processEventQueue(SimplePublisher.java:224)
```
An upload of a single block of a large file/stream failed due to a transient failure of an S3 front end server.

For unencrypted uploads, this block is simply posted again; recovery is transparent.
However, the cipher used used in CSE-KMS is unable to recover.

There is no fix for this other than the application itself completely regenerating the entire file/upload

Please note that this is a very rare problem for applications running within AWS infrastructure.
Original file line number Diff line number Diff line change
Expand Up @@ -1007,164 +1007,6 @@ of the destination of a rename is a directory -only that it is _not_ a file.
You can rename a directory or file deep under a file if you try -after which
there is no guarantee of the files being found in listings. Try not to do that.

## <a name="encryption"></a> S3 Server Side Encryption

### `AWSS3IOException` `KMS.NotFoundException` "Invalid arn" when using SSE-KMS

When performing file operations, the user may run into an issue where the KMS
key arn is invalid.

```
org.apache.hadoop.fs.s3a.AWSS3IOException: innerMkdirs on /test:
S3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException;
Request ID: CA89F276B3394565),
S3 Extended Request ID: ncz0LWn8zor1cUO2fQ7gc5eyqOk3YfyQLDn2OQNoe5Zj/GqDLggUYz9QY7JhdZHdBaDTh+TL5ZQ=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
```

Possible causes:

* the KMS key ARN is entered incorrectly, or
* the KMS key referenced by the ARN is in a different region than the S3 bucket
being used.

### Using SSE-C "Bad Request"

When performing file operations the user may run into an unexpected 400/403
error such as
```
org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/:
S3Exception:
Bad Request (Service: Amazon S3; Status Code: 400;
Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99)
```

This can happen in the cases of not specifying the correct SSE-C encryption key.
Such cases can be as follows:
1. An object is encrypted using SSE-C on S3 and either the wrong encryption type
is used, no encryption is specified, or the SSE-C specified is incorrect.
2. A directory is encrypted with a SSE-C keyA and the user is trying to move a
file using configured SSE-C keyB into that structure.

## <a name="client-side-encryption"></a> S3 Client Side Encryption

### java.lang.NoClassDefFoundError: software/amazon/encryption/s3/S3EncryptionClient

With the move to the V2 AWS SDK, CSE is implemented via
[amazon-s3-encryption-client-java](https://github.com/aws/amazon-s3-encryption-client-java/tree/v3.1.1)
which is not packaged in AWS SDK V2 bundle jar and needs to be added separately.

Fix: add amazon-s3-encryption-client-java jar version 3.1.1 to the class path.

### Instruction file not found for S3 object

Reading an unencrypted file would fail when read through CSE enabled client by default.
```
software.amazon.encryption.s3.S3EncryptionClientException: Instruction file not found!
Please ensure the object you are attempting to decrypt has been encrypted
using the S3 Encryption Client.
```
CSE enabled client should read encrypted data only.

Fix: set `fs.s3a.encryption.cse.v1.compatibility.enabled=true`
### CSE-KMS method requires KMS key ID

KMS key ID is required for CSE-KMS to encrypt data, not providing one leads
to failure.

```
2021-07-07 11:33:04,550 WARN fs.FileSystem: Failed to initialize filesystem
s3a://ap-south-cse/: java.lang.IllegalArgumentException: CSE-KMS
method requires KMS key ID. Use fs.s3a.encryption.key property to set it.
-ls: CSE-KMS method requires KMS key ID. Use fs.s3a.encryption.key property to
set it.
```

set `fs.s3a.encryption.key=<KMS_KEY_ID>` generated through AWS console.

### `software.amazon.awssdk.services.kms.model.IncorrectKeyException` The key ID in the request does not identify a CMK that can perform this operation.

KMS key ID used to PUT(encrypt) the data, must be the one used to GET the
data.
```
cat: open s3a://ap-south-cse/encryptedData.txt at 0 on
s3a://ap-south-cse/encryptedData.txt:
software.amazon.awssdk.services.kms.model.IncorrectKeyException: The key ID in the
request does not identify a CMK that can perform this operation. (Service: AWSKMS;
Status Code: 400; ErrorCode: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null):IncorrectKeyException:
The key ID in the request does not identify a CMK that can perform this
operation. (Service: AWSKMS ; Status Code: 400; Error Code: IncorrectKeyException;
Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null)
```
Use the same KMS key ID used to upload data to download and read it as well.

### `software.amazon.awssdk.services.kms.model.NotFoundException` key/<KMS_KEY_ID> does not exist

Using a KMS key ID from a different region than the bucket used to store data
would lead to failure while uploading.

```
mkdir: PUT 0-byte object on testmkdir:
software.amazon.awssdk.services.kms.model.NotFoundException: Key
'arn:aws:kms:ap-south-1:152813717728:key/<KMS_KEY_ID>'
does not exist (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null):NotFoundException:
Key 'arn:aws:kms:ap-south-1:152813717728:key/<KMS_KEY_ID>'
does not exist(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException;
Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null)
```
If S3 bucket region is different from the KMS key region,
set`fs.s3a.encryption.cse.kms.region=<KMS_REGION>`

### `software.amazon.awssdk.services.kms.mode.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: software.amazon.awssdk.services.kms.mode.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)
```

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

### software.amazon.awssdk.services.kms.mode.NotFoundException: Invalid keyId

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

```
Caused by: software.amazon.awssdk.services.kms.model.NotFoundException: Invalid keyId abc
(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID:
9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
```

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

### software.amazon.awssdk.services.kms.model.KmsException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>

User doesn't have authorization to the specific AWS KMS Key ID.
```
Caused by: software.amazon.awssdk.services.kms.model.KmsException:
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)
```

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.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
Loading
Loading