You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Steve Loughran (Jira)" <ji...@apache.org> on 2022/02/21 12:45:00 UTC

[jira] [Commented] (HADOOP-18132) S3 exponential backoff

    [ https://issues.apache.org/jira/browse/HADOOP-18132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17495506#comment-17495506 ] 

Steve Loughran commented on HADOOP-18132:
-----------------------------------------

holden,

as John Zhuge noted, this is already done.

* AWS SDK does its own throttling. you can disable this in  "fs.s3a.experimental.aws.s3.throttling", pushing it all up to the s3a code. but the copy operations during a rename don't have s3a code around it, so this isn't safe.
* the wrapping uses fs.s3a.retry.throttle.interval (500ms) for fs.s3a.retry.throttle.limit limit (20) as inputs to exponentialBackoffRetry(). see S3ARetryPolicy.


in MAPREDUCE-7341 I'm adding RateLimiter use to ensure that load against an abfs store stays below the throttle point. this is because abfs renames are a bit flaky once things overload.

now we have cut s3guard out of trunk, once the MAPREDUCE-7341 is in there's an opportunity for someone to provide rate limiting in s3a so we could limit the submission rate of operations to below the point where throttling occurs. i believe that staying below that sleep/backoff/retry point will actually deliver more throughput than recovery, because of all those pauses. it will also let us run background jobs with lower allocated capacity than the priority ones.

now, are you seeing real problems here?

1. look at HADOOP-16823. Are you doing big delete operations? and seeing problems there? as every delete in the bulk operation is a single write call. you can control the page size to spread load there and ensure the retries don't trigger throttling again.

2. hadoop 3.3.2 (latest RC is out) has an option to turn on auditing, after which we add an http referrer header which includes caller info as query params in the http referrer header. https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md 
if you are using the s3a committers, we even attach spark job id, so you get to do some good root cause tracking when throttling happens, even when it is multiple apps triggering the problem.

> S3 exponential backoff
> ----------------------
>
>                 Key: HADOOP-18132
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18132
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/s3
>            Reporter: Holden Karau
>            Priority: Major
>
> S3 API has limits which we can exceed when using a large number of writers/readers/or listers. We should add randomized-exponential back-off to the s3 client when it encounters:
>  
> com.amazonaws.services.s3.model.AmazonS3Exception: Please reduce your request rate. (Service: Amazon S3; Status Code: 503; Error Code: SlowDown; 
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

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