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 7d167446c6..5caadd73a8 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 @@ -228,6 +228,41 @@ As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`: ``` + +### "403 Access denied" when trying to write data + +Data can be read, but attempts to write data or manipulate the store fail with +403/Access denied. + +The bucket may have an access policy which the request does not comply with. + +``` +java.nio.file.AccessDeniedException: test/: PUT 0-byte object on test/: + com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; + Error Code: AccessDenied; Request ID: EDC662AD2EEEA33C; + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:210) + at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:110) + at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:259) + at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:313) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:255) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:230) + at org.apache.hadoop.fs.s3a.S3AFileSystem.createEmptyObject(S3AFileSystem.java:2691) + at org.apache.hadoop.fs.s3a.S3AFileSystem.createFakeDirectory(S3AFileSystem.java:2666) + at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:2030) + at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1965) + at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2305) +``` + +In the AWS S3 management console, select the "permissions" tab for the bucket, then "bucket policy". +If there is no bucket policy, then the error cannot be caused by one. + +If there is a bucket access policy, e.g. required encryption headers, +then the settings of the s3a client must guarantee the relevant headers are set +(e.g. the encryption options match). +Note: S3 Default Encryption options are not considered here: +if the bucket policy requires AES256 as the encryption policy on PUT requests, +then the encryption option must be set in the s3a client so that the header is set. + ## Connectivity Problems ### Error message "The bucket you are attempting to access must be addressed using the specified endpoint" @@ -412,8 +447,13 @@ It may go away if the operation is retried. ### 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; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA= +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) @@ -427,6 +467,104 @@ java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multi 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 connectons and/or IO threads have been used up, +and none are being freed. + + +1. The pools aren't big enough. Increas `fs.s3a.connection.maximum` for +the http connections, and `fs.s3a.threads.max` for the thread 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). + +### "Timeout waiting for connection from pool" + +This the same problem as above, exhibiting itself as the http connection +pool determining that it has run out of capacity. + +``` + +java.io.InterruptedIOException: getFileStatus on s3a://example/fork-0007/test: + com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:145) + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:119) + at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2040) + at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857) + at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890) + at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826) + at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230) + ... +Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleRetryableException(AmazonHttpClient.java:1069) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035) + 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) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168) + at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249) + at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162) + at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022) + at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857) + at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890) + at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826) + at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230) +... +Caused by: com.amazonaws.thirdparty.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool + at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager.leaseConnection(PoolingHttpClientConnectionManager.java:286) + at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager$1.get(PoolingHttpClientConnectionManager.java:263) + at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:498) + at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70) + at com.amazonaws.http.conn.$Proxy15.get(Unknown Source) + at com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:190) + at com.amazonaws.thirdparty.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184) + at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184) + at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) + at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) + at com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(SdkHttpClient.java:72) + at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1190) + 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) + at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168) + at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249) + at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162) + at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022) + at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857) + at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890) + at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826) + at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230) +``` + +This is the same problem as the previous one, exhibited differently. + ### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient` ```