Details

    • Type: Sub-task
    • Status: In Progress
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 2.8.0
    • Fix Version/s: None
    • Component/s: fs/s3
    • Labels:
      None

      Description

      AWS S3, IAM, KMS, DDB etc all throttle callers: the S3A code needs to handle this without failing, as if it slows down its requests it can recover.

      1. Look at all the places where we are calling S3A via the AWS SDK and make sure we are retrying with some backoff & jitter policy, ideally something unified. This must be more systematic than the case-by-case, problem-by-problem strategy we are implicitly using.
      2. Many of the AWS S3 SDK calls do implement retry (e.g PUT/multipart PUT), but we need to check the other parts of the process: login, initiate/complete MPU, ...

      Related

      HADOOP-13811 Failed to sanitize XML document destined for handler class
      HADOOP-13664 S3AInputStream to use a retry policy on read failures

      This stuff is all hard to test. A key need is to be able to differentiate recoverable throttle & network failures from unrecoverable problems like: auth, network config (e.g bad endpoint), etc.

      May be the opportunity to add a faulting subclass of Amazon S3 client which can be configured in IT Tests to fail at specific points. Ryan Blue's mcok S3 client does this in HADOOP-13786, but it is for 100% mock. I'm thinking of something with similar fault raising, but in front of the real S3A client

        Issue Links

          Activity

          Hide
          stevel@apache.org Steve Loughran added a comment -

          If we could set the socket factory in the test JVMs, we could even simulate network failures under the AWS SDK. This would be the best fault injection of all short of having an SDN and custom DNS server to as part of the test infra

          Show
          stevel@apache.org Steve Loughran added a comment - If we could set the socket factory in the test JVMs, we could even simulate network failures under the AWS SDK. This would be the best fault injection of all short of having an SDN and custom DNS server to as part of the test infra
          Hide
          stevel@apache.org Steve Loughran added a comment -

          +retry logic should include handling of any load-throttling problems related to integrated services, such as SSE-KMS. S3Guard is already doing this for DDB.

          Show
          stevel@apache.org Steve Loughran added a comment - +retry logic should include handling of any load-throttling problems related to integrated services, such as SSE-KMS. S3Guard is already doing this for DDB.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          +DELETE would seem something to look at here; you don't want a delete to fail just because it went down once. Complex idempotency issues though. Two DELETE calls back-to-back are not idempotent, not if someone else gets in there first. Given this is an object store we are talking about, not a filesystem, does that matter?

          Show
          stevel@apache.org Steve Loughran added a comment - +DELETE would seem something to look at here; you don't want a delete to fail just because it went down once. Complex idempotency issues though. Two DELETE calls back-to-back are not idempotent, not if someone else gets in there first. Given this is an object store we are talking about, not a filesystem, does that matter?
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13205 may be this too, though it's not as broad.

          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13205 may be this too, though it's not as broad.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          I'm thinking if we target Java 8+, we could do this with closures nicely, similar to how we now have intercept() and eventually() in LambdaTestUtils; some method which would take an IOE-raising closure and a retry policy, and repeat the closure until the retry policy gave up. On success: return the result of the operation

          MultipartUpload mpu = execute(new RetryAllButAuth(), () -> initiateMultipartUpload())
          

          This would really keep those try/catch/while loops under control, and make it a lot easier to use in what is becoming a fairly complex piece of code.

          Show
          stevel@apache.org Steve Loughran added a comment - I'm thinking if we target Java 8+, we could do this with closures nicely, similar to how we now have intercept() and eventually() in LambdaTestUtils; some method which would take an IOE-raising closure and a retry policy, and repeat the closure until the retry policy gave up. On success: return the result of the operation MultipartUpload mpu = execute( new RetryAllButAuth(), () -> initiateMultipartUpload()) This would really keep those try/catch/while loops under control, and make it a lot easier to use in what is becoming a fairly complex piece of code.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Transient failure, seen on the s3guard+committer branch, but without s3guard turned on.

          A 400 is trouble as it has so many meanings. The good news: the usual unrecoverable failures (auth &c) will show up in FS.initialize, so if
          we don't retry there (or only retry a couple of times), then maybe everywhere else a retry strategy will work

          Running org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir
          Tests run: 9, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 14.903 sec <<< FAILURE! - in org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir
          testRecursiveRootListing(org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir)  Time elapsed: 6.394 sec  <<< ERROR!
          org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on user/stevel/: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE), S3 Extended Request ID: fkUz/wPcebNi4Mp5fAwVWRw/BEPv/2fmn74+1bEqCft/yhp3xMfcQSYI7O56YF1YZ7NfDLUTzmw=: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE)
          	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:179)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1932)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1874)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1836)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1661)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1637)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1211)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218)
          	at org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest.testRecursiveRootListing(AbstractContractRootDirectoryTest.java:221)
          	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          	at java.lang.reflect.Method.invoke(Method.java:498)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
          	at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1245)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1130)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1915)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1874)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1836)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1661)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1637)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1211)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218)
          	at org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest.testRecursiveRootListing(AbstractContractRootDirectoryTest.java:221)
          	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          	at java.lang.reflect.Method.invoke(Method.java:498)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Transient failure, seen on the s3guard+committer branch, but without s3guard turned on. A 400 is trouble as it has so many meanings. The good news: the usual unrecoverable failures (auth &c) will show up in FS.initialize, so if we don't retry there (or only retry a couple of times), then maybe everywhere else a retry strategy will work Running org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir Tests run: 9, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 14.903 sec <<< FAILURE! - in org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir testRecursiveRootListing(org.apache.hadoop.fs.contract.s3a.ITestS3AContractRootDir) Time elapsed: 6.394 sec <<< ERROR! org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on user/stevel/: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE), S3 Extended Request ID: fkUz/wPcebNi4Mp5fAwVWRw/BEPv/2fmn74+1bEqCft/yhp3xMfcQSYI7O56YF1YZ7NfDLUTzmw=: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:179) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1932) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1874) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1836) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1661) at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1637) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1211) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218) at org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest.testRecursiveRootListing(AbstractContractRootDirectoryTest.java:221) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: A67536F45A9683CE) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132) at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1245) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1130) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1915) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1874) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1836) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1661) at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1637) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1211) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218) at org.apache.hadoop.fs.contract.ContractTestUtils.treeWalk(ContractTestUtils.java:1218) at org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest.testRecursiveRootListing(AbstractContractRootDirectoryTest.java:221) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          + S3aInputStream retries on some non-recoverable events, as it does one extra attempt on any exception. This can lead to 404s triggering a retry rather than fail fast.

          testSequentialRead(org.apache.hadoop.fs.contract.s3a.ITestS3AContractOpen)  Time elapsed: 1.221 sec  <<< ERROR!
          java.io.FileNotFoundException: Reopen at position 0 on s3a://hwdev-steve-ireland-new/fork-0007/test/testsequentialread.txt: com.amazonaws.services.s3.model.AmazonS3Exception: The specified key does not exist. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchKey; Request ID: 8D81F218D02DE21E), S3 Extended Request ID: aXUWP6yYGSsP9ofVawyIteGZWBmkNTFjmRCvwAR1KyJmtR0A6H6UOggE4OlYB2ZOJ99F3MV74fU=
          	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:166)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:165)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.onReadFailure(S3AInputStream.java:348)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:321)
          	at java.io.FilterInputStream.read(FilterInputStream.java:83)
          	at org.apache.hadoop.fs.contract.AbstractContractOpenTest.testSequentialRead(AbstractContractOpenTest.java:156)
          	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          	at java.lang.reflect.Method.invoke(Method.java:498)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The specified key does not exist. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchKey; Request ID: 8D81F218D02DE21E)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
          	at com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:1373)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:158)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.onReadFailure(S3AInputStream.java:348)
          	at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:321)
          	at java.io.FilterInputStream.read(FilterInputStream.java:83)
          	at org.apache.hadoop.fs.contract.AbstractContractOpenTest.testSequentialRead(AbstractContractOpenTest.java:156)
          	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          	at java.lang.reflect.Method.invoke(Method.java:498)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          

          Given this is a failure path, its not too expensive, but there are probably a few like this (auth fail too).

          Show
          stevel@apache.org Steve Loughran added a comment - + S3aInputStream retries on some non-recoverable events, as it does one extra attempt on any exception. This can lead to 404s triggering a retry rather than fail fast. testSequentialRead(org.apache.hadoop.fs.contract.s3a.ITestS3AContractOpen) Time elapsed: 1.221 sec <<< ERROR! java.io.FileNotFoundException: Reopen at position 0 on s3a: //hwdev-steve-ireland- new /fork-0007/test/testsequentialread.txt: com.amazonaws.services.s3.model.AmazonS3Exception: The specified key does not exist. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchKey; Request ID: 8D81F218D02DE21E), S3 Extended Request ID: aXUWP6yYGSsP9ofVawyIteGZWBmkNTFjmRCvwAR1KyJmtR0A6H6UOggE4OlYB2ZOJ99F3MV74fU= at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:166) at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:165) at org.apache.hadoop.fs.s3a.S3AInputStream.onReadFailure(S3AInputStream.java:348) at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:321) at java.io.FilterInputStream.read(FilterInputStream.java:83) at org.apache.hadoop.fs.contract.AbstractContractOpenTest.testSequentialRead(AbstractContractOpenTest.java:156) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The specified key does not exist. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchKey; Request ID: 8D81F218D02DE21E) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132) at com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:1373) at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:158) at org.apache.hadoop.fs.s3a.S3AInputStream.onReadFailure(S3AInputStream.java:348) at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:321) at java.io.FilterInputStream.read(FilterInputStream.java:83) at org.apache.hadoop.fs.contract.AbstractContractOpenTest.testSequentialRead(AbstractContractOpenTest.java:156) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) Given this is a failure path, its not too expensive, but there are probably a few like this (auth fail too).
          Hide
          stevel@apache.org Steve Loughran added a comment -

          see also HADOOP-11572 about how to handle exceptions raised on large delete sets. Retry there is less of "repeat the POST" as some more sophisticated recovery.

          Show
          stevel@apache.org Steve Loughran added a comment - see also HADOOP-11572 about how to handle exceptions raised on large delete sets. Retry there is less of "repeat the POST" as some more sophisticated recovery.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          from stack overflow, evidence the aws xfer manager doesn't retry on network failures\

          17/09/09 03:45:33 INFO AmazonHttpClient: Unable to execute HTTP request: Read timed out
          java.net.SocketTimeoutException: Read timed out
              at java.net.SocketInputStream.socketRead0(Native Method)
              at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
              at java.net.SocketInputStream.read(SocketInputStream.java:171)
              at java.net.SocketInputStream.read(SocketInputStream.java:141)
              at sun.security.ssl.InputRecord.readFully(InputRecord.java:465)
              at sun.security.ssl.InputRecord.read(InputRecord.java:503)
              at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:983)
              at sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:940)
              at sun.security.ssl.AppInputStream.read(AppInputStream.java:105)
              at org.apache.http.impl.io.AbstractSessionInputBuffer.fillBuffer(AbstractSessionInputBuffer.java:160)
              at org.apache.http.impl.io.SocketInputBuffer.fillBuffer(SocketInputBuffer.java:84)
              at org.apache.http.impl.io.AbstractSessionInputBuffer.readLine(AbstractSessionInputBuffer.java:273)
              at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:140)
              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.executeHelper(AmazonHttpClient.java:384)
              at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232)
              at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528)
              at com.amazonaws.services.s3.AmazonS3Client.copyObject(AmazonS3Client.java:1507)
              at com.amazonaws.services.s3.transfer.internal.CopyCallable.copyInOneChunk(CopyCallable.java:143)
              at com.amazonaws.services.s3.transfer.internal.CopyCallable.call(CopyCallable.java:131)
              at com.amazonaws.services.s3.transfer.internal.CopyMonitor.copy(CopyMonitor.java:189)
              at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:134)
              at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:46)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
          
          Show
          stevel@apache.org Steve Loughran added a comment - from stack overflow, evidence the aws xfer manager doesn't retry on network failures\ 17/09/09 03:45:33 INFO AmazonHttpClient: Unable to execute HTTP request: Read timed out java.net.SocketTimeoutException: Read timed out at java.net.SocketInputStream.socketRead0(Native Method) at java.net.SocketInputStream.socketRead(SocketInputStream.java:116) at java.net.SocketInputStream.read(SocketInputStream.java:171) at java.net.SocketInputStream.read(SocketInputStream.java:141) at sun.security.ssl.InputRecord.readFully(InputRecord.java:465) at sun.security.ssl.InputRecord.read(InputRecord.java:503) at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:983) at sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:940) at sun.security.ssl.AppInputStream.read(AppInputStream.java:105) at org.apache.http.impl.io.AbstractSessionInputBuffer.fillBuffer(AbstractSessionInputBuffer.java:160) at org.apache.http.impl.io.SocketInputBuffer.fillBuffer(SocketInputBuffer.java:84) at org.apache.http.impl.io.AbstractSessionInputBuffer.readLine(AbstractSessionInputBuffer.java:273) at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:140) 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.executeHelper(AmazonHttpClient.java:384) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:232) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3528) at com.amazonaws.services.s3.AmazonS3Client.copyObject(AmazonS3Client.java:1507) at com.amazonaws.services.s3.transfer.internal.CopyCallable.copyInOneChunk(CopyCallable.java:143) at com.amazonaws.services.s3.transfer.internal.CopyCallable.call(CopyCallable.java:131) at com.amazonaws.services.s3.transfer.internal.CopyMonitor.copy(CopyMonitor.java:189) at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:134) at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:46) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang. Thread .run( Thread .java:748)

            People

            • Assignee:
              stevel@apache.org Steve Loughran
              Reporter:
              stevel@apache.org Steve Loughran
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:

                Development