hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Aaron Fabbri (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13761) S3Guard: implement retries for DDB failures and throttling; translate exceptions
Date Thu, 25 Jan 2018 02:15:00 GMT

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

Aaron Fabbri commented on HADOOP-13761:
---------------------------------------

{quote}
Managed to break tests when working with a bucket whose DDB table was precreated at {{ hadoop
s3guard init -write 20 -read 20}} & five parallel test cases
{quote}

Just noticing your comment from September [~stevel@apache.org].  This is pretty annoying,
according to [sdk docs|https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Programming.Errors.html#Programming.Errors.RetryAndBackoff]:

{quote}
The AWS SDKs implement automatic retry logic and exponential backoff.
{quote}

I confirmed this when I created the {{ITestDynamoDBMetadataStoreScale}} tests with low provisioned
I/O as well.  However, here is a hint from those same SDK docs:

{quote}
However, after a minute, if the request has not succeeded, the problem might be the request
size exceeding your provisioned throughput, and not the request rate. 
{quote}

Hypothesis: SDK does behave as advertised (the bulk write interface explicitly exposes "stuff
left for retry" so it seems to be the intention), *except* when your batch size > provisioned
I/O units.

That is, you had a batch of over 20 items to write at once and the SDK gives up and says "give
me a smaller batch".

Another thing to note here, is this JIRA has a lot of work mentioned in it.. so heads up I
may split things up.

> S3Guard: implement retries for DDB failures and throttling; translate exceptions
> --------------------------------------------------------------------------------
>
>                 Key: HADOOP-13761
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13761
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.0.0-beta1
>            Reporter: Aaron Fabbri
>            Assignee: Aaron Fabbri
>            Priority: Blocker
>
> Following the S3AFileSystem integration patch in HADOOP-13651, we need to add retry logic.
> In HADOOP-13651, I added TODO comments in most of the places retry loops are needed,
including:
> - open(path).  If MetadataStore reflects recent create/move of file path, but we fail
to read it from S3, retry.
> - delete(path).  If deleteObject() on S3 fails, but MetadataStore shows the file exists,
retry.
> - rename(src,dest).  If source path is not visible in S3 yet, retry.
> - listFiles(). Skip for now. Not currently implemented in S3Guard. I will create a separate
JIRA for this as it will likely require interface changes (i.e. prefix or subtree scan).
> We may miss some cases initially and we should do failure injection testing to make sure
we're covered.  Failure injection tests can be a separate JIRA to make this easier to review.
> We also need basic configuration parameters around retry policy.  There should be a way
to specify maximum retry duration, as some applications would prefer to receive an error eventually,
than waiting indefinitely.  We should also be keeping statistics when inconsistency is detected
and we enter a retry loop.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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


Mime
View raw message