hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Steve Loughran (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13786) Add S3Guard committer for zero-rename commits to S3 endpoints
Date Fri, 15 Sep 2017 16:55:01 GMT

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

Steve Loughran commented on HADOOP-13786:
-----------------------------------------

Patch 038; 

# in sync with trunk now that MAPREDUCE-6956 starts to give us the committer flexibility in
FileOutputFormat we need.
# test executed with throttling ramped up to 50% of s3 requests; tuned tests and inconsistent
client for this to all work
 
One thing I've done to help myself and other understand what the retry policy is for methods
in and near s3aFS is to add a source-time-only annotation, @Retries, which I've used to 
{code}
  @Retries.Retry_translated
  private void createFakeDirectory(final String objectName)
      throws IOException {
    if (!objectName.endsWith("/")) {
      createEmptyObject(objectName + "/");
    } else {
      createEmptyObject(objectName);
    }
  }
{code}

That says: something underneath is retrying and translating. Which tells callers: don't bother
wrapping it. In contrast, this says once with no translation
{code]
  @Retries.Once_raw
  InitiateMultipartUploadResult initiateMultipartUpload(
      InitiateMultipartUploadRequest request) throws IOException {
    LOG.debug("Initiate multipart upload to {}", request.getKey());
    incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED);
    return getAmazonS3Client().initiateMultipartUpload(request);
  }
{code}

With the tags its easier to review the code, to make sure that between all low-level once_untranslated
methods and those of the public APIs, you have retry and exception translation. I like this
for seeing what's happening: it's clearer than just a comment in the javadocs, as it forces
you to say what's going on. It's all just declarative, nothing to actually read the tags (e.g
no javadoc plugin), and nothing in the binaries. Maybe we should add comments to the javadocs
unless I can actually automate the generation (which would be very slick)

tested: yes, ireland and frankfurt, as noted 50% error rate. Things doing rename() fail, because
I haven't gone near the retry logic there. But as the new committers don't do renames, they
are happy :)


> Add S3Guard committer for zero-rename commits to S3 endpoints
> -------------------------------------------------------------
>
>                 Key: HADOOP-13786
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13786
>             Project: Hadoop Common
>          Issue Type: New Feature
>          Components: fs/s3
>    Affects Versions: 3.0.0-beta1
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>         Attachments: cloud-intergration-test-failure.log, HADOOP-13786-036.patch, HADOOP-13786-037.patch,
HADOOP-13786-038.patch, HADOOP-13786-HADOOP-13345-001.patch, HADOOP-13786-HADOOP-13345-002.patch,
HADOOP-13786-HADOOP-13345-003.patch, HADOOP-13786-HADOOP-13345-004.patch, HADOOP-13786-HADOOP-13345-005.patch,
HADOOP-13786-HADOOP-13345-006.patch, HADOOP-13786-HADOOP-13345-006.patch, HADOOP-13786-HADOOP-13345-007.patch,
HADOOP-13786-HADOOP-13345-009.patch, HADOOP-13786-HADOOP-13345-010.patch, HADOOP-13786-HADOOP-13345-011.patch,
HADOOP-13786-HADOOP-13345-012.patch, HADOOP-13786-HADOOP-13345-013.patch, HADOOP-13786-HADOOP-13345-015.patch,
HADOOP-13786-HADOOP-13345-016.patch, HADOOP-13786-HADOOP-13345-017.patch, HADOOP-13786-HADOOP-13345-018.patch,
HADOOP-13786-HADOOP-13345-019.patch, HADOOP-13786-HADOOP-13345-020.patch, HADOOP-13786-HADOOP-13345-021.patch,
HADOOP-13786-HADOOP-13345-022.patch, HADOOP-13786-HADOOP-13345-023.patch, HADOOP-13786-HADOOP-13345-024.patch,
HADOOP-13786-HADOOP-13345-025.patch, HADOOP-13786-HADOOP-13345-026.patch, HADOOP-13786-HADOOP-13345-027.patch,
HADOOP-13786-HADOOP-13345-028.patch, HADOOP-13786-HADOOP-13345-028.patch, HADOOP-13786-HADOOP-13345-029.patch,
HADOOP-13786-HADOOP-13345-030.patch, HADOOP-13786-HADOOP-13345-031.patch, HADOOP-13786-HADOOP-13345-032.patch,
HADOOP-13786-HADOOP-13345-033.patch, HADOOP-13786-HADOOP-13345-035.patch, objectstore.pdf,
s3committer-master.zip
>
>
> A goal of this code is "support O(1) commits to S3 repositories in the presence of failures".
Implement it, including whatever is needed to demonstrate the correctness of the algorithm.
(that is, assuming that s3guard provides a consistent view of the presence/absence of blobs,
show that we can commit directly).
> I consider ourselves free to expose the blobstore-ness of the s3 output streams (ie.
not visible until the close()), if we need to use that to allow us to abort commit operations.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
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