diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 8dc6bba1cc..b741bc2301 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1087,4 +1087,10 @@ private Constants() { */ public static final String AWS_REGION = "fs.s3a.endpoint.region"; + /** + * The special S3 region which can be used to talk to any bucket. + * Value {@value}. + */ + public static final String AWS_S3_CENTRAL_REGION = "us-east-1"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 6e84497cb9..7dc920ce50 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -22,6 +22,7 @@ import java.net.URI; import com.amazonaws.ClientConfiguration; +import com.amazonaws.SdkClientException; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.handlers.RequestHandler2; import com.amazonaws.services.s3.AmazonS3; @@ -41,10 +42,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; +import org.apache.hadoop.fs.store.LogExactlyOnce; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT; +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; /** * The default {@link S3ClientFactory} implementation. @@ -64,6 +68,19 @@ public class DefaultS3ClientFactory extends Configured protected static final Logger LOG = LoggerFactory.getLogger(DefaultS3ClientFactory.class); + /** + * A one-off warning of default region chains in use. + */ + private static final LogExactlyOnce WARN_OF_DEFAULT_REGION_CHAIN = + new LogExactlyOnce(LOG); + + /** + * Warning message printed when the SDK Region chain is in use. + */ + private static final String SDK_REGION_CHAIN_IN_USE = + "S3A filesystem client is using" + + " the SDK region resolution chain."; + /** * Create the client by preparing the AwsConf configuration * and then invoking {@code buildAmazonS3Client()}. @@ -94,9 +111,14 @@ public AmazonS3 createS3Client( awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix()); } - return buildAmazonS3Client( - awsConf, - parameters); + try { + return buildAmazonS3Client( + awsConf, + parameters); + } catch (SdkClientException e) { + // SDK refused to build. + throw translateException("creating AWS S3 client", uri.toString(), e); + } } /** @@ -109,6 +131,7 @@ public AmazonS3 createS3Client( * @param awsConf AWS configuration * @param parameters parameters * @return new AmazonS3 client + * @throws SdkClientException if the configuration is invalid. */ protected AmazonS3 buildAmazonS3Client( final ClientConfiguration awsConf, @@ -141,6 +164,21 @@ protected AmazonS3 buildAmazonS3Client( // no idea what the endpoint is, so tell the SDK // to work it out at the cost of an extra HEAD request b.withForceGlobalBucketAccessEnabled(true); + // HADOOP-17771 force set the region so the build process doesn't halt. + String region = getConf().getTrimmed(AWS_REGION, AWS_S3_CENTRAL_REGION); + LOG.debug("fs.s3a.endpoint.region=\"{}\"", region); + if (!region.isEmpty()) { + // there's either an explicit region or we have fallen back + // to the central one. + LOG.debug("Using default endpoint; setting region to {}", region); + b.setRegion(region); + } else { + // no region. + // allow this if people really want it; it is OK to rely on this + // when deployed in EC2. + WARN_OF_DEFAULT_REGION_CHAIN.warn(SDK_REGION_CHAIN_IN_USE); + LOG.debug(SDK_REGION_CHAIN_IN_USE); + } } final AmazonS3 client = b.build(); return client; @@ -206,7 +244,7 @@ protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3, createEndpointConfiguration( final String endpoint, final ClientConfiguration awsConf, String awsRegion) { - LOG.debug("Creating endpoint configuration for {}", endpoint); + LOG.debug("Creating endpoint configuration for \"{}\"", endpoint); if (endpoint == null || endpoint.isEmpty()) { // the default endpoint...we should be using null at this point. LOG.debug("Using default endpoint -no need to generate a configuration"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index d6142f49c9..cf962b87a4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -121,4 +121,9 @@ private InternalConstants() { */ public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000; + /** + * The system property used by the AWS SDK to identify the region. + */ + public static final String AWS_REGION_SYSPROP = "aws.region"; + } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index aff7694c52..48b99caf65 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -438,6 +438,12 @@ you'll need to remove the `profile` prefix from the AWS configuration section he aws_session_token = ... aws_security_token = ... ``` +Note: + +1. The `region` setting is only used if `fs.s3a.endpoint.region` is set to the empty string. +1. For the credentials to be available to applications running in a Hadoop cluster, the + configuration files MUST be in the `~/.aws/` directory on the local filesystem in + all hosts in the cluster. ### Using Session Credentials with `TemporaryAWSCredentialsProvider` @@ -802,8 +808,10 @@ options are covered in [Testing](./testing.md). fs.s3a.endpoint.region AWS S3 region for a bucket, which bypasses the parsing of - fs.s3a.endpoint to know the region. Would be helpful in avoiding errors - while using privateLink URL and explicitly set the bucket region. + fs.s3a.endpoint to know the region. Would be helpful in avoiding errors + while using privateLink URL and explicitly set the bucket region. + If set to a blank string (or 1+ space), falls back to the + (potentially brittle) SDK region resolution process. 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 d91607d3fd..30047edfb5 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 @@ -18,11 +18,17 @@ ## Introduction -Common problems working with S3 are +Common problems working with S3 are: -1. Classpath setup -1. Authentication -1. Incorrect configuration +1. [Classpath setup](#classpath) +1. [Authentication](#authentication) +1. [Access Denial](#access_denied) +1. [Connectivity Problems](#connectivity) +1. [File System Semantics](#semantics) +1. [Encryption](#encryption) +1. [Other Errors](#other) + +This document also includes some [best pactises](#best) to aid troubleshooting. Troubleshooting IAM Assumed Roles is covered in its @@ -572,7 +578,7 @@ S3 sts endpoint and region like the following: ## Connectivity Problems -### Error message "The bucket you are attempting to access must be addressed using the specified endpoint" +### Error "The bucket you are attempting to access must be addressed using the specified endpoint" This surfaces when `fs.s3a.endpoint` is configured to use an S3 service endpoint which is neither the original AWS one, `s3.amazonaws.com` , nor the one where @@ -611,6 +617,101 @@ can be used: Using the explicit endpoint for the region is recommended for speed and to use the V4 signing API. +### `Unable to find a region via the region provider chain` + +S3A client creation fails, possibly after a pause of some seconds. + +This failure surfaces when _all_ the following conditions are met: + +1. Deployment outside EC2. +1. `fs.s3a.endpoint` is unset. +1. `fs.s3a.endpoint.region` is set to `""`. (Hadoop 3.3.2+ only) +1. Without the file `~/.aws/config` existing or without a region set in it. +1. Without the JVM system property `aws.region` declaring a region. +1. Without the environment variable `AWS_REGION` declaring a region. + +Stack trace (Hadoop 3.3.1): +``` +Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. + Must provide an explicit region in the builder or setup environment to supply a region. + at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462) + at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424) + at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46) + at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:145) + at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:97) + at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:788) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:478) +``` + +Log and stack trace on later releases, with +"S3A filesystem client is using the SDK region resolution chain." +warning that the SDK resolution chain is in use: + +``` +2021-06-23 19:56:55,971 [main] WARN s3a.DefaultS3ClientFactory (LogExactlyOnce.java:warn(39)) - + S3A filesystem client is using the SDK region resolution chain. + +2021-06-23 19:56:56,073 [main] WARN fs.FileSystem (FileSystem.java:createFileSystem(3464)) - + Failed to initialize fileystem s3a://osm-pds/planet: + org.apache.hadoop.fs.s3a.AWSClientIOException: creating AWS S3 client on s3a://osm-pds: + com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. + Must provide an explicit region in the builder or setup environment to supply a region.: + Unable to find a region via the region provider chain. + Must provide an explicit region in the builder or setup environment to supply a region. + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:208) + at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:122) + at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:788) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:478) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3460) + at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:172) + at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3565) + at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3518) + at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:592) +Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. + Must provide an explicit region in the builder or setup environment to supply a region. + at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462) + at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424) + at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46) + at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:185) + at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:117) + ... 21 more +``` + +Due to changes in S3 client construction in Hadoop 3.3.1 this option surfaces in +non-EC2 deployments where no AWS endpoint was declared: +[HADOOP-17771](https://issues.apache.org/jira/browse/HADOOP-17771). On Hadoop +3.3.2 and later it takes active effort to create this stack trace. + +**Fix: set `fs.s3a.endpoint` to `s3.amazonaws.com`** + +Set `fs.s3a.endpoint` to the endpoint where the data is stored +(best), or to `s3.amazonaws.com` (second-best). + +```xml + + fs.s3a.endpoint + s3.amazonaws.com + +``` + +For Apache Spark, this can be done in `spark-defaults.conf` + +``` +spark.hadoop.fs.s3a.endpoint s3.amazonaws.com +``` + +Or in Scala by editing the spark configuration during setup. + +```scala +sc.hadoopConfiguration.set("fs.s3a.endpoint", "s3.amazonaws.com") +``` + +Tip: set the logging of `org.apache.hadoop.fs.s3a.DefaultS3ClientFactory` +to `DEBUG` to see how the endpoint and region configuration is determined. + +``` +log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=DEBUG +``` ### "Timeout waiting for connection from pool" when writing data @@ -792,257 +893,10 @@ Again, we believe this is caused by the connection to S3 being broken. It may go away if the operation is retried. -## Other Errors - -### `SdkClientException` Unable to verify integrity of data upload - -Something has happened to the data as it was uploaded. - -``` -Caused by: org.apache.hadoop.fs.s3a.AWSClientIOException: saving output on dest/_task_tmp.-ext-10000/_tmp.000000_0: - com.amazonaws.AmazonClientException: Unable to verify integrity of data upload. - Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64) - didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3. - You may need to delete the data stored in Amazon S3. - (metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null, - bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0): - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:144) - at org.apache.hadoop.fs.s3a.S3AOutputStream.close(S3AOutputStream.java:121) - at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) - at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) - at org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat$1.close(HiveIgnoreKeyTextOutputFormat.java:99) - at org.apache.hadoop.hive.ql.exec.FileSinkOperator$FSPaths.closeWriters(FileSinkOperator.java:190) - ... 22 more -Caused by: com.amazonaws.AmazonClientException: Unable to verify integrity of data upload. - Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64) - didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3. - You may need to delete the data stored in Amazon S3. - (metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null, - bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0) - at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1492) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47) - ... 4 more -``` - -As it uploads data to S3, the AWS SDK builds up an MD5 checksum of what was -PUT/POSTed. When S3 returns the checksum of the uploaded data, that is compared -with the local checksum. If there is a mismatch, this error is reported. - -The uploaded data is already on S3 and will stay there, though if this happens -during a multipart upload, it may not be visible (but still billed: clean up your -multipart uploads via the `hadoop s3guard uploads` command). - -Possible causes for this - -1. A (possibly transient) network problem, including hardware faults. -1. A proxy server is doing bad things to the data. -1. Some signing problem, especially with third-party S3-compatible object stores. - -This is a very, very rare occurrence. - -If the problem is a signing one, try changing the signature algorithm. - -```xml - - fs.s3a.signing-algorithm - S3SignerType - -``` - -We cannot make any promises that it will work, -only that it has been known to make the problem go away "once" - -### `AWSS3IOException` The Content-MD5 you specified did not match what we received - -Reads work, but writes, even `mkdir`, fail: - -``` -org.apache.hadoop.fs.s3a.AWSS3IOException: copyFromLocalFile(file:/tmp/hello.txt, s3a://bucket/hello.txt) - on file:/tmp/hello.txt: - The Content-MD5 you specified did not match what we received. - (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), - S3 Extended Request ID: null - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:127) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:69) - at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:1494) - at org.apache.hadoop.tools.cloudup.Cloudup.uploadOneFile(Cloudup.java:466) - at org.apache.hadoop.tools.cloudup.Cloudup.access$000(Cloudup.java:63) - at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:353) - at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:350) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:748) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - The Content-MD5 you specified did not match what we received. - (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), - S3 Extended Request ID: null - at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1307) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:894) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:597) - at com.amazonaws.http.AmazonHttpClient.doExecute(AmazonHttpClient.java:363) - at com.amazonaws.http.AmazonHttpClient.executeWithTimer(AmazonHttpClient.java:329) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:308) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3659) - at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1422) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47) - at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) - ... 4 more -``` - -This stack trace was seen when interacting with a third-party S3 store whose -expectations of headers related to the AWS V4 signing mechanism was not -compatible with that of the specific AWS SDK Hadoop was using. - -Workaround: revert to V2 signing. - -```xml - - fs.s3a.signing-algorithm - S3SignerType - -``` - -### When writing data: "java.io.FileNotFoundException: Completing multi-part upload" - - -A multipart upload was trying to complete, but failed as there was no upload -with that ID. - -``` -java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: - com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. - The upload ID may be invalid, or the upload may have been aborted or completed. - (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; - at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272) - at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) - at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) -``` - -This can happen when all outstanding uploads have been aborted, including -the active ones. - -If the bucket has a lifecycle policy of deleting multipart uploads, make -sure that the expiry time of the deletion is greater than that required -for all open writes to complete the write, -*and for all jobs using the S3A committers to commit their work.* - - -### Application hangs after reading a number of files - - -The pool of https client connections and/or IO threads have been used up, -and none are being freed. - - -1. The pools aren't big enough. See ["Timeout waiting for connection from pool"](#timeout_from_pool) -2. Likely root cause: whatever code is reading files isn't calling `close()` -on the input streams. Make sure your code does this! -And if it's someone else's: make sure you have a recent version; search their -issue trackers to see if its a known/fixed problem. -If not, it's time to work with the developers, or come up with a workaround -(i.e closing the input stream yourself). - - - -### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient` - -``` -[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond -org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond - at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143) - at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57) - at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261) - at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283) - at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259) - at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209) - at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272) - at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66) - at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124) - at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686) - at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488) - at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731) - at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41) - at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28) - at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:745) -``` - -These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client -will attempt to retry the operation; it may just be a transient event. If there -are many such exceptions in logs, it may be a symptom of connectivity or network -problems. - -### `AWSBadRequestException` IllegalLocationConstraintException/The unspecified location constraint is incompatible - -``` - Cause: org.apache.hadoop.fs.s3a.AWSBadRequestException: put on : - com.amazonaws.services.s3.model.AmazonS3Exception: - The unspecified location constraint is incompatible for the region specific - endpoint this request was sent to. - (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; - - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:178) - at org.apache.hadoop.fs.s3a.S3ALambda.execute(S3ALambda.java:64) - at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadObject(WriteOperationHelper.java:451) - at org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker.aboutToComplete(MagicCommitTracker.java:128) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:373) - at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) - at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) - at org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2429) - at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:106) - at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:91) - ... - Cause: com.amazonaws.services.s3.model.AmazonS3Exception: - The unspecified location constraint is incompatible for the region specific endpoint - this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; - Request ID: EEBC5A08BCB3A645) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667) - at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221) - ... -``` - -Something has been trying to write data to "/". - -## File System Semantics +## File System Semantics These are the issues where S3 does not appear to behave the way a filesystem -"should". +"should". That's because it "isn't". ### File not visible/saved @@ -1185,7 +1039,7 @@ We also recommend using applications/application options which do not rename files when committing work or when copying data to S3, but instead write directly to the final destination. -## Rename not behaving as "expected" +### Rename not behaving as "expected" S3 is not a filesystem. The S3A connector mimics file and directory rename by @@ -1303,7 +1157,7 @@ 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. -## Message appears in logs "Not all bytes were read from the S3ObjectInputStream" +### Message appears in logs "Not all bytes were read from the S3ObjectInputStream" This is a message which can be generated by the Amazon SDK when the client application @@ -1378,8 +1232,250 @@ The specified bucket does not exist at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1367) ``` +## Other Errors -## Other Issues +### `SdkClientException` Unable to verify integrity of data upload + +Something has happened to the data as it was uploaded. + +``` +Caused by: org.apache.hadoop.fs.s3a.AWSClientIOException: saving output on dest/_task_tmp.-ext-10000/_tmp.000000_0: + com.amazonaws.AmazonClientException: Unable to verify integrity of data upload. + Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64) + didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3. + You may need to delete the data stored in Amazon S3. + (metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null, + bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0): + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:144) + at org.apache.hadoop.fs.s3a.S3AOutputStream.close(S3AOutputStream.java:121) + at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) + at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) + at org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat$1.close(HiveIgnoreKeyTextOutputFormat.java:99) + at org.apache.hadoop.hive.ql.exec.FileSinkOperator$FSPaths.closeWriters(FileSinkOperator.java:190) + ... 22 more +Caused by: com.amazonaws.AmazonClientException: Unable to verify integrity of data upload. + Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64) + didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3. + You may need to delete the data stored in Amazon S3. + (metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null, + bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0) + at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1492) + at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131) + at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123) + at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139) + at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47) + ... 4 more +``` + +As it uploads data to S3, the AWS SDK builds up an MD5 checksum of what was +PUT/POSTed. When S3 returns the checksum of the uploaded data, that is compared +with the local checksum. If there is a mismatch, this error is reported. + +The uploaded data is already on S3 and will stay there, though if this happens +during a multipart upload, it may not be visible (but still billed: clean up +your multipart uploads via the `hadoop s3guard uploads` command). + +Possible causes for this + +1. A (possibly transient) network problem, including hardware faults. +1. A proxy server is doing bad things to the data. +1. Some signing problem, especially with third-party S3-compatible object + stores. + +This is a very, very rare occurrence. + +If the problem is a signing one, try changing the signature algorithm. + +```xml + + fs.s3a.signing-algorithm + S3SignerType + +``` + +We cannot make any promises that it will work, only that it has been known to +make the problem go away "once" + +### `AWSS3IOException` The Content-MD5 you specified did not match what we received + +Reads work, but writes, even `mkdir`, fail: + +``` +org.apache.hadoop.fs.s3a.AWSS3IOException: copyFromLocalFile(file:/tmp/hello.txt, s3a://bucket/hello.txt) + on file:/tmp/hello.txt: + The Content-MD5 you specified did not match what we received. + (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), + S3 Extended Request ID: null + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:127) + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:69) + at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:1494) + at org.apache.hadoop.tools.cloudup.Cloudup.uploadOneFile(Cloudup.java:466) + at org.apache.hadoop.tools.cloudup.Cloudup.access$000(Cloudup.java:63) + at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:353) + at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:350) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) + at java.lang.Thread.run(Thread.java:748) +Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: + The Content-MD5 you specified did not match what we received. + (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), + S3 Extended Request ID: null + at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1307) + at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:894) + at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:597) + at com.amazonaws.http.AmazonHttpClient.doExecute(AmazonHttpClient.java:363) + at com.amazonaws.http.AmazonHttpClient.executeWithTimer(AmazonHttpClient.java:329) + at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:308) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3659) + at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1422) + at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131) + at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123) + at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139) + at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47) + at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) + ... 4 more +``` + +This stack trace was seen when interacting with a third-party S3 store whose +expectations of headers related to the AWS V4 signing mechanism was not +compatible with that of the specific AWS SDK Hadoop was using. + +Workaround: revert to V2 signing. + +```xml + + fs.s3a.signing-algorithm + S3SignerType + +``` + +### When writing data: "java.io.FileNotFoundException: Completing multi-part upload" + +A multipart upload was trying to complete, but failed as there was no upload +with that ID. + +``` +java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: + com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. + The upload ID may be invalid, or the upload may have been aborted or completed. + (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; + at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182) + at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770) + at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) + at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) + at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272) + at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) + at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) +``` + +This can happen when all outstanding uploads have been aborted, including the +active ones. + +If the bucket has a lifecycle policy of deleting multipart uploads, make sure +that the expiry time of the deletion is greater than that required for all open +writes to complete the write, +*and for all jobs using the S3A committers to commit their work.* + +### Application hangs after reading a number of files + +The pool of https client connections and/or IO threads have been used up, and +none are being freed. + +1. The pools aren't big enough. + See ["Timeout waiting for connection from pool"](#timeout_from_pool) +2. Likely root cause: whatever code is reading files isn't calling `close()` + on the input streams. Make sure your code does this! + And if it's someone else's: make sure you have a recent version; search their + issue trackers to see if its a known/fixed problem. If not, it's time to work + with the developers, or come up with a workaround + (i.e closing the input stream yourself). + +### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient` + +``` +[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) + - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond +org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond + at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143) + at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57) + at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261) + at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283) + at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259) + at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209) + at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272) + at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66) + at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124) + at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686) + at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488) + at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884) + at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) + at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) + at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728) + at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) + at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) + at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731) + at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41) + at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28) + at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) + at java.util.concurrent.FutureTask.run(FutureTask.java:266) + at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) + at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) + at java.lang.Thread.run(Thread.java:745) +``` + +These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client +will attempt to retry the operation; it may just be a transient event. If there +are many such exceptions in logs, it may be a symptom of connectivity or network +problems. + +### `AWSBadRequestException` IllegalLocationConstraintException/The unspecified location constraint is incompatible + +``` + Cause: org.apache.hadoop.fs.s3a.AWSBadRequestException: put on : + com.amazonaws.services.s3.model.AmazonS3Exception: + The unspecified location constraint is incompatible for the region specific + endpoint this request was sent to. + (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; + + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:178) + at org.apache.hadoop.fs.s3a.S3ALambda.execute(S3ALambda.java:64) + at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadObject(WriteOperationHelper.java:451) + at org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker.aboutToComplete(MagicCommitTracker.java:128) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:373) + at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) + at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) + at org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2429) + at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:106) + at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:91) + ... + Cause: com.amazonaws.services.s3.model.AmazonS3Exception: + The unspecified location constraint is incompatible for the region specific endpoint + this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; + Request ID: EEBC5A08BCB3A645) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667) + at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649) + at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221) + ... +``` + +Something has been trying to write data to "/". + +## Best Practises ### Enabling low-level logging @@ -1444,10 +1540,20 @@ http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Content-Length: 0 http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Server: AmazonS3 execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be kept alive for 60000 MILLISECONDS + ``` +### Enable S3 Server-side Logging -## Reducing failures by configuring retry policy +The [Auditing](auditing) feature of the S3A connector can be used to generate +S3 Server Logs with information which can be used to debug problems +working with S3, such as throttling events. + +Consult the [auditing documentation](auditing) documentation. +As auditing is enabled by default, enabling S3 Logging for a bucket +should be sufficient to collect these logs. + +### Reducing failures by configuring retry policy The S3A client can ba configured to retry those operations which are considered retryable. That can be because they are idempotent, or diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java index abd637a524..761dd55806 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java @@ -18,13 +18,25 @@ package org.apache.hadoop.fs.s3a; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + import com.amazonaws.ClientConfiguration; import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.util.AwsHostNameUtils; import org.assertj.core.api.Assertions; import org.junit.Test; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; + import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_REGION_SYSPROP; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test to check correctness of S3A endpoint regions in @@ -36,6 +48,7 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase { private static final String AWS_ENDPOINT_TEST = "test-endpoint"; private static final String AWS_ENDPOINT_TEST_WITH_REGION = "test-endpoint.some-region.amazonaws.com"; + public static final String MARS_NORTH_2 = "mars-north-2"; /** * Test to verify that setting a region with the config would bypass the @@ -88,4 +101,90 @@ private AwsClientBuilder.EndpointConfiguration createEpr(String endpoint, return DefaultS3ClientFactory.createEndpointConfiguration(endpoint, new ClientConfiguration(), awsRegion); } + + + @Test + public void testInvalidRegionDefaultEndpoint() throws Throwable { + describe("Create a client with an invalid region and the default endpoint"); + Configuration conf = getConfiguration(); + // we are making a big assumption about the timetable for AWS + // region rollout. + // if this test ever fails because this region now exists + // -congratulations! + conf.set(AWS_REGION, MARS_NORTH_2); + createMarsNorth2Client(conf); + } + + @Test + public void testUnsetRegionDefaultEndpoint() throws Throwable { + describe("Create a client with no region and the default endpoint"); + Configuration conf = getConfiguration(); + conf.unset(AWS_REGION); + createS3Client(conf, DEFAULT_ENDPOINT, AWS_S3_CENTRAL_REGION); + } + + /** + * By setting the system property {@code "aws.region"} we can + * guarantee that the SDK region resolution chain will always succeed + * (and fast). + * Clearly there is no validation of the region during the build process. + */ + @Test + public void testBlankRegionTriggersSDKResolution() throws Throwable { + describe("Create a client with a blank region and the default endpoint." + + " This will trigger the SDK Resolution chain"); + Configuration conf = getConfiguration(); + conf.set(AWS_REGION, ""); + System.setProperty(AWS_REGION_SYSPROP, MARS_NORTH_2); + try { + createMarsNorth2Client(conf); + } finally { + System.clearProperty(AWS_REGION_SYSPROP); + } + } + + /** + * Create an S3 client bonded to an invalid region; + * verify that calling {@code getRegion()} triggers + * a failure. + * @param conf configuration to use in the building. + */ + private void createMarsNorth2Client(Configuration conf) throws Exception { + AmazonS3 client = createS3Client(conf, DEFAULT_ENDPOINT, MARS_NORTH_2); + intercept(IllegalArgumentException.class, MARS_NORTH_2, client::getRegion); + } + + /** + * Create an S3 client with the given conf and endpoint. + * The region name must then match that of the expected + * value. + * @param conf configuration to use. + * @param endpoint endpoint. + * @param expectedRegion expected region + * @return the client. + * @throws URISyntaxException parse problems. + * @throws IOException IO problems + */ + private AmazonS3 createS3Client(Configuration conf, + String endpoint, + String expectedRegion) + throws URISyntaxException, IOException { + + DefaultS3ClientFactory factory + = new DefaultS3ClientFactory(); + factory.setConf(conf); + S3ClientFactory.S3ClientCreationParameters parameters + = new S3ClientFactory.S3ClientCreationParameters() + .withCredentialSet(new AnonymousAWSCredentialsProvider()) + .withEndpoint(endpoint) + .withMetrics(new EmptyS3AStatisticsContext() + .newStatisticsFromAwsSdk()); + AmazonS3 client = factory.createS3Client( + new URI("s3a://localhost/"), + parameters); + Assertions.assertThat(client.getRegionName()) + .describedAs("Client region name") + .isEqualTo(expectedRegion); + return client; + } }