[ https://issues.apache.org/jira/browse/HADOOP-13761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16363517#comment-16363517 ]
Aaron Fabbri commented on HADOOP-13761: --------------------------------------- Some entertaining logs from the new scale test introduced here (batch size exceeds provisioned throughput). Multiple of seconds of backoff timer.. still never fails though. {noformat} DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:completeAncestry(548)) - auto-create ancestor path s3a://bucket-new/scaleTestBWEP0 for child path s3a://bucket-new/scaleTestBWEP0/lvl1/lvl2/lvl3/lvl4/lvl5/lvl6/lvl7/lvl8/lvl9/lvl10/lvl11/lvl12/lvl13/lvl14/lvl15/lvl16/lvl17/lvl18/lvl19/lvl20/lvl21/lvl22/lvl23/lvl2423 DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:put(693)) - Saving batch of 25 items to table fabbri-dev3, region us-west-2 DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:retryBackoff(660)) - Sleeping 250 msec before next retry DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:retryBackoff(660)) - Sleeping 586 msec before next retry DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:retryBackoff(660)) - Sleeping 747 msec before next retry DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:retryBackoff(660)) - Sleeping 2295 msec before next retry DEBUG s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:put(681)) - Saving to table table-dev3 in region us-west-2: PathMetadata{fileStatus=FileStatus{path=s3a{noformat} > 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 > Attachments: HADOOP-13761.001.patch, HADOOP-13761.002.patch > > > 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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org