diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
index a343afc042ec64..10a03fcbe04575 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
@@ -36,7 +36,7 @@
public final class ByteBufferInputStream extends InputStream {
private static final Logger LOG =
LoggerFactory.getLogger(DataBlocks.class);
-
+
/** Size of the buffer. */
private final int size;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
index f7c72f85309595..39131d2f231f8a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
@@ -22,12 +22,15 @@
/**
* A 5xx response came back from a service.
- * The 500 error considered retriable by the AWS SDK, which will have already
+ *
+ * The 500 error is considered retryable by the AWS SDK, which will have already
* tried it {@code fs.s3a.attempts.maximum} times before reaching s3a
* code.
- * How it handles other 5xx errors is unknown: S3A FS code will treat them
- * as unrecoverable on the basis that they indicate some third-party store
- * or gateway problem.
+ *
+ * These are rare, but can occur; they are considered retryable.
+ * Note that HADOOP-19221 shows a failure condition where the
+ * SDK itself did not recover on retry from the error.
+ * Mitigation for the specific failure sequence is now in place.
*/
public class AWSStatus500Exception extends AWSServiceIOException {
public AWSStatus500Exception(String operation,
@@ -35,8 +38,4 @@ public AWSStatus500Exception(String operation,
super(operation, cause);
}
- @Override
- public boolean retryable() {
- return false;
- }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
index faf105c8e2c862..1c0f86dbf6bc6b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
@@ -228,15 +228,15 @@ protected Map, RetryPolicy> createExceptionMap() {
// throttled requests are can be retried, always
policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
- // Status 5xx error code is an immediate failure
+ // Status 5xx error code has historically been treated as an immediate failure
// this is sign of a server-side problem, and while
// rare in AWS S3, it does happen on third party stores.
// (out of disk space, etc).
// by the time we get here, the aws sdk will have
- // already retried.
+ // already retried, if it is configured to retry exceptions.
// there is specific handling for some 5XX codes (501, 503);
// this is for everything else
- policyMap.put(AWSStatus500Exception.class, fail);
+ policyMap.put(AWSStatus500Exception.class, retryAwsClientExceptions);
// subclass of AWSServiceIOException whose cause is always S3Exception
policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
index ed24fc568d79aa..f983e689fc5853 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
@@ -575,13 +575,18 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
numParts, length));
}
- List parts = new ArrayList<>((int) numParts);
-
LOG.debug("File size is {}, number of parts to upload = {}",
length, numParts);
// Open the file to upload.
- parts = uploadFileData(uploadId, localFile, destKey, progress, length, numParts, uploadPartSize);
+ List parts = uploadFileData(
+ uploadId,
+ localFile,
+ destKey,
+ progress,
+ length,
+ numParts,
+ uploadPartSize);
commitData.bindCommitData(parts);
statistics.commitUploaded(length);
@@ -608,7 +613,7 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
* Upload file data using content provider API.
* This a rewrite of the previous code to address HADOOP-19221;
* our own {@link UploadContentProviders} file content provider
- * is used to upload part of a file.
+ * is used to upload each part of the file.
* @param uploadId upload ID
* @param localFile locally staged file
* @param destKey destination path
@@ -616,7 +621,8 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
* @param length file length
* @param numParts number of parts to upload
* @param uploadPartSize max size of a part
- * @throws IOException
+ * @return the ordered list of parts
+ * @throws IOException IO failure
*/
private List uploadFileData(
final String uploadId,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
index 50a6a2f366904e..15c2ab9b2548fe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
@@ -22,7 +22,6 @@
import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges;
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
-import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
index e53e4a002265a7..80d22ca37c2412 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
@@ -29,7 +29,7 @@ Common problems working with S3 are:
7. [Other Errors](#other)
8. [SDK Upgrade Warnings](#upgrade_warnings)
-This document also includes some [best pactises](#best) to aid troubleshooting.
+This document also includes some [best practises](#best) to aid troubleshooting.
Troubleshooting IAM Assumed Roles is covered in its
@@ -236,8 +236,60 @@ read requests are allowed, but operations which write to the bucket are denied.
Check the system clock.
-### "Bad Request" exception when working with data stores in an AWS region other than us-eaast
+### `Class does not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
+
+A credential provider listed in `fs.s3a.aws.credentials.provider` does not implement
+the interface `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
+
+```
+InstantiationIOException: `s3a://stevel-gcs/': Class org.apache.hadoop.fs.s3a.S3ARetryPolicy does not implement
+ software.amazon.awssdk.auth.credentials.AwsCredentialsProvider (configuration key fs.s3a.aws.credentials.provider)
+ at org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf(InstantiationIOException.java:128)
+ at org.apache.hadoop.fs.s3a.S3AUtils.getInstanceFromReflection(S3AUtils.java:604)
+ at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSV2CredentialProvider(CredentialProviderListFactory.java:299)
+ at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList(CredentialProviderListFactory.java:245)
+ at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList(CredentialProviderListFactory.java:144)
+ at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:971)
+ at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:624)
+ at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3601)
+ at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171)
+ at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3702)
+ at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3653)
+ at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:555)
+ at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366)
+
+```
+
+There's two main causes
+
+1. A class listed there is not an implementation of the interface.
+ Fix: review the settings and correct as appropriate.
+1. A class listed there does implement the interface, but it has been loaded in a different
+ classloader, so the JVM does not consider it to be an implementation.
+ Fix: learn the entire JVM classloader model and see if you can then debug it.
+ Tip: having both the AWS Shaded SDK and individual AWS SDK modules on your classpath
+ may be a cause of this.
+
+If you see this and you are trying to use the S3A connector with Spark, then the cause can
+be that the isolated classloader used to load Hive classes is interfering with the S3A
+connector's dynamic loading of `software.amazon.awssdk` classes. To fix this, declare that
+the classes in the aws SDK are loaded from the same classloader which instantiated
+the S3A FileSystem instance:
+
+```
+spark.sql.hive.metastore.sharedPrefixes software.amazon.awssdk.
+```
+
+
+## 400 Bad Request errors
+
+S3 stores return HTTP status code 400 "Bad Request" when the client make a request which
+the store considers invalid.
+
+This is most commonly caused by signing errors
+
+### "Bad Request" exception when working with data stores in an AWS region other than us-east
```
@@ -286,47 +338,37 @@ S3 region as `ca-central-1`.
```
-### `Classdoes not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
-
-A credential provider listed in `fs.s3a.aws.credentials.provider` does not implement
-the interface `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
+### 400 + RequestTimeout "Your socket connection to the server was not read from or written to within the timeout period"
```
-InstantiationIOException: `s3a://stevel-gcs/': Class org.apache.hadoop.fs.s3a.S3ARetryPolicy does not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider (configuration key fs.s3a.aws.credentials.provider)
- at org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf(InstantiationIOException.java:128)
- at org.apache.hadoop.fs.s3a.S3AUtils.getInstanceFromReflection(S3AUtils.java:604)
- at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSV2CredentialProvider(CredentialProviderListFactory.java:299)
- at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList(CredentialProviderListFactory.java:245)
- at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList(CredentialProviderListFactory.java:144)
- at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:971)
- at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:624)
- at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3601)
- at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171)
- at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3702)
- at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3653)
- at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:555)
- at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366)
-
+org.apache.hadoop.fs.s3a.AWSBadRequestException: upload part #1 upload ID KylDNXbmiZlZE5JI2aKVcVeA66ly:
+ software.amazon.awssdk.services.s3.model.S3Exception:
+ Your socket connection to the server was not read from or written to within the timeout period.
+ Idle connections will be closed.
+ (Service: S3, Status Code: 400, Request ID: TT17CRYF6HJH2G0Y, Extended Request ID: ...):
+ RequestTimeout:
+ Your socket connection to the server was not read from or written to within the timeout period.
+ Idle connections will be closed. (Service: S3, Status Code: 400, Request ID: TT17CRYF6HJH2G0Y, Extended Request ID: ...
```
-There's two main causes
+This is an obscure failure which was encountered as part of
+[HADOOP-19221](https://issues.apache.org/jira/browse/HADOOP-19221) : an upload of part of a file could not
+be succesfully retried after a failure was reported on the first attempt.
-1. A class listed there is not an implementation of the interface.
- Fix: review the settings and correct as appropriate.
-1. A class listed there does implement the interface, but it has been loaded in a different
- classloader, so the JVM does not consider it to be an implementation.
- Fix: learn the entire JVM classloader model and see if you can then debug it.
- Tip: having both the AWS Shaded SDK and individual AWS SDK modules on your classpath
- may be a cause of this.
+1. It was only encountered during uploading files via the Staging Committers
+2. And is a regression in the V2 SDK.
+3. This should have been addressed in the S3A connector.
-If you see this and you are trying to use the S3A connector with Spark, then the cause can
-be that the isolated classloader used to load Hive classes is interfering with the S3A
-connector's dynamic loading of `software.amazon.awssdk` classes. To fix this, declare that
-the classes in the aws SDK are loaded from the same classloader which instantiated
-the S3A FileSystem instance:
+* If it is encountered on a hadoop release with HADOOP-19221, then this is a regression -please report it.
+* If it is encountered on a release without the fix, please upgrade.
+It may be that the problem arises in the AWS SDK's "TransferManager", which is used for a
+higher performance upload of data from the local fileystem. If this is the case. disable this feature:
```
-spark.sql.hive.metastore.sharedPrefixes software.amazon.awssdk.
+
+ fs.s3a.optimized.copy.from.local.enabled
+ false
+
```
## "The security token included in the request is invalid"
@@ -501,7 +543,43 @@ endpoint and region like the following:
${sts.region}
```
+## HTTP 500 status code "We encountered an internal error"
+
+```
+We encountered an internal error. Please try again.
+(Service: S3, Status Code: 500, Request ID: , Extended Request ID: )
+```
+
+The [status code 500](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/500) indicates
+the S3 store has reported an internal problem.
+When raised by Amazon S3, we believe this is a rare sign of a problem within the S3 system
+or another part of the cloud infrastructure on which it depends.
+Retrying _should_ make it go away.
+
+The 500 error is considered retryable by the AWS SDK, which will have already
+tried it `fs.s3a.attempts.maximum` times before reaching the S3A client -which
+will also retry.
+
+If encountered against a third party store (the lack of an extended request ID always implies this),
+then it may be a permanent server-side failure. Fix that.
+
+* All HTTP status codes other than 503 (service unavailable) and 501 (unsupported) are
+treated as 500 exceptions.
+* The S3A Filesystem IOStatistics count the number of 500 errors received.
+
+## HTTP 503 status code "slow down" or 429 "Too Many Requests"
+
+The [status code 503](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/503)
+is returned by AWS S3 when the IO rate limit of the bucket is reached.
+
+Google's cloud storage returns the response [429 Too Many Requests](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/429)
+for the same situation.
+
+The AWS S3 documentation [covers this and suggests mitigation strategies](https://repost.aws/knowledge-center/http-5xx-errors-s3).
+Note that it can also be caused by throttling in the KMS bencryption subsystem if
+SSE-KMS or DSSE-KMS is used to encrypt data.
+Consult [performance - throttling](./performance.html#throttling) for details on throttling.
## Connectivity Problems
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
index 6f0c66a78ecd43..b0a0a59f12f654 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java
@@ -138,7 +138,7 @@ public void testBlockFactoryIO() throws Throwable {
int expected = bufferLen;
assertAvailableValue(stream, expected);
- assertReadEquals(stream,'t');
+ assertReadEquals(stream, 't');
stream.mark(Integer.MAX_VALUE);
expected--;
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestUploadRecovery.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestUploadRecovery.java
index 5ac71296782b6f..3f6041eb70fac7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestUploadRecovery.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestUploadRecovery.java
@@ -51,6 +51,7 @@
import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MULTIPART_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_OPERATIONS_PURGE_UPLOADS;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER;
@@ -78,9 +79,9 @@
*
* Fault injection is implemented in {@link FaultInjector}; this uses the evaluator
* function {@link #evaluator} to determine if the request type is that for which
- * failures are targeted; for when there is a match then {@link #requestFailureCount}
+ * failures are targeted; for when there is a match then {@link #REQUEST_FAILURE_COUNT}
* is decremented and, if the count is still positive, an error is raised with the
- * error code defined in {@link #failureStatusCode}.
+ * error code defined in {@link #FAILURE_STATUS_CODE}.
* This happens after the request has already succeeded against the S3 store.
*/
@RunWith(Parameterized.class)
@@ -117,14 +118,15 @@ public static Collection