[ 
https://issues.apache.org/jira/browse/HADOOP-18310?focusedWorklogId=784532&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-784532
 ]

ASF GitHub Bot logged work on HADOOP-18310:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 24/Jun/22 10:52
            Start Date: 24/Jun/22 10:52
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on code in PR #4483:
URL: https://github.com/apache/hadoop/pull/4483#discussion_r905945310


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java:
##########
@@ -214,7 +214,10 @@ protected Map<Class<? extends Exception>, RetryPolicy> 
createExceptionMap() {
 
     // policy on a 400/bad request still ambiguous.
     // Treated as an immediate failure
-    policyMap.put(AWSBadRequestException.class, fail);
+    RetryPolicy awsBadRequestExceptionRetryPolicy =
+        configuration.getBoolean(FAIL_ON_AWS_BAD_REQUEST, 
DEFAULT_FAIL_ON_AWS_BAD_REQUEST) ?
+            fail : retryIdempotentCalls;

Review Comment:
   1. should retry on all calls, rather than just idempotent ones, as long as 
we are confident that the request is never executed before the failure
   2. I don't believe the normal exponential backoff strategy is the right one, 
as the initial delays are very short lived (500ms), whereas if you are hoping 
that credential providers will fetch new credentials, an initial delay of a few 
seconds would seem better. I wouldn't even bother with exponential growth here, 
just say 6 times at 10 seconds.
   
   I think we would also want to log at warn that this is happening, assuming 
this is rare. 



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##########
@@ -1203,4 +1203,18 @@ private Constants() {
    * Default maximum read size in bytes during vectored reads : {@value}.
    */
   public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 
1253376; //1M
+
+  /**
+   * Flag for immediate failure when observing a {@link 
AWSBadRequestException}.
+   * If it's disabled and set to false, the failure is treated as retryable.
+   * Value {@value}.
+   */
+  public static final String FAIL_ON_AWS_BAD_REQUEST = 
"fs.s3a.fail.on.aws.bad.request";

Review Comment:
   I now think "fs.s3a.retry.on.400.response.enabled" would be better, with 
default flipped. docs would say "experimental"
   
   and assuming we do have a custom policy, adjacent 
   
   ```
   fs.s3a.retry.on.400.response.delay  // delay between attempts, default "10s"
   fs.s3a.retry.on.400.response.attempts // number of attempts, default 6
   ```
   
   fs.s3a.retry.on.400



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java:
##########
@@ -311,12 +311,25 @@ public void testRetryAWSConnectivity() throws Throwable {
    */
   @Test(expected = AWSBadRequestException.class)
   public void testRetryBadRequestNotIdempotent() throws Throwable {
-    invoker.retry("test", null, false,
+
+    invoker.retry("test", null, true,
         () -> {
           throw BAD_REQUEST;
         });
   }
 
+  @Test
+  public void testRetryBadRequestIdempotent() throws Throwable {

Review Comment:
   test looks ok.





Issue Time Tracking
-------------------

    Worklog Id:     (was: 784532)
    Time Spent: 2h 10m  (was: 2h)

> Add option and make 400 bad request retryable
> ---------------------------------------------
>
>                 Key: HADOOP-18310
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18310
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 3.3.4
>            Reporter: Tak-Lon (Stephen) Wu
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> When one is using a customized credential provider via 
> fs.s3a.aws.credentials.provider, e.g. 
> org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, when the provided 
> credential by this pluggable provider is expired and return an error code of 
> 400 as bad request exception.
> Here, the current S3ARetryPolicy will fail immediately and does not retry on 
> the S3A level. 
> Our recent use case in HBase found this use case could lead to a Region 
> Server got immediate abandoned from this Exception without retry, when the 
> file system is trying open or S3AInputStream is trying to reopen the file. 
> especially the S3AInputStream use cases, we cannot find a good way to retry 
> outside of the file system semantic (because if a ongoing stream is failing 
> currently it's considered as irreparable state), and thus we come up with 
> this optional flag for retrying in S3A.
> {code}
> Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The provided 
> token has expired. (Service: Amazon S3; Status Code: 400; Error Code: 
> ExpiredToken; Request ID: XYZ; S3 Extended Request ID: ABC; Proxy: null), S3 
> Extended Request ID: 123
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1862)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1415)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1384)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1154)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:811)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:779)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:753)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:713)
>       at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:695)
>       at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:559)
>       at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:539)
>       at 
> com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5453)
>       at 
> com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5400)
>       at 
> com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:1524)
>       at 
> org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1506)
>       at 
> org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:217)
>       at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
>       ... 35 more
> {code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to