[
https://issues.apache.org/jira/browse/HADOOP-16823?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17028458#comment-17028458
]
Steve Loughran commented on HADOOP-16823:
-----------------------------------------
I've realised and that if we handle all failures in the S3A code, throttling on
multi part copies may not be recovered from.
We delegate that work to the transfer manager into the SDK which partitions the
file, copies each part in a separate thread/HTTP request and then coalesces.
The only retry all logic our client puts around that is the full copy operation
-farriers on individual part copy made immediately escalate into the the
failure of the full copy. So imagine if you are copying a 2 GB file and a one
128 MB copy failed - the whole operation would fail and need to be retried.
We don't want that.
Ultimately we are going to have to move away from transfer manager -we have
discussed that in the past and given it is not in the 2.0 SDK something we will
be forced to do next year anyway. It's just been clear this would be an extra
piece of work we didn't really want to do until forced just because of the
effort it would take -at least 2-3 weeks and more scale tests to write.
I don't want to have to do it right now on what should be a small throttling
fix.
Plan:
* add a switch to throttle in the AWS client SDK or not -but leave on by default
* but shrink page size of a bulk delete to 200 entries, so it is less likely to
trigger overloads and the retries less traumatic.
* issue: hard code that vs configurable? fixed: simplifies test, avoids people
recreating the throttle problem and intentionally. Configurable: lets you alter
the settings during debugging without having to push out a whole new release
* Add a special throttling log4j log; logs normal throttling at info,
DeleteObjects at warn + size of entry and first & last paths. That way, blame
assignment even without full request context propagation,
> Manage S3 Throttling exclusively in S3A client
> ----------------------------------------------
>
> Key: HADOOP-16823
> URL: https://issues.apache.org/jira/browse/HADOOP-16823
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/s3
> Affects Versions: 3.2.1
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Minor
>
> Currently AWS S3 throttling is initially handled in the AWS SDK, only
> reaching the S3 client code after it has given up.
> This means we don't always directly observe when throttling is taking place.
> Proposed:
> * disable throttling retries in the AWS client library
> * add a quantile for the S3 throttle events, as DDB has
> * isolate counters of s3 and DDB throttle events to classify issues better
> Because we are taking over the AWS retries, we will need to expand the
> initial delay en retries and the number of retries we should support before
> giving up.
> Also: should we log throttling events? It could be useful but there is a risk
> of logs overloading especially if many threads in the same process were
> triggering the problem.
> Proposed: log at debug.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]