[ 
https://issues.apache.org/jira/browse/HADOOP-13887?focusedWorklogId=613517&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-613517
 ]

ASF GitHub Bot logged work on HADOOP-13887:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 22/Jun/21 14:27
            Start Date: 22/Jun/21 14:27
    Worklog Time Spent: 10m 
      Work Description: mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-866032252


   Force pushing due to merge conflicts. 
   
   - **Multipart upload:** 
              - Added the serial uploads by setting the "Active upload blocks" 
equal to 1, when CSE is enabled. 
              - For the last part, while closing the S3ABlockOutputStream, 
we'll pass "isCSEEnabled" as the parameter for lastPart while uploading so that 
we can have isLastPart=true for CSE. 
              - There is a boundary condition when part size == last part or 
part size is multiple of bytes you have to upload, that we don't have any 
active blocks while closing the stream. For this, we would skip the uploading 
of the last block in write and wait for close(), so that our isLastPart = true 
works every time.
   - **Multipart Get:** 
              - So, when we upload via multipart in CSE, not all blocks are 
padded with 16 bytes, hence, we don't need to worry about the HEAD call we were 
thinking of doing while trying to figure out the content length. We can just 
strip out the padded length from the file. 
              - UNENCRYPTED_CONTENT_LENGTH header is also not included while 
multi-part upload in CSE, so, we would have to subtract 16 bytes while 
s3aGetFileStatus. 
   - Only multipart work is done in S3ABlockOutputStream, so directly calling 
multipart from S3AFilesystem won't work, thus the hasCapabilities changes. 
   - Custom Signer, as @bogthe said earlier, that this header : 
x-amz-content-sha256:UNSIGNED-PAYLOAD, should only be used for S3 Service, that 
holds true, and while signing in ITestCustomSigner, we were using AWSS3V4Signer 
even for AWSKMS service rather than AWSV4Signer, which works for AWSKMS, since 
it doesn't have that header by default. 
   
   Ran tests for both with and without CSE on ap-south-1: 
   Without CSE: 
   ```
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 1434, Failures: 0, Errors: 0, Skipped: 466
   
   Scale: 
   
   [ERROR] Errors: 
   [ERROR]   
ITestS3AContractRootDir>AbstractContractRootDirectoryTest.testRecursiveRootListing:267
 » TestTimedOut
   [INFO] 
   [ERROR] Tests run: 151, Failures: 0, Errors: 1, Skipped: 28
   ```
   
   With CSE: 
   ```
   [INFO] 
   [ERROR] Tests run: 1435, Failures: 0, Errors: 17, Skipped: 551
   
   Scale:
   
   [ERROR] Errors: 
   [ERROR]   
ITestS3AContractRootDir>AbstractContractRootDirectoryTest.testRecursiveRootListing:267
 » TestTimedOut
   [INFO] 
   [ERROR] Tests run: 151, Failures: 0, Errors: 1, Skipped: 28
   ```
   
   Errors in CSE are: 
   ```
   [ERROR] 
testMultipartUploadAbort(org.apache.hadoop.fs.contract.s3a.ITestS3AContractMultipartUploader)
  Time elapsed: 1.23 s  <<< ERROR!
   java.lang.UnsupportedOperationException: Multi-part uploader not supported 
for Client side encryption.
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.createMultipartUploader(S3AFileSystem.java:5041)
   ```
   The weird thing is it skips in IDE, but for some reason, I am not able to 
make it skip in mvn terminal. 
   This is the code used to skip this in the case of CSE in the setup() of 
AbstractContractMultipartUploaderTest.java: 
   ```
   Assume.assumeTrue("Multipart uploader is not supported",
           fs.hasPathCapability(testPath, 
CommonPathCapabilities.FS_MULTIPART_UPLOADER));
   ```
   Any feedback as to why this is happening would be really helpful. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 613517)
    Time Spent: 3.5h  (was: 3h 20m)

> Encrypt S3A data client-side with AWS SDK (S3-CSE)
> --------------------------------------------------
>
>                 Key: HADOOP-13887
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13887
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 2.8.0
>            Reporter: Jeeyoung Kim
>            Assignee: Igor Mazur
>            Priority: Minor
>              Labels: pull-request-available
>         Attachments: HADOOP-13887-002.patch, HADOOP-13887-007.patch, 
> HADOOP-13887-branch-2-003.patch, HADOOP-13897-branch-2-004.patch, 
> HADOOP-13897-branch-2-005.patch, HADOOP-13897-branch-2-006.patch, 
> HADOOP-13897-branch-2-008.patch, HADOOP-13897-branch-2-009.patch, 
> HADOOP-13897-branch-2-010.patch, HADOOP-13897-branch-2-012.patch, 
> HADOOP-13897-branch-2-014.patch, HADOOP-13897-trunk-011.patch, 
> HADOOP-13897-trunk-013.patch, HADOOP-14171-001.patch, S3-CSE Proposal.pdf
>
>          Time Spent: 3.5h
>  Remaining Estimate: 0h
>
> Expose the client-side encryption option documented in Amazon S3 
> documentation  - 
> http://docs.aws.amazon.com/AmazonS3/latest/dev/UsingClientSideEncryption.html
> Currently this is not exposed in Hadoop but it is exposed as an option in AWS 
> Java SDK, which Hadoop currently includes. It should be trivial to propagate 
> this as a parameter passed to the S3client used in S3AFileSystem.java



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to