You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/02/18 14:35:43 UTC

[GitHub] [hadoop] mehakmeet opened a new pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

mehakmeet opened a new pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706


   Tests: mvn -T 32 clean verify -Ddynamo -Dauth -Dscale -Dparallel-tests
   Region: ap-south-1
   
   ```
   [INFO] Results:
   [INFO]
   [WARNING] Tests run: 535, Failures: 0, Errors: 0, Skipped: 5
   ```
   
   ```
   [INFO] Results:
   [INFO]
   [ERROR] Failures:
   [ERROR]   ITestS3AEncryptionCSEAsymmetric>ITestS3AEncryptionCSE.testEncryption:48->ITestS3AEncryptionCSE.validateEncryptionForFilesize:82->AbstractS3ATestBase.writeThenReadFile:196->AbstractS3ATestBase.writeThenReadFile:209->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0001/test/testEncryption0 status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0001/test/testEncryption0; isDirectory=false; length=16; replication=1; blocksize=33554432; modification_time=1613656460000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=91f7a9a65b91c2332c572ea76bdbe822 versionId=null expected:<0> but was:<16>
   [ERROR]   ITestS3AEncryptionCSEAsymmetric>ITestS3AEncryptionCSE.testEncryptionOverRename:63->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0001/test/testEncryptionOverRename status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0001/test/testEncryptionOverRename; isDirectory=false; length=1040; replication=1; blocksize=33554432; modification_time=1613656458000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=97712202251ecc080bbbf909c9000718 versionId=null expected:<1024> but was:<1040>
   [ERROR]   ITestS3AEncryptionCSEKms>ITestS3AEncryptionCSE.testEncryption:48->ITestS3AEncryptionCSE.validateEncryptionForFilesize:82->AbstractS3ATestBase.writeThenReadFile:196->AbstractS3ATestBase.writeThenReadFile:209->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0001/test/testEncryption0 status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0001/test/testEncryption0; isDirectory=false; length=16; replication=1; blocksize=33554432; modification_time=1613656468000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=74cb30cc19ee58be3bcd462120c16eb7 versionId=null expected:<0> but was:<16>
   [ERROR]   ITestS3AEncryptionCSEKms>ITestS3AEncryptionCSE.testEncryptionOverRename:63->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0001/test/testEncryptionOverRename status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0001/test/testEncryptionOverRename; isDirectory=false; length=1040; replication=1; blocksize=33554432; modification_time=1613656465000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=d03375638396ae14794cde1a7e5fd4f8 versionId=null expected:<1024> but was:<1040>
   [ERROR]   ITestS3AEncryptionCSESymmetric>ITestS3AEncryptionCSE.testEncryption:48->ITestS3AEncryptionCSE.validateEncryptionForFilesize:82->AbstractS3ATestBase.writeThenReadFile:196->AbstractS3ATestBase.writeThenReadFile:209->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0002/test/testEncryption0 status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0002/test/testEncryption0; isDirectory=false; length=16; replication=1; blocksize=33554432; modification_time=1613656453000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=ff01163e592622879cd0cdb7005618ca versionId=null expected:<0> but was:<16>
   [ERROR]   ITestS3AEncryptionCSESymmetric>ITestS3AEncryptionCSE.testEncryptionOverRename:63->Assert.assertEquals:645->Assert.failNotEquals:834->Assert.fail:88 Wrong file length of file s3a://mehakmeet-singh-data/fork-0002/test/testEncryptionOverRename status: S3AFileStatus{path=s3a://mehakmeet-singh-data/fork-0002/test/testEncryptionOverRename; isDirectory=false; length=1040; replication=1; blocksize=33554432; modification_time=1613656451000; access_time=0; owner=mehakmeet.singh; group=mehakmeet.singh; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=true; isErasureCoded=false} isEmptyDirectory=FALSE eTag=1802986c25982cdacb58aff624626eeb versionId=null expected:<1024> but was:<1040>
   ```
   When we do fs.getFileStatus().getLen() to get the content length of the encrypted file, it is not the same as the original length and hence, the tests are breaking.
   
   ```[INFO]
   [ERROR] Tests run: 1427, Failures: 7, Errors: 21, Skipped: 458
   ```
    Failures other than S3-CSE tests are config related
    
    ```
    [ERROR] Errors:
   [ERROR]   ITestS3AContractRootDir>AbstractContractRootDirectoryTest.testRecursiveRootListing:267 » TestTimedOut
   [INFO]
   [ERROR] Tests run: 151, Failures: 0, Errors: 1, Skipped: 28
   ```
   
   When I remove the checks for file content Lengths from the tests, the tests run successfully, hence, the Key-wrap Algo and Content Encryption Algo are used as intended successfully. 
   
   One possible way we explored to tackle the padding issue was to tweak the s3GetFileStatus call to return a FileStatus with "UNENCRYPTED_CONTENT_LENGTH" header which comes in the user metadata, but this would still break where we don't get these headers to do our tasks. Hence, consistency of content Length needs to be maintained. 
   P.S: Need more tests to validate that even with tweaking, we are breaking some tests.
   
   CC: @steveloughran  
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-787448183


   we get the filelength back in the list API calls; with [the v2 LIST](https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjectsV2.html) being the one we default to. it returns file name, length, timestamp, etag, storage class and optionally owner (we don't)
   
   This is what has blocked us before: code which goes from list dir -> fileStatus[]. foreach.open().read().
   
   if the length of a file in listing != that of the file in read, things break. Lots of things. 
   
   if we still can't do unpadded CSE, it's still unusable for our workloads
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672611147



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -43,9 +52,12 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       losing battle here. I've got it wrong in some of the audit work and then it complicates backporting




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r658081650



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -94,9 +105,81 @@ public AmazonS3 createS3Client(
       awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix());
     }
 
-    return buildAmazonS3Client(
-        awsConf,
-        parameters);
+    if (conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) == null) {
+      return buildAmazonS3Client(
+          awsConf,
+          parameters);
+    } else {
+      return newAmazonS3EncryptionClient(
+          awsConf,
+          parameters);
+    }
+  }
+
+  /**
+   * Create an {@link AmazonS3} client of type
+   * {@link AmazonS3EncryptionV2} if CSE is enabled.
+   *
+   * @param awsConf    AWS configuration.
+   * @param parameters parameters
+   *
+   * @return new AmazonS3 client.
+   */
+  protected AmazonS3 newAmazonS3EncryptionClient(
+      final ClientConfiguration awsConf,
+      final S3ClientCreationParameters parameters){
+
+    AmazonS3 client;
+    AmazonS3EncryptionClientV2Builder builder =

Review comment:
       As this extends AmazonS3Builder, it should be possible for all the common setup code (endpoints, regions etc) to be pulled out into a shared method. This will help us the next time we have to do emergency fixes, and will ensure that changes to the normal path also get picked up by by the new client

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -356,6 +357,19 @@
   private AuditManagerS3A auditManager =
       AuditIntegration.stubAuditManager();
 
+  /**
+   * S3 client side encryption adds padding to the content length of constant
+   * length of 16 bytes(at the moment, since we have only 1 content

Review comment:
       nit: add space before bracket

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -760,7 +768,8 @@ public void maybeRethrowUploadFailure() throws IOException {
      * @throws IOException upload failure
      * @throws PathIOException if too many blocks were written
      */
-    private void uploadBlockAsync(final S3ADataBlocks.DataBlock block)
+    private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
+        Boolean isLast)

Review comment:
       why `Boolean` and not `boolean`?

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -508,6 +524,10 @@ public void initialize(URI name, Configuration originalConf)
       blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
       blockOutputActiveBlocks = intOption(conf,
           FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
+      // If CSE is enabled, do multipart uploads serially.
+      if(isCSEEnabled) {

Review comment:
       * nit: add a space
   * log at debug that CSE is enabled

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while(listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path filePath = path(getMethodName());
+    byte[] fileContent = new byte[BIG_FILE_SIZE];
+    // PUT a 15MB file using CSE to simulate multipart in CSE.
+    createFile(fs, filePath, true, fileContent);
+    LOG.info("Multi-part upload successful...");
+
+    try(FSDataInputStream in = fs.open(filePath)) {
+      in.seek(BIG_FILE_SIZE);
+      in.seek(0);
+      in.readFully(0, fileContent);
+
+      verifyFileContents(fs, filePath, fileContent);
+    }
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+
+    // To simulate multi part put and get in small files, we'll set the
+    // threshold and part size to 5MB.
+    conf.set(Constants.MULTIPART_SIZE, String.valueOf(MULTIPART_MIN_SIZE));
+    conf.set(Constants.MIN_MULTIPART_THRESHOLD, String.valueOf(MULTIPART_MIN_SIZE));
+    return conf;
+  }
+
+  /**
+   * Method to validate CSE for different file sizes.
+   *
+   * @param len length of the file.
+   */
+  protected void validateEncryptionForFilesize(int len) throws IOException {
+    skipTest();
+    describe("Create an encrypted file of size " + len);
+    // Creating a unique path by adding file length in file name.
+    Path path = writeThenReadFile(getMethodName() + len, len);
+    assertEncrypted(path);
+    rm(getFileSystem(), path, false, false);
+  }
+
+  /**
+   * Skip tests if certain conditions are met.
+   */
+  protected abstract void skipTest();

Review comment:
       rename `maybeSkipTest()`

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
##########
@@ -239,6 +240,7 @@ protected String getTestTableName(String suffix) {
   }
 
   /**
+<<<<<<< HEAD

Review comment:
       merge marker in comments; easily missed. 

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while(listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path filePath = path(getMethodName());
+    byte[] fileContent = new byte[BIG_FILE_SIZE];
+    // PUT a 15MB file using CSE to simulate multipart in CSE.
+    createFile(fs, filePath, true, fileContent);
+    LOG.info("Multi-part upload successful...");
+
+    try(FSDataInputStream in = fs.open(filePath)) {
+      in.seek(BIG_FILE_SIZE);
+      in.seek(0);
+      in.readFully(0, fileContent);
+
+      verifyFileContents(fs, filePath, fileContent);
+    }
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+
+    // To simulate multi part put and get in small files, we'll set the
+    // threshold and part size to 5MB.
+    conf.set(Constants.MULTIPART_SIZE, String.valueOf(MULTIPART_MIN_SIZE));

Review comment:
       call S3ATestUtils.removeBaseAndBucketOverrides on these options to make sure there's no bucket override in anyone's test setup which will then cause tests to fail

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -3655,7 +3676,14 @@ S3AFileStatus s3GetFileStatus(final Path path,
         // look for the simple file
         ObjectMetadata meta = getObjectMetadata(key);
         LOG.debug("Found exact file: normal file {}", key);
-        return new S3AFileStatus(meta.getContentLength(),
+        long contentLength = meta.getContentLength();
+        // check if CSE is enabled, then strip padded length.
+        if (isCSEEnabled
+            && meta.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
+            && contentLength >= CSE_PADDING_LENGTH) {

Review comment:
       isn't there some unpadded length header we can use?

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -356,6 +357,19 @@
   private AuditManagerS3A auditManager =
       AuditIntegration.stubAuditManager();
 
+  /**
+   * S3 client side encryption adds padding to the content length of constant
+   * length of 16 bytes(at the moment, since we have only 1 content
+   * encryption algorithm). Use this to subtract while listing the content
+   * length when certain conditions are met.
+   */
+  public static final int CSE_PADDING_LENGTH = 16;

Review comment:
       move to InternalConstants and refer to it when the padding is needed, e.g in tests

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
##########
@@ -62,10 +67,25 @@ protected void verifyReadBytes(FileSystem.Statistics stats) {
     Assert.assertEquals(2 * blockSize, stats.getBytesRead());
   }
 
+  /**
+   * A method to verify the bytes written.
+   *
+   * NOTE: if Client side encryption is enabled, expected bytes written
+   * should increase by 16(padding of data) + 130(KMS key generation) in case
+   * of storage type{@link CryptoStorageMode} as ObjectMetadata(Default). If
+   * Crypto Storage mode is instruction file then add additional bytes as
+   * that file is stored separately and would account for bytes written.
+   *
+   * @param stats Filesystem statistics.
+   */
   @Override
   protected void verifyWrittenBytes(FileSystem.Statistics stats) {
     //No extra bytes are written
-    Assert.assertEquals(blockSize, stats.getBytesWritten());
+    long expectedBlockSize = blockSize;
+    if(conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) != null) {
+      expectedBlockSize += 16 + 130;

Review comment:
       is that 16 some constant somewhere? If so, please switch to make it more obvious

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while(listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path filePath = path(getMethodName());
+    byte[] fileContent = new byte[BIG_FILE_SIZE];
+    // PUT a 15MB file using CSE to simulate multipart in CSE.

Review comment:
       "force"

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -5075,17 +5103,22 @@ public boolean hasPathCapability(final Path path, final String capability)
       return isMagicCommitEnabled();
 
     case SelectConstants.S3_SELECT_CAPABILITY:
-      // select is only supported if enabled
-      return SelectBinding.isSelectEnabled(getConf());
+      // select is only supported if enabled and client side encryption is
+      // disabled
+      if(!isCSEEnabled) {

Review comment:
       simpler jut to merge with the return, eg.
   
   ```
   return !isCSEEnabled && electBinding.isSelectEnabled(getConf());
   ```
   
   Also, will `requireSelectSupport` need the same check?

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
##########
@@ -70,13 +70,16 @@ public void setup() throws Exception {
     invoker = new Invoker(new S3ARetryPolicy(getConfiguration()),
         Invoker.NO_OP
     );
+    ifCSEThenSkip();

Review comment:
       should be `skipIfClientSideException()` for consistency with the others

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
##########
@@ -182,10 +197,21 @@ public void sign(SignableRequest<?> request, AWSCredentials credentials) {
       } catch (IOException e) {
         throw new RuntimeException("Failed to get current Ugi", e);
       }
-      AWSS3V4Signer realSigner = new AWSS3V4Signer();
-      realSigner.setServiceName("s3");
-      realSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY));
-      realSigner.sign(request, credentials);
+      if(bucketName.equals("kms")) {

Review comment:
       nit: add spaces before the (s

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while(listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    skipTest();

Review comment:
       this should only execute on a scale run. `getConf().getBoolean(KEY_SCALE_TESTS_ENABLED, false)` should provide that

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
##########
@@ -524,10 +525,15 @@ public static S3AFileStatus createFileStatus(Path keyPath,
       long blockSize,
       String owner,
       String eTag,
-      String versionId) {
+      String versionId,
+      boolean isCSEEnabled) {

Review comment:
       do you think it's time we moved to a builder here?

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);

Review comment:
       wondering if we should use DurationInfo to log durations of the write rename and verify operations? Might be interesting to look at

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "

Review comment:
       nit, "aren't the same"

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
##########
@@ -62,10 +67,25 @@ protected void verifyReadBytes(FileSystem.Statistics stats) {
     Assert.assertEquals(2 * blockSize, stats.getBytesRead());
   }
 
+  /**
+   * A method to verify the bytes written.
+   *
+   * NOTE: if Client side encryption is enabled, expected bytes written
+   * should increase by 16(padding of data) + 130(KMS key generation) in case
+   * of storage type{@link CryptoStorageMode} as ObjectMetadata(Default). If
+   * Crypto Storage mode is instruction file then add additional bytes as
+   * that file is stored separately and would account for bytes written.
+   *
+   * @param stats Filesystem statistics.
+   */
   @Override
   protected void verifyWrittenBytes(FileSystem.Statistics stats) {
     //No extra bytes are written
-    Assert.assertEquals(blockSize, stats.getBytesWritten());
+    long expectedBlockSize = blockSize;
+    if(conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) != null) {
+      expectedBlockSize += 16 + 130;
+    }
+    Assert.assertEquals(expectedBlockSize, stats.getBytesWritten());

Review comment:
       add message for the assert

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryptionKms.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+
+import com.amazonaws.services.s3.Headers;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import org.assertj.core.api.Assertions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfKmsKeyIdIsNotSet;
+
+/**
+ * Testing the S3 CSE - KME method.
+ */
+public class ITestS3AClientSideEncryptionKms
+    extends ITestS3AClientSideEncryption {
+
+  private static final String KMS_KEY_WRAP_ALGO = "kms+context";
+  private static final String KMS_CONTENT_ENCRYPTION_ALGO = "AES/GCM/NoPadding";
+
+  /**
+   * Creating custom configs for KMS testing.
+   *
+   * @return Configuration.
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    return conf;
+  }
+
+  @Override
+  protected void skipTest() {
+    skipIfEncryptionTestsDisabled(getConfiguration());
+    skipIfKmsKeyIdIsNotSet(getConfiguration());
+  }
+
+  @Override
+  protected void assertEncrypted(Path path) throws IOException {
+    ObjectMetadata md = getFileSystem().getObjectMetadata(path);

Review comment:
       what about using fs.getXAttr() to retrieve these headers? It'd help verify that codepath

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
##########
@@ -301,8 +301,15 @@ public HeaderProcessing(final StoreContext storeContext,
         md.getContentEncoding());
     maybeSetHeader(headers, XA_CONTENT_LANGUAGE,
         md.getContentLanguage());
-    maybeSetHeader(headers, XA_CONTENT_LENGTH,
-        md.getContentLength());
+    // If CSE is enabled, use the unencrypted content length.
+    if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
+        && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) {

Review comment:
       nice catch




-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet edited a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet edited a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-783802582


   Also on that note, I figured something in the aws-sdk while doing a put request we have this in S3CryptoModuleBase.java:
   ```
   // Record the original, unencrypted content-length so it can be accessed
           // later
           final long plaintextLength = plaintextLength(request, metadata);
           if (plaintextLength >= 0) {
               metadata.addUserMetadata(Headers.UNENCRYPTED_CONTENT_LENGTH,
                                        Long.toString(plaintextLength));
               // Put the ciphertext length in the metadata
               metadata.setContentLength(ciphertextLength(plaintextLength));
           }
           request.setMetadata(metadata);
   ```
   Every put request in AmazonS3EncryptionClientV2, is wrapped like this. but it seems this isn’t the issue, in LIST ops, we don’t use these, right? so, how do we get file length in LIST calls?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-883337716


   > What was the conclusion on the ITestS3AFileContextStatistics failure? Side effect of the padding?
   
   Seems like there are few different ways to set the KMS Key ID, other than just using the Key ID. 
   - Key ID.
   - Key ARN.
   - Alias ARN.
   - Alias name.
   
   These all would result in different number of bytes sent to KMS service for key generation op, resulting in different bytesWritten value. 
   This test was always kind of a workaround, with having some constant bytes, now I have added the value of key ID set, so that it works for different method, but other request params like "EncryptionContext" and "KeySpec", still added as constant bytes.  


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-882843973


   catching up on this; letting you go through bogdan's comments first.
   
   What was the conclusion on the ITestS3AFileContextStatistics failure? Side effect of the padding?


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841789913


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 10 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 51s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  7s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  17m  0s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt) |  hadoop-tools/hadoop-aws: The patch generated 6 new + 38 unchanged - 0 fixed = 44 total (was 38)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 13s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m 11s | [/new-spotbugs-hadoop-tools_hadoop-aws.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/new-spotbugs-hadoop-tools_hadoop-aws.html) |  hadoop-tools/hadoop-aws generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  16m 18s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  7s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 40s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-aws |
   |  |  org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled isn't final and can't be protected from malicious code  In S3AFileSystem.java:be protected from malicious code  In S3AFileSystem.java |
   |  |  Write to static field org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:[line 531] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5935ac359db6 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3b9bf5810138c52b75f5df50c5443723b4ec9f97 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/testReport/ |
   | Max. process+thread count | 515 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r668725098



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
##########
@@ -2029,3 +2029,51 @@ signer for all services.
 For a specific service, the service specific signer is looked up first.
 If that is not specified, the common signer is looked up. If this is
 not specified as well, SDK settings are used.
+
+### <a name="cse"></a> Amazon S3 Client Side Encryption
+Amazon S3 Client Side Encryption(CSE), uses `AmazonS3EncryptionClientV2.java
+` AmazonS3 client. The encryption and decryption is done in AWS SDK side. Atm
+, Only KMS CSE method is supported.
+
+What is needed to be set to enable CSE for a bucket?
+- Create an AWS KMS Key ID from AWS console for your bucket, with same region as
+ your bucket.
+- If already created, [view the kms key ID by these steps.](https://docs.aws.amazon.com/kms/latest/developerguide/find-cmk-id-arn.html)
+- Set `fs.s3a.cse.method=KMS`.
+- Set `fs.s3a.cse.kms.keyId=<KMS_KEY_ID>`.

Review comment:
       I think property name is wrong. should be `fs.s3a.cse.kms.key-id`. Found while running tests. 




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879636061


   I setup the CSE tests with S3guard off. All good except one test failing always
   
   > [ERROR] testStatistics(org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics)  Time elapsed: 4.229 s  <<< FAILURE!
   java.lang.AssertionError: Mismatch in bytes written expected:<658> but was:<698>
   	at org.junit.Assert.fail(Assert.java:89)
   	at org.junit.Assert.failNotEquals(Assert.java:835)
   	at org.junit.Assert.assertEquals(Assert.java:647)
   	at org.apache.hadoop.fs.s3a.fileContext.ITestS3AFileContextStatistics.verifyWrittenBytes(ITestS3AFileContextStatistics.java:89)
   	at org.apache.hadoop.fs.FCStatisticsBaseTest.testStatistics(FCStatisticsBaseTest.java:103)
   [INFO] 
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestS3AFileContextStatistics>FCStatisticsBaseTest.testStatistics:103->verifyWrittenBytes:89 Mismatch in bytes written expected:<658> but was:<698>


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-882843973


   catching up on this; letting you go through bogdan's comments first.
   
   What was the conclusion on the ITestS3AFileContextStatistics failure? Side effect of the padding?


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-882843973


   catching up on this; letting you go through bogdan's comments first.
   
   What was the conclusion on the ITestS3AFileContextStatistics failure? Side effect of the padding?


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879811892


   Bad news I'm afraid: I've got an extra bit of work for this patch.
   
   The encryption option and key must be included in delegation tokens, so that encryption setting I have on my client is picked up and propagated to all workers in a launched job. 
   
   We already do this for all the service side encryption options, by creating and serializing a EncryptionSecrets instance in the token.
   
   This class already supports passing S3-CSE information, so what is needed is to hook this up with CSE as well as SSE call
   
   - [ ] setEncryptionSecrets in FileSystem.initialize needs to build the secrets from the client side options, if active
   - [ ] bindAWSClient needs to set client side encryption options from any DT.
   
   you may want to add code in EncryptionSecretOperations to help - EncryptionSecrets MUST NOT use any AWS SDK call to avoid classloading issues.
   
   There's makes me think that the CSE binding/setup code needs to be merged with the SSE stuff a bit more
   # The encryption secrets built up in S3AFS.initialize() MUST use the client side
   secrets if set, so they are picked up by DTs
   # FileSystem.setEncryptionSecrets() should set the isCSEEnabled flag.
   # CSE-only ITest to call S3AFS.getEncryptionSecrets() & verify that all is good
   # See if ITestSessionDelegationInFileystem can pick up and propagate CSE options.
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841789913


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 10 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 51s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  7s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  17m  0s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt) |  hadoop-tools/hadoop-aws: The patch generated 6 new + 38 unchanged - 0 fixed = 44 total (was 38)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 13s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m 11s | [/new-spotbugs-hadoop-tools_hadoop-aws.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/new-spotbugs-hadoop-tools_hadoop-aws.html) |  hadoop-tools/hadoop-aws generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  16m 18s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  7s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 40s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-aws |
   |  |  org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled isn't final and can't be protected from malicious code  In S3AFileSystem.java:be protected from malicious code  In S3AFileSystem.java |
   |  |  Write to static field org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:[line 531] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5935ac359db6 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3b9bf5810138c52b75f5df50c5443723b4ec9f97 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/testReport/ |
   | Max. process+thread count | 515 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r578586956



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionCSESymmetric.java
##########
@@ -0,0 +1,86 @@
+package org.apache.hadoop.fs.s3a;

Review comment:
       asf-license 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672610832



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
##########
@@ -242,6 +242,7 @@ protected void logTimePerIOP(String operation,
 
   @Test
   public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+    skipIfClientSideEncryption();

Review comment:
       ok

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -43,9 +52,12 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       losing battle here. I've got it wrong in some of the audit work and then it complicates backporting

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       good point. Looking at source there's no indication this is bad news if other uploads are in progress, and it would simplify code.
   
   why not set it?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be

Review comment:
       we've not done anything with client side keys. probably relevant on random IO where we do many GET's across a single input stream, so could cache it there.




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] bogthe commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
bogthe commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-843269836


   Did some digging and found out some more context: 
   - For CSE `uploadPart` has to have parts [uploaded sequentially](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/AmazonS3EncryptionClient.html#uploadPart-com.amazonaws.services.s3.model.UploadPartRequest-);
   - There was a request to support them parallel but it [doesn't look like they're going to be added to V1](https://github.com/aws/aws-sdk-java/issues/2248) and [it's on the feature request for V2](https://github.com/aws/aws-sdk-java-v2/projects/1)


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-866239357


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 49s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 12 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  12m 58s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  27m  4s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  29m 16s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |  23m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   3m 56s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 35s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 31s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 47s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 40s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m  4s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 21s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |  20m 21s | [/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1977 unchanged - 0 fixed = 1978 total (was 1977)  |
   | +1 :green_heart: |  compile  |  18m 17s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |  18m 17s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 1853 unchanged - 0 fixed = 1854 total (was 1853)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 50s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 7 new + 52 unchanged - 0 fixed = 59 total (was 52)  |
   | +1 :green_heart: |  mvnsite  |   2m 37s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javadoc  |   0m 48s | [/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 63 unchanged - 0 fixed = 64 total (was 63)  |
   | +1 :green_heart: |  spotbugs  |   4m 11s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 53s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m  3s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 18s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  0s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 217m 49s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 383cb995b907 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c1ccf2c4419e88f086ccce1aa754fff2f1823060 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-886762551


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 36s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 21 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  12m 46s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m  9s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 33s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |  21m  5s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   3m 47s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 38s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 30s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  3s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  17m 29s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  17m 55s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 41s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  23m  4s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |  23m  4s | [/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/11/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1908 unchanged - 0 fixed = 1909 total (was 1908)  |
   | +1 :green_heart: |  compile  |  19m 55s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |  19m 55s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/11/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 1784 unchanged - 0 fixed = 1785 total (was 1784)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   3m 47s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   2m 37s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 30s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m 26s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 56s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m 45s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 25s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 212m 11s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell markdownlint |
   | uname | Linux 495e149c810a 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 1a053229dd21e9ae44d1af1936d5be6244e6bf1d |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/11/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/11/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841783066






-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-866043189


   Seems like git pull, didn't work for me, just before pushing, still have some conflicts, I'll rebase once more by fetching and applying. 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-884925668


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 35s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 21 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  11m 27s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 30s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  21m 13s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |  18m 22s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   3m 48s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 35s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 30s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 45s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 33s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 57s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 25s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |  20m 25s | [/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1908 unchanged - 0 fixed = 1909 total (was 1908)  |
   | +1 :green_heart: |  compile  |  18m 28s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |  18m 28s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 1784 unchanged - 0 fixed = 1785 total (was 1784)  |
   | -1 :x: |  blanks  |   0m  0s | [/blanks-tabs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/artifact/out/blanks-tabs.txt) |  The patch 136 line(s) with tabs.  |
   | -0 :warning: |  checkstyle  |   3m 53s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 1 new + 57 unchanged - 0 fixed = 58 total (was 57)  |
   | +1 :green_heart: |  mvnsite  |   2m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 33s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  9s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m 12s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 17s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 52s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 196m 57s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell markdownlint |
   | uname | Linux 3ece512640c8 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 274a30e2bc3c8b76d03ecc4723edf3fec42c1580 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/testReport/ |
   | Max. process+thread count | 2837 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/10/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672613441



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       good point. Looking at source there's no indication this is bad news if other uploads are in progress, and it would simplify code.
   
   why not set it?




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-866239357


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 49s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 12 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  12m 58s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  27m  4s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  29m 16s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |  23m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   3m 56s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 35s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 31s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 47s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 40s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m  4s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 21s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |  20m 21s | [/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1977 unchanged - 0 fixed = 1978 total (was 1977)  |
   | +1 :green_heart: |  compile  |  18m 17s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |  18m 17s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 1853 unchanged - 0 fixed = 1854 total (was 1853)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 50s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 7 new + 52 unchanged - 0 fixed = 59 total (was 52)  |
   | +1 :green_heart: |  mvnsite  |   2m 37s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javadoc  |   0m 48s | [/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 63 unchanged - 0 fixed = 64 total (was 63)  |
   | +1 :green_heart: |  spotbugs  |   4m 11s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 53s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m  3s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 18s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m  0s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 217m 49s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 383cb995b907 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c1ccf2c4419e88f086ccce1aa754fff2f1823060 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-781440565


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 32s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  4s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 24s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +0 :ok: |  spotbugs  |  16m 28s |  |  Both FindBugs and SpotBugs are enabled, using SpotBugs.  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  javac  |   0m 35s | [/diff-compile-javac-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 1 new + 14 unchanged - 0 fixed = 15 total (was 14)  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 1 new + 14 unchanged - 0 fixed = 15 total (was 14)  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 44s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  0s | [/patch-spotbugs-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/patch-spotbugs-hadoop-tools_hadoop-aws.txt) |  hadoop-tools/hadoop-aws cannot run computeBugHistory from spotbugs  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 52s |  |  hadoop-aws in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 32s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/patch-asflicense-problems.txt) |  The patch generated 2 ASF License warnings.  |
   |  |   |  74m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle |
   | uname | Linux 6886d87a5bb7 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e391844e8e4 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/testReport/ |
   | Max. process+thread count | 732 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r668593736



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1682,3 +1682,105 @@ com.amazonaws.SdkClientException: Unable to execute HTTP request:
 
 When this happens, try to set `fs.s3a.connection.request.timeout` to a larger value or disable it
 completely by setting it to `0`.
+
+### <a name="client-side-encryption"></a> S3 Client Side Encryption

Review comment:
       These problems are more of a "what you shouldn't do", so it kind of answers itself in the problem statements. 




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r665558709



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
##########
@@ -524,10 +525,15 @@ public static S3AFileStatus createFileStatus(Path keyPath,
       long blockSize,
       String owner,
       String eTag,
-      String versionId) {
+      String versionId,
+      boolean isCSEEnabled) {

Review comment:
       Would require a few changes, don't know if it's worth it in this patch or should we do kind of a refactor patch separately?




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841783066


   Had merge conflicts so had to force push. 
   Tests: 
   ```
   [ERROR] Tests run: 1430, Failures: 1, Errors: 34, Skipped: 538
   ```
   Scale:
   ```
   [ERROR] Tests run: 151, Failures: 3, Errors: 21, Skipped: 29
   ```
   
   Most errors are MultiPart upload related: 
   ```
   com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   ```
   Simply adding 16(Padding length) to multipart upload block size won't work. The part sizes need to be a multiple of 16, so it has that restriction for CSE. Also, one more thing to note here is that it assumes the last part to be an exception, which makes me believe that multipart upload in CSE has to be sequential(or can we parallel upload the starting parts and then upload the last part?)? So, potentially another constraint while uploading could have performance impacts here apart from the HEAD calls being required while downloading/listing. 
   @steveloughran 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran edited a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran edited a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879811892






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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r669440984



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -43,9 +52,12 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       ah, missed it :/




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879828382


   I'm seeing 
   ```
   2021-07-14 12:54:09,519 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnLegacyCryptoMode(409)) - The S3 Encryption Client is configured to read encrypted data with legacy encryption modes through the CryptoMode setting. If you don't have objects encrypted with these legacy modes, you should disable support for them to enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
   2021-07-14 12:54:09,525 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnRangeGetsEnabled(401)) - The S3 Encryption Client is configured to support range get requests. Range gets do not provide authenticated encryption properties even when used with an authenticated mode (AES-GCM). See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
   
   ```
   I'm assuming this is legit because we want those range requests. We need to document this in encryption.md; in troubleshooting say "yes"


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-887386029


   Merged to trunk! please CP and test against 3.3 and I'll merge in there.


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r668588940



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
##########
@@ -2029,3 +2029,51 @@ signer for all services.
 For a specific service, the service specific signer is looked up first.
 If that is not specified, the common signer is looked up. If this is
 not specified as well, SDK settings are used.
+
+### <a name="cse"></a> Amazon S3 Client Side Encryption
+Amazon S3 Client Side Encryption(CSE), uses `AmazonS3EncryptionClientV2.java
+` AmazonS3 client. The encryption and decryption is done in AWS SDK side. Atm
+, Only KMS CSE method is supported.
+
+What is needed to be set to enable CSE for a bucket?
+- Create an AWS KMS Key ID from AWS console for your bucket, with same region as
+ your bucket.
+- If already created, [view the kms key ID by these steps.](https://docs.aws.amazon.com/kms/latest/developerguide/find-cmk-id-arn.html)
+- Set `fs.s3a.cse.method=KMS`.
+- Set `fs.s3a.cse.kms.keyId=<KMS_KEY_ID>`.
+
+*Note:* If `fs.s3a.cse.method=KMS
+` is set, `fs.s3a.cse.kms.keyId=<KMS_KEY_ID>` property needs to be set for
+ CSE to work.
+
+Limitations of CSE on S3A:
+
+- No multipart uploader API support.
+- Only CSE enabled clients should be using encrypted data.
+- No S3 Select support.
+- Multipart uploads through S3ABlockOutputStream would be serial and partsize
+ would be a multiple of 16 bytes.
+- Performance hit.
+
+All encrypted data via CSE have a padding of 16 bytes. This would cause
+ inconsistencies in content length and thus lead to errors in application
+  that rely on content length before opening a file or calculation of splits on
+   a table and so on. To rectify this inconsistency, 16 bytes would be
+    stripped from content length when:
+
+- CSE is enabled.
+- If a getFileStatus call is made, a check to see if any client side
+     encryption algorithm was used.
+- contentLength >= 16 bytes.

Review comment:
       2nd one is specific to getFileStatus call, in directory listing we won't have object metadata to verify if an encryption algorithm is set or not. 




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-878934344


   Interesting you see those errors since they should be skipped if CSE is enabled, I'll try to see what's wrong.


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-783336386


   Added the "UNENCRYPTED_CONTENT_LENGTH" header in getFileStatus calls and replaced the content length with it if client-side encryption(CSE) is enabled. This helped pass the initial set of tests for all methods of CSE, but after adding a "Directory Listing" test to check the content lengths of files of different sizes, it broke:
   
   ```
   [INFO] Results:
   [INFO]
   [ERROR] Failures:
   [ERROR]   ITestS3AEncryptionCSESymmetric>ITestS3AEncryptionCSE.testDirectoryListingFileLengths:105 [File length isn't same as expected from directory listing]
   Expecting:
     <[16, 17, 271, 4111]>
   to contain exactly in any order:
     <[0, 1, 255, 4095]>
   elements not found:
     <[0, 1, 255, 4095]>
   and elements not expected:
     <[16, 17, 271, 4111]>
   
   [INFO]
   [ERROR] Tests run: 3, Failures: 1, Errors: 0, Skipped: 0
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] bogthe commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
bogthe commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841812391


   > Had merge conflicts so had to force push.
   > Tests:
   > 
   > ```
   > [ERROR] Tests run: 1430, Failures: 1, Errors: 34, Skipped: 538
   > ```
   > 
   > Scale:
   > 
   > ```
   > [ERROR] Tests run: 151, Failures: 3, Errors: 21, Skipped: 29
   > ```
   > 
   > Most errors are MultiPart upload related:
   > 
   > ```
   > com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   > ```
   > 
   > Simply adding 16(Padding length) to multipart upload block size won't work. The part sizes need to be a multiple of 16, so it has that restriction for CSE. Also, one more thing to note here is that it assumes the last part to be an exception, which makes me believe that multipart upload in CSE has to be sequential(or can we parallel upload the starting parts and then upload the last part?)? So, potentially another constraint while uploading could have performance impacts here apart from the HEAD calls being required while downloading/listing.
   > @steveloughran
   
   Hi @mehakmeet , regarding multipart uploads. The last part is always an exception with regular multi part uploads too! You can do parallel uploads and even upload the last part first and it would still work (for regular multi-part). My assumption is that for multi part uploads with CSE enabled the same functionality holds (except for cipher block size, but the minimum part size for regular multi-part is 5MB = 5 * 1024 * 1024 which is still a multiple of 16 :D ). 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-867746180


   Hey, someone just broke things.Sorry. 
   
   
   > The weird thing is it skips in IDE, but for some reason, I am not able to make it skip in mvn terminal
   
   What I usually miss out there is you need to rebuild hadoop-common after changing the tests there; running tests in hadoop-aws pick up the old version.


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-783532925


   we've discussed using that header before -but not gone with because, as you note: LIST operations don't pick up the header. All code which uses the list to determine object length is going to break.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672614095



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be

Review comment:
       we've not done anything with client side keys. probably relevant on random IO where we do many GET's across a single input stream, so could cache it there.




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r673869794



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -619,6 +620,19 @@ clients where S3-CSE has not been enabled.
 - Set `fs.s3a.server-side-encryption-algorithm=CSE-KMS`.
 - Set `fs.s3a.server-side-encryption.key=<KMS_KEY_ID>`.
 
+KMS_KEY_ID:
+
+Identifies the symmetric CMK that encrypts the data key.
+To specify a CMK, use its key ID, key ARN, alias name, or alias ARN. When
+using an alias name, prefix it with "alias/". To specify a CMK in a
+different AWSaccount, you must use the key ARN or alias ARN.
+
+For example:
+- Key ID: 1234abcd-12ab-34cd-56ef-1234567890ab

Review comment:
       can you wrap these IDs & things with backticks so they stay in a fixed font?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -619,6 +620,19 @@ clients where S3-CSE has not been enabled.
 - Set `fs.s3a.server-side-encryption-algorithm=CSE-KMS`.
 - Set `fs.s3a.server-side-encryption.key=<KMS_KEY_ID>`.
 
+KMS_KEY_ID:
+
+Identifies the symmetric CMK that encrypts the data key.
+To specify a CMK, use its key ID, key ARN, alias name, or alias ARN. When
+using an alias name, prefix it with "alias/". To specify a CMK in a
+different AWSaccount, you must use the key ARN or alias ARN.

Review comment:
       add a space between AWS and account

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
 ```
 We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) 
 is required for range gets to work. 
+
+### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK
+
+If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
+able to generate unique data key for encryption. 
+
+```
+Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
+	at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
+	at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
+	at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
+	at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
+	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
+	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
+	... 49 more
+```
+
+Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to
+work. 
+
+### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId
+
+If the value in `fs.s3a.server-side-encryption.key` property, does not exist
+/valid in AWS KMS CMK(Customer managed keys), then this error would be seen.
+
+```
+Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)

Review comment:
       again: split line

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
 ```
 We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) 
 is required for range gets to work. 
+
+### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK
+
+If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
+able to generate unique data key for encryption. 
+
+```
+Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)

Review comment:
       can you break this line down, but do it so that key search strings don't get split, e.g
   
   ```
   Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException:
   You cannot generate a data key with an asymmetric CMK
   (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; 
   Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
   ```
   

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
 ```
 We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) 
 is required for range gets to work. 
+
+### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK
+
+If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
+able to generate unique data key for encryption. 
+
+```
+Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
+	at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
+	at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
+	at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
+	at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
+	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
+	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
+	... 49 more
+```
+
+Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to
+work. 
+
+### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId
+
+If the value in `fs.s3a.server-side-encryption.key` property, does not exist
+/valid in AWS KMS CMK(Customer managed keys), then this error would be seen.
+
+```
+Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
+	at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
+	at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
+	at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
+	at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
+	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
+	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
+	... 49 more
+```
+
+Check if `fs.s3a.server-side-encryption.key` is set correctly and matches the
+same on AWS console.
+
+### com.amazonaws.services.kms.model.AWSKMSException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>
+
+User doesn't have authorisation to the specific AWS KMS Key ID. 

Review comment:
       best to use US_EN spelling to avoid creating complaints about spelling. So "authorization"

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1309,6 +1309,129 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp
 ```
 We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) 
 is required for range gets to work. 
+
+### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK
+
+If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be
+able to generate unique data key for encryption. 
+
+```
+Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
+	at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
+	at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
+	at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
+	at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
+	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
+	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
+	... 49 more
+```
+
+Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to
+work. 
+
+### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId
+
+If the value in `fs.s3a.server-side-encryption.key` property, does not exist
+/valid in AWS KMS CMK(Customer managed keys), then this error would be seen.
+
+```
+Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
+	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
+	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
+	at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190)
+	at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179)
+	at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482)
+	at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160)
+	at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156)
+	at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792)
+	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789)
+	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440)
+	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)
+	... 49 more
+```
+
+Check if `fs.s3a.server-side-encryption.key` is set correctly and matches the
+same on AWS console.
+
+### com.amazonaws.services.kms.model.AWSKMSException: User: <User_ARN> is not authorized to perform : kms :GenerateDataKey on resource: <KEY_ID>
+
+User doesn't have authorisation to the specific AWS KMS Key ID. 
+```
+Caused by: com.amazonaws.services.kms.model.AWSKMSException: User: arn:aws

Review comment:
       split line again




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r669093714



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
##########
@@ -2029,3 +2029,51 @@ signer for all services.
 For a specific service, the service specific signer is looked up first.
 If that is not specified, the common signer is looked up. If this is
 not specified as well, SDK settings are used.
+
+### <a name="cse"></a> Amazon S3 Client Side Encryption
+Amazon S3 Client Side Encryption(CSE), uses `AmazonS3EncryptionClientV2.java
+` AmazonS3 client. The encryption and decryption is done in AWS SDK side. Atm

Review comment:
       "atm?"
   
   Recommend an intro, features and limitations section:
   
   
   A key reason this feature (HADOOP-13887) has been unavailable for a long time is that the AWS S3 client pads uploaded objects with a 16 byte footer. This meant that files were shorter when being read that when are listed them through any of the list API calls/getFileStatus(). Which broke many applications, including anything seeking near the end of a file to read a footer, as ORC and Parquet do.
   
   There is now a workaround: compensate for the footer in listings when CSE is enabled.
   
   * when listing files and directories, 16 bytes are subtracted from the length of all non-empty objects.
   * directory markers MAY be longer than 0 bytes long.
   
   This "appears" to work; secondly it does in the testing to date. However, the length of files when listed through the S3A client is now going to be shorter than the length of files listed with other clients -including S3A clients where S3-CSE has not been enabled.
   
   features
   * Supports client side encryption with keys managed in AWS KMS
   * encryption settings propagated into jobs through any issued delegation tokens
   * encryption information stored as headers in the uploaded object
   
   Limitations
   * performance will be reduced. All encrypt/decrypt is now being done on the client.
   * writing files may be slower, as only a single block can be encrypted and uploaded at a time
   * Multipart Uploader API disabled
   
   
   
   
   

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
##########
@@ -2029,3 +2029,51 @@ signer for all services.
 For a specific service, the service specific signer is looked up first.
 If that is not specified, the common signer is looked up. If this is
 not specified as well, SDK settings are used.
+
+### <a name="cse"></a> Amazon S3 Client Side Encryption

Review comment:
       move to encryption.md

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -112,15 +124,73 @@ public AmazonS3 createS3Client(
     }
 
     try {
-      return buildAmazonS3Client(
-          awsConf,
-          parameters);
+      if (conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) == null) {
+        return buildAmazonS3Client(
+            awsConf,
+            parameters);
+      } else {
+        return buildAmazonS3EncryptionClient(
+            awsConf,
+            parameters);
+      }
     } catch (SdkClientException e) {
       // SDK refused to build.
       throw translateException("creating AWS S3 client", uri.toString(), e);
     }
   }
 
+  /**
+   * Create an {@link AmazonS3} client of type
+   * {@link AmazonS3EncryptionV2} if CSE is enabled.
+   *
+   * @param awsConf    AWS configuration.
+   * @param parameters parameters
+   *
+   * @return new AmazonS3 client.
+   */
+  protected AmazonS3 buildAmazonS3EncryptionClient(

Review comment:
       InconsistentS3Client doesn't do this; I can see tests skip it. Should we have that client throw some Unsupported Exception here

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
##########
@@ -179,11 +178,9 @@ public void setup() throws Exception {
     if (useInconsistentClient()) {
       AmazonS3 client = getFileSystem()
           .getAmazonS3ClientForTesting("fault injection");
-      Assert.assertTrue(
-          "AWS client is not inconsistent, even though the test requirees it "
-          + client,
-          client instanceof InconsistentAmazonS3Client);
-      inconsistentClient = (InconsistentAmazonS3Client) client;
+      if(client instanceof InconsistentAmazonS3Client) {

Review comment:
       nit: add space after if

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -43,9 +52,12 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       can you move up to L44. Yes, it its a PITA.

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
##########
@@ -242,6 +242,7 @@ protected void logTimePerIOP(String operation,
 
   @Test
   public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+    skipIfClientSideEncryption();

Review comment:
       why are we turning this off ? Performance? seek() perf?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1682,3 +1682,105 @@ com.amazonaws.SdkClientException: Unable to execute HTTP request:
 
 When this happens, try to set `fs.s3a.connection.request.timeout` to a larger value or disable it
 completely by setting it to `0`.
+
+### <a name="client-side-encryption"></a> S3 Client Side Encryption

Review comment:
       The troubleshooting documents need to assume that the person reading them or googling for them has just seen a specific error message and is trying to work out what's wrong/what to do.
   
   This means the phrasing here needs to be tweaked slightly
   
   * move these into the "encryption" section (ahead of those which clearly shouldn't be there :)
   * Add a ### subtitle for every stack trace
   * with the error text the title of it -so it's the first thing people see
   * interpretation after the stack trace
   * and ideally a recommendation, if there's a simple solution (including "don't")
   
   ###  "Instruction file not found for S3 object"
   
   Reading a file fails when the client is configured with S3-CSE
   
   
   (insert <stack trace>)
     
   The file is not encrypted with S3-CSE.
   




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-880716535


   Just realized I need to slightly tweak `ITestSessionDelegationInFileystem.java` for it to run CSE-KMS or SSE-KMS. Will add one more commit for that. 


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r578586956



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionCSESymmetric.java
##########
@@ -0,0 +1,86 @@
+package org.apache.hadoop.fs.s3a;

Review comment:
       asf-license 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r578587149



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ACSEMaterialProviderConfig.java
##########
@@ -0,0 +1,14 @@
+package org.apache.hadoop.fs.s3a;

Review comment:
       asf-license




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879058793


   > Interesting you see those errors since they should be skipped if CSE is enabled, I'll try to see what's wrong.
   
   No that was without enabling CSE as said above. 
   
   Now I am trying to run the tests after enabling CSE. And the first issue I see is 
   `[ERROR] Tests run: 17, Failures: 0, Errors: 17, Skipped: 0, Time elapsed: 22.59 s <<< FAILURE! - in org.apache.hadoop.fs.contract.s3a.ITestS3AContractMultipartUploader
   [ERROR] testMultipartUpload(org.apache.hadoop.fs.contract.s3a.ITestS3AContractMultipartUploader)  Time elapsed: 6.382 s  <<< ERROR!
   java.lang.UnsupportedOperationException: Multi-part uploader not supported for Client side encryption.
   	at org.apache.hadoop.fs.s3a.S3AFileSystem.createMultipartUploader(S3AFileSystem.java:5396)
   	at org.apache.hadoop.fs.s3a.S3AFileSystem.createMultipartUploader(S3AFileSystem.java:249)
   	at org.apache.hadoop.fs.contract.AbstractContractMultipartUploaderTest.setup(AbstractContractMultipartUploaderTest.java:92)
   	at org.apache.hadoop.fs.contract.s3a.ITestS3AContractMultipartUploader.setup(ITestS3AContractMultipartUploader.java:115)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   `
   
   This should have been skipped rather that failed. 
   
   I will update on how the rest goes. 


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841789913


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 10 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 51s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  7s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  17m  0s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 19s | [/results-checkstyle-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt) |  hadoop-tools/hadoop-aws: The patch generated 6 new + 38 unchanged - 0 fixed = 44 total (was 38)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 13s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m 11s | [/new-spotbugs-hadoop-tools_hadoop-aws.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/new-spotbugs-hadoop-tools_hadoop-aws.html) |  hadoop-tools/hadoop-aws generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  16m 18s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  7s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 40s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-aws |
   |  |  org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled isn't final and can't be protected from malicious code  In S3AFileSystem.java:be protected from malicious code  In S3AFileSystem.java |
   |  |  Write to static field org.apache.hadoop.fs.s3a.S3AFileSystem.isCSEEnabled from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:from instance method org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(URI, Configuration)  At S3AFileSystem.java:[line 531] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5935ac359db6 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3b9bf5810138c52b75f5df50c5443723b4ec9f97 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/testReport/ |
   | Max. process+thread count | 515 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672037371



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       I was thinking about this myself but thought shouldn't we keep the non-CSE behavior as it was, that is pre this patch, we weren't setting the lastPart=true for non-CSE multi-part uploads, and what would be the implications with multiple active blocks as well? But it does make sense to have it true since it would always be the last part uploaded... @steveloughran thoughts??




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r668460078



##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
##########
@@ -2029,3 +2029,51 @@ signer for all services.
 For a specific service, the service specific signer is looked up first.
 If that is not specified, the common signer is looked up. If this is
 not specified as well, SDK settings are used.
+
+### <a name="cse"></a> Amazon S3 Client Side Encryption
+Amazon S3 Client Side Encryption(CSE), uses `AmazonS3EncryptionClientV2.java
+` AmazonS3 client. The encryption and decryption is done in AWS SDK side. Atm
+, Only KMS CSE method is supported.
+
+What is needed to be set to enable CSE for a bucket?
+- Create an AWS KMS Key ID from AWS console for your bucket, with same region as
+ your bucket.
+- If already created, [view the kms key ID by these steps.](https://docs.aws.amazon.com/kms/latest/developerguide/find-cmk-id-arn.html)
+- Set `fs.s3a.cse.method=KMS`.
+- Set `fs.s3a.cse.kms.keyId=<KMS_KEY_ID>`.
+
+*Note:* If `fs.s3a.cse.method=KMS
+` is set, `fs.s3a.cse.kms.keyId=<KMS_KEY_ID>` property needs to be set for
+ CSE to work.
+
+Limitations of CSE on S3A:
+
+- No multipart uploader API support.
+- Only CSE enabled clients should be using encrypted data.
+- No S3 Select support.
+- Multipart uploads through S3ABlockOutputStream would be serial and partsize
+ would be a multiple of 16 bytes.
+- Performance hit.
+
+All encrypted data via CSE have a padding of 16 bytes. This would cause
+ inconsistencies in content length and thus lead to errors in application
+  that rely on content length before opening a file or calculation of splits on
+   a table and so on. To rectify this inconsistency, 16 bytes would be
+    stripped from content length when:
+
+- CSE is enabled.
+- If a getFileStatus call is made, a check to see if any client side
+     encryption algorithm was used.
+- contentLength >= 16 bytes.

Review comment:
       When all three conditions are met right?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
##########
@@ -1682,3 +1682,105 @@ com.amazonaws.SdkClientException: Unable to execute HTTP request:
 
 When this happens, try to set `fs.s3a.connection.request.timeout` to a larger value or disable it
 completely by setting it to `0`.
+
+### <a name="client-side-encryption"></a> S3 Client Side Encryption

Review comment:
       Shouldn't we describe solutions to these problems as well?




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-781440565


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 32s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  4s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 24s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +0 :ok: |  spotbugs  |  16m 28s |  |  Both FindBugs and SpotBugs are enabled, using SpotBugs.  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  javac  |   0m 35s | [/diff-compile-javac-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  hadoop-tools_hadoop-aws-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 1 new + 14 unchanged - 0 fixed = 15 total (was 14)  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/diff-compile-javac-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 1 new + 14 unchanged - 0 fixed = 15 total (was 14)  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 44s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  0s | [/patch-spotbugs-hadoop-tools_hadoop-aws.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/patch-spotbugs-hadoop-tools_hadoop-aws.txt) |  hadoop-tools/hadoop-aws cannot run computeBugHistory from spotbugs  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 52s |  |  hadoop-aws in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 32s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/patch-asflicense-problems.txt) |  The patch generated 2 ASF License warnings.  |
   |  |   |  74m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle |
   | uname | Linux 6886d87a5bb7 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e391844e8e4 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/testReport/ |
   | Max. process+thread count | 732 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r673873663



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
##########
@@ -437,14 +442,14 @@ private Constants() {
 
   /**
    * Used to specify which AWS KMS key to use if
-   * {@link #SERVER_SIDE_ENCRYPTION_ALGORITHM} is
+   * {@link #S3_ENCRYPTION_ALGORITHM} is
    * {@code SSE-KMS} (will default to aws/s3
    * master key if left blank).
    * With with {@code SSE_C}, the base-64 encoded AES 256 key.
    * May be set within a JCEKS file.
    * Value: "{@value}".
    */
-  public static final String SERVER_SIDE_ENCRYPTION_KEY =
+  public static final String S3_ENCRYPTION_KEY =

Review comment:
       Constants.java is tagged public/evolving; we can'd remove constants.
   
   can you reinstate the original constants. you could just make it an @Deprecated reference to the new one, but it MUST still be there

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
##########
@@ -524,10 +525,15 @@ public static S3AFileStatus createFileStatus(Path keyPath,
       long blockSize,
       String owner,
       String eTag,
-      String versionId) {
+      String versionId,
+      boolean isCSEEnabled) {

Review comment:
       if you want to do it....




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-883485574


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 27 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  12m 29s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  22m 39s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 31s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |  19m 12s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   4m  1s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 23s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 30s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   2m 15s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  17m  1s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  17m 22s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m  3s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |  22m  3s | [/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1908 unchanged - 0 fixed = 1909 total (was 1908)  |
   | +1 :green_heart: |  compile  |  19m  4s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |  19m  4s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 1784 unchanged - 0 fixed = 1785 total (was 1784)  |
   | -1 :x: |  blanks  |   0m  0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/blanks-eol.txt) |  The patch has 27 line(s) that end in blanks. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | -1 :x: |  blanks  |   0m  0s | [/blanks-tabs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/blanks-tabs.txt) |  The patch 136 line(s) with tabs.  |
   | -0 :warning: |  checkstyle  |   3m 55s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 6 new + 114 unchanged - 5 fixed = 120 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   2m 21s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 30s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javadoc  |   0m 38s | [/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/results-javadoc-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 2 new + 63 unchanged - 0 fixed = 65 total (was 63)  |
   | +1 :green_heart: |  spotbugs  |   4m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  17m 19s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  16m 57s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 25s |  |  hadoop-aws in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 49s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 205m 53s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2706 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell markdownlint |
   | uname | Linux 96056f4ad5cd 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e3d5922cc9af942132d1f20fa1eb03a487b84c13 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/testReport/ |
   | Max. process+thread count | 2250 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2706/9/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] bogthe commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
bogthe commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r671136384



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       Why not set it always to `true` since it's the last part? i.e. `uploadCurrentBlock(true)`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.

Review comment:
       nit: Maybe this needs to be made a bit clearer that data is always encrypted inside of S3 and decrypted when it's being retrieved? 

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -356,6 +359,11 @@
   private AuditManagerS3A auditManager =
       AuditIntegration.stubAuditManager();
 
+  /**
+   * Is this S3AFS instance using S3 client side encryption?

Review comment:
       nit*: `S3A FS`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the

Review comment:
       nit: `Working with SSE-C data hard` -> `Working with SSE-C data is harder` or `Working with SSE-C data is more difficult`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be
+used multiple times.
+
+S3 Client side encryption (CSE-KMS) is an experimental feature added in July
+2021.
+
+This encrypts the data on the client, before transmitting to S3, where it is
+stored encrypted. The data is unencrypted after downloading when it is being
+read back.
+
+in CSE-KMS, the ID of an AWS-KMS key is provided to the S3A client; 
+the client communicates with AWS-KMS to request a new encryption key, which
+KMS returns along with the same key encrypted with the KMS key.
+The S3 client encrypts the payload *and* attaches the KMS-encrypted version
+of the key as a header to the object.
+
+When downloading data, this header is extracted, passed to AWS KMS, and,
+if the client has the appropriate permissions, the symmetric key
+retrieved and returned.

Review comment:
       nit: `retrieved and returned.` -> `retrieved.` ?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -419,7 +472,81 @@ the data, so guaranteeing that access to thee data can be read by everyone
 granted access to that key, and nobody without access to it.
 
 
-###<a name="changing-encryption"></a> Use rename() to encrypt files with new keys
+### <a name="fattr"></a> Using `hadoop fs -getfattr` to view encryption information.
+
+The S3A client retrieves all HTTP headers from an object and returns
+them in the "XAttr" list of attributed, prefixed with `header.`.
+This makes them retrievable in the `getXAttr()` API calls, which
+is available on the command line through the `hadoop fs -getfattr -d` command.
+
+This makes viewing the encryption headers of a file straightforward
+
+Here is an example of the operation invoked on a file where the client is using CSE-KMS:
+```
+bin/hadoop fs -getfattr -d s3a://test-london/file2
+
+2021-07-14 12:59:01,554 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnLegacyCryptoMode(409)) - The S3 Encryption Client is configured to read encrypted data with legacy encryption modes through the CryptoMode setting. If you don't have objects encrypted with these legacy modes, you should disable support for them to enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
+2021-07-14 12:59:01,558 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnRangeGetsEnabled(401)) - The S3 Encryption Client is configured to support range get requests. Range gets do not provide authenticated encryption properties even when used with an authenticated mode (AES-GCM). See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
+# file: s3a://test-london/file2
+header.Content-Length="0"
+header.Content-Type="application/octet-stream"
+header.ETag="63b3f4bd6758712c98f1be86afad095a"
+header.Last-Modified="Wed Jul 14 12:56:06 BST 2021"
+header.x-amz-cek-alg="AES/GCM/NoPadding"
+header.x-amz-iv="ZfrgtxvcR41yNVkw"
+header.x-amz-key-v2="AQIDAHjZrfEIkNG24/xy/pqPPLcLJulg+O5pLgbag/745OH4VQFuiRnSS5sOfqXJyIa4FOvNAAAAfjB8BgkqhkiG9w0BBwagbzBtAgEAMGgGCSqGSIb3DQEHATAeBglghkgBZQMEAS4wEQQM7b5GZzD4eAGTC6gaAgEQgDte9Et/dhR7LAMU/NaPUZSgXWN5pRnM4hGHBiRNEVrDM+7+iotSTKxFco+VdBAzwFZfHjn0DOoSZEukNw=="
+header.x-amz-matdesc="{"aws:x-amz-cek-alg":"AES/GCM/NoPadding"}"
+header.x-amz-server-side-encryption="AES256"
+header.x-amz-tag-len="128"
+header.x-amz-unencrypted-content-length="0"
+header.x-amz-version-id="zXccFCB9eICszFgqv_paG1pzaUKY09Xa"
+header.x-amz-wrap-alg="kms+context"
+```
+
+Analysis
+
+1. the WARN commands are the AWS SDK warning that because the S3A client uses an encryption algorithm which seek() requires,
+the SDK considers it less secure than the most recent algorithm(s). Ignore.
+   
+* `header.x-amz-server-side-encryption="AES256"` : the file has been encrypted with S3-SSE. This is set up as the S3 default encryption,
+so even when CSE is enabled, the data is doubly encrypted.
+* `header.x-amz-cek-alg="AES/GCM/NoPadding`: client-side encrypted with the `"AES/GCM/NoPadding` algorithm.
+* `header.x-amz-iv="ZfrgtxvcR41yNVkw"`:
+* `header.x-amz-key-v2="AQIDAHjZrfEIkNG24/xy/pqPPLcLJulg+O5pLgbag/745OH4VQFuiRnSS5sOfqXJyIa4FOvNAAAAfjB8BgkqhkiG9w0BBwagbzBtAgEAMGgGCSqGSIb3DQEHATAeBglghkgBZQMEAS4wEQQM7b5GZzD4eAGTC6gaAgEQgDte9Et/dhR7LAMU/NaPUZSgXWN5pRnM4hGHBiRNEVrDM+7+iotSTKxFco+VdBAzwFZfHjn0DOoSZEukNw=="`:
+* `header.x-amz-unencrypted-content-length="0"`: this is the length of the unencrypted data. The S3A client *DOES NOT* use this header;
+* `header.x-amz-wrap-alg="kms+context"`: the algorithm used to encrypt the CSE key
+  it always removes 16 bytes from non-empty files when declaring the length.
+* `header.x-amz-version-id="zXccFCB9eICszFgqv_paG1pzaUKY09Xa"`: the bucket is versioned; this is the version ID.
+
+And a directory encrypted with S3-SSE only
+
+```
+bin/hadoop fs -getfattr -d s3a://test-london/user/stevel/target/test/data/sOCOsNgEjv
+
+# file: s3a://test-london/user/stevel/target/test/data/sOCOsNgEjv
+header.Content-Length="0"
+header.Content-Type="application/x-directory"
+header.ETag="d41d8cd98f00b204e9800998ecf8427e"
+header.Last-Modified="Tue Jul 13 20:12:07 BST 2021"
+header.x-amz-server-side-encryption="AES256"
+header.x-amz-version-id="KcDOVmznIagWx3gP1HlDqcZvm1mFWZ2a"
+```
+
+A file with no-encryption (on a bucket without versioning but with intelligent tiering)

Review comment:
       nit: missing `:`

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+  private static final int SMALL_FILE_SIZE = 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFileSize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    maybeSkipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(SMALL_FILE_SIZE, 'a', 'z');
+    writeDataset(fs, src, data, data.length, SMALL_FILE_SIZE,
+        true, false);
+
+    ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    maybeSkipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths aren't the same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while (listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    maybeSkipTest();
+    assume("Scale test disabled: to enable set property " +
+        KEY_SCALE_TESTS_ENABLED, getTestPropertyBool(
+        getConfiguration(),
+        KEY_SCALE_TESTS_ENABLED,
+        DEFAULT_SCALE_TESTS_ENABLED));
+    S3AFileSystem fs = getFileSystem();
+    Path filePath = path(getMethodName());
+    byte[] fileContent = dataset(BIG_FILE_SIZE, 'a', 26);
+    int offsetSeek = fileContent[BIG_FILE_SIZE - 4];
+
+    // PUT a 15MB file using CSE to force multipart in CSE.
+    createFile(fs, filePath, true, fileContent);
+    LOG.info("Multi-part upload successful...");
+
+    try (FSDataInputStream in = fs.open(filePath)) {
+      // Verify random IO.
+      in.seek(BIG_FILE_SIZE - 4);
+      assertEquals("Byte at a specific position not equal to actual byte",
+          offsetSeek, in.read());
+      in.seek(0);
+      assertEquals("Byte at a specific position not equal to actual byte",
+          'a', in.read());
+
+      // Verify seek-read between two multipart blocks.
+      in.seek(MULTIPART_MIN_SIZE - 1);
+      int byteBeforeBlockEnd = fileContent[MULTIPART_MIN_SIZE];
+      assertEquals("Byte before multipart block end mismatch",
+          byteBeforeBlockEnd - 1, in.read());
+      assertEquals("Byte at multipart end mismatch",
+          byteBeforeBlockEnd, in.read());
+      assertEquals("Byte after multipart end mismatch",
+          byteBeforeBlockEnd + 1, in.read());
+
+      // Verify end of file seek read.
+      in.seek(BIG_FILE_SIZE + 1);
+      assertEquals("Byte at eof mismatch",
+          -1, in.read());
+
+      // Verify full read.
+      in.readFully(0, fileContent);
+      verifyFileContents(fs, filePath, fileContent);
+    }
+  }
+
+  /**
+   * Testing how unencrypted and encrypted data behaves when read through
+   * CSE enabled and disabled FS respectively.
+   */
+  @Test
+  public void testEncryptionEnabledAndDisabledFS() throws Exception {

Review comment:
       This is awesome

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
##########
@@ -524,10 +525,15 @@ public static S3AFileStatus createFileStatus(Path keyPath,
       long blockSize,
       String owner,
       String eTag,
-      String versionId) {
+      String versionId,
+      boolean isCSEEnabled) {

Review comment:
       is it worth creating a ticket in JIRA now so we don't forget about it?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be
+used multiple times.
+
+S3 Client side encryption (CSE-KMS) is an experimental feature added in July
+2021.
+
+This encrypts the data on the client, before transmitting to S3, where it is
+stored encrypted. The data is unencrypted after downloading when it is being
+read back.
+
+in CSE-KMS, the ID of an AWS-KMS key is provided to the S3A client; 

Review comment:
       nit: Capital `i` 

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -419,7 +472,81 @@ the data, so guaranteeing that access to thee data can be read by everyone
 granted access to that key, and nobody without access to it.
 
 
-###<a name="changing-encryption"></a> Use rename() to encrypt files with new keys
+### <a name="fattr"></a> Using `hadoop fs -getfattr` to view encryption information.
+
+The S3A client retrieves all HTTP headers from an object and returns
+them in the "XAttr" list of attributed, prefixed with `header.`.
+This makes them retrievable in the `getXAttr()` API calls, which
+is available on the command line through the `hadoop fs -getfattr -d` command.
+
+This makes viewing the encryption headers of a file straightforward
+
+Here is an example of the operation invoked on a file where the client is using CSE-KMS:
+```
+bin/hadoop fs -getfattr -d s3a://test-london/file2
+
+2021-07-14 12:59:01,554 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnLegacyCryptoMode(409)) - The S3 Encryption Client is configured to read encrypted data with legacy encryption modes through the CryptoMode setting. If you don't have objects encrypted with these legacy modes, you should disable support for them to enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
+2021-07-14 12:59:01,558 [main] WARN  s3.AmazonS3EncryptionClientV2 (AmazonS3EncryptionClientV2.java:warnOnRangeGetsEnabled(401)) - The S3 Encryption Client is configured to support range get requests. Range gets do not provide authenticated encryption properties even when used with an authenticated mode (AES-GCM). See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html
+# file: s3a://test-london/file2
+header.Content-Length="0"
+header.Content-Type="application/octet-stream"
+header.ETag="63b3f4bd6758712c98f1be86afad095a"
+header.Last-Modified="Wed Jul 14 12:56:06 BST 2021"
+header.x-amz-cek-alg="AES/GCM/NoPadding"
+header.x-amz-iv="ZfrgtxvcR41yNVkw"
+header.x-amz-key-v2="AQIDAHjZrfEIkNG24/xy/pqPPLcLJulg+O5pLgbag/745OH4VQFuiRnSS5sOfqXJyIa4FOvNAAAAfjB8BgkqhkiG9w0BBwagbzBtAgEAMGgGCSqGSIb3DQEHATAeBglghkgBZQMEAS4wEQQM7b5GZzD4eAGTC6gaAgEQgDte9Et/dhR7LAMU/NaPUZSgXWN5pRnM4hGHBiRNEVrDM+7+iotSTKxFco+VdBAzwFZfHjn0DOoSZEukNw=="
+header.x-amz-matdesc="{"aws:x-amz-cek-alg":"AES/GCM/NoPadding"}"
+header.x-amz-server-side-encryption="AES256"
+header.x-amz-tag-len="128"
+header.x-amz-unencrypted-content-length="0"
+header.x-amz-version-id="zXccFCB9eICszFgqv_paG1pzaUKY09Xa"
+header.x-amz-wrap-alg="kms+context"
+```
+
+Analysis
+
+1. the WARN commands are the AWS SDK warning that because the S3A client uses an encryption algorithm which seek() requires,
+the SDK considers it less secure than the most recent algorithm(s). Ignore.
+   
+* `header.x-amz-server-side-encryption="AES256"` : the file has been encrypted with S3-SSE. This is set up as the S3 default encryption,
+so even when CSE is enabled, the data is doubly encrypted.
+* `header.x-amz-cek-alg="AES/GCM/NoPadding`: client-side encrypted with the `"AES/GCM/NoPadding` algorithm.
+* `header.x-amz-iv="ZfrgtxvcR41yNVkw"`:
+* `header.x-amz-key-v2="AQIDAHjZrfEIkNG24/xy/pqPPLcLJulg+O5pLgbag/745OH4VQFuiRnSS5sOfqXJyIa4FOvNAAAAfjB8BgkqhkiG9w0BBwagbzBtAgEAMGgGCSqGSIb3DQEHATAeBglghkgBZQMEAS4wEQQM7b5GZzD4eAGTC6gaAgEQgDte9Et/dhR7LAMU/NaPUZSgXWN5pRnM4hGHBiRNEVrDM+7+iotSTKxFco+VdBAzwFZfHjn0DOoSZEukNw=="`:
+* `header.x-amz-unencrypted-content-length="0"`: this is the length of the unencrypted data. The S3A client *DOES NOT* use this header;
+* `header.x-amz-wrap-alg="kms+context"`: the algorithm used to encrypt the CSE key
+  it always removes 16 bytes from non-empty files when declaring the length.
+* `header.x-amz-version-id="zXccFCB9eICszFgqv_paG1pzaUKY09Xa"`: the bucket is versioned; this is the version ID.
+
+And a directory encrypted with S3-SSE only

Review comment:
       nit: missing `:`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -435,6 +562,78 @@ for reading as for writing, and you must supply that key for reading. There
 you need to copy one bucket to a different bucket, one with a different key.
 Use `distCp`for this, with per-bucket encryption policies.
 
+## <a name="cse"></a> Amazon S3 Client Side Encryption
+
+### Introduction
+Amazon S3 Client Side Encryption(S3-CSE), is used to encrypt data on the
+client-side and then transmit it over to S3 storage. The same encrypted data
+is then transmitted over to client while reading and then
+decrypted on the client-side. 
+
+S3-CSE, uses `AmazonS3EncryptionClientV2.java`  as the AmazonS3 client. The
+encryption and decryption is done by AWS SDK. As of July 2021, Only CSE-KMS
+method is supported. 
+
+A key reason this feature (HADOOP-13887) has been unavailable for a long time
+is that the AWS S3 client pads uploaded objects with a 16 byte footer. This
+meant that files were shorter when being read than when are listed them
+through any of the list API calls/getFileStatus(). Which broke many
+applications, including anything seeking near the end of a file to read a
+footer, as ORC and Parquet do.
+
+There is now a workaround: compensate for the footer in listings when CSE is enabled.
+
+- When listing files and directories, 16 bytes are subtracted from the length
+of all non-empty objects( greater than or equal to 16 bytes). 
+- Directory markers MAY be longer than 0 bytes long.
+
+This "appears" to work; secondly it does in the testing as of July 2021. However
+, the length of files when listed through the S3A client is now going to be
+shorter than the length of files listed with other clients -including S3A
+clients where S3-CSE has not been enabled.
+
+### Features
+
+- Supports client side encryption with keys managed in AWS KMS.
+- encryption settings propagated into jobs through any issued delegation tokens.
+- encryption information stored as headers in the uploaded object.
+
+### Limitations
+
+- Performance will be reduced. All encrypt/decrypt is now being done on the
+ client.
+- Writing files may be slower, as only a single block can be encrypted and
+ uploaded at a time

Review comment:
       nit: missing `.` or `;`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -419,7 +472,81 @@ the data, so guaranteeing that access to thee data can be read by everyone
 granted access to that key, and nobody without access to it.
 
 
-###<a name="changing-encryption"></a> Use rename() to encrypt files with new keys
+### <a name="fattr"></a> Using `hadoop fs -getfattr` to view encryption information.
+
+The S3A client retrieves all HTTP headers from an object and returns
+them in the "XAttr" list of attributed, prefixed with `header.`.
+This makes them retrievable in the `getXAttr()` API calls, which
+is available on the command line through the `hadoop fs -getfattr -d` command.
+
+This makes viewing the encryption headers of a file straightforward

Review comment:
       nit: Missing `.`

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be

Review comment:
       I'm not sure if Hadoop does it but KMS keys can be cached by the client which reduces billing and throttling. If Hadoop supports it might be worth mentioning it here, otherwise it would be nice to add it (not in this PR ofc!).




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r669526447



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -112,15 +124,73 @@ public AmazonS3 createS3Client(
     }
 
     try {
-      return buildAmazonS3Client(
-          awsConf,
-          parameters);
+      if (conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) == null) {
+        return buildAmazonS3Client(
+            awsConf,
+            parameters);
+      } else {
+        return buildAmazonS3EncryptionClient(
+            awsConf,
+            parameters);
+      }
     } catch (SdkClientException e) {
       // SDK refused to build.
       throw translateException("creating AWS S3 client", uri.toString(), e);
     }
   }
 
+  /**
+   * Create an {@link AmazonS3} client of type
+   * {@link AmazonS3EncryptionV2} if CSE is enabled.
+   *
+   * @param awsConf    AWS configuration.
+   * @param parameters parameters
+   *
+   * @return new AmazonS3 client.
+   */
+  protected AmazonS3 buildAmazonS3EncryptionClient(
+      final ClientConfiguration awsConf,
+      final S3ClientCreationParameters parameters){
+
+    AmazonS3 client;
+    AmazonS3EncryptionClientV2Builder builder =
+        new AmazonS3EncryptionClientV2Builder();
+    Configuration conf = getConf();
+
+    //CSE-KMS Method
+    String kmsKeyId = conf.get(CLIENT_SIDE_ENCRYPTION_KMS_KEY_ID);

Review comment:
       Use `S3AUtils.lookupPassword()`

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -418,7 +426,10 @@ public void initialize(URI name, Configuration originalConf)
       invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
       instrumentation = new S3AInstrumentation(uri);
       initializeStatisticsBinding();
-
+      // If CSE method is set then CSE is enabled.
+      isCSEEnabled = conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) != null;

Review comment:
       Use `S3AUtils.lookupPassword()`




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672610832



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
##########
@@ -242,6 +242,7 @@ protected void logTimePerIOP(String operation,
 
   @Test
   public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+    skipIfClientSideEncryption();

Review comment:
       ok




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r693398280



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
##########
@@ -420,7 +420,12 @@ private Constants() {
       "fs.s3a.multipart.purge.age";
   public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400;
 
-  // s3 server-side encryption, see S3AEncryptionMethods for valid options
+  /**
+   * s3 server-side encryption or s3 client side encryption method, see

Review comment:
       having two was too complex. as for a single name and deprecating the other -yes that's a great idea! we haven't shipped yet, so if you can add a patch there it'd be welcome. 
   
   (FWIW, in my local test setups i will stay on the old setting so tests on older branches still work. changing a config name is tricky, even with the deprecation mechanism)




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-867530154


   CC: @mukund-thakur 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r669513771



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
##########
@@ -524,10 +525,15 @@ public static S3AFileStatus createFileStatus(Path keyPath,
       long blockSize,
       String owner,
       String eTag,
-      String versionId) {
+      String versionId,
+      boolean isCSEEnabled) {

Review comment:
       ok, for now keep separate




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r669439506



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
##########
@@ -242,6 +242,7 @@ protected void logTimePerIOP(String operation,
 
   @Test
   public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+    skipIfClientSideEncryption();

Review comment:
       Uses Landsat data that isn't encrypted which causes failures. 




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r578587149



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ACSEMaterialProviderConfig.java
##########
@@ -0,0 +1,14 @@
+package org.apache.hadoop.fs.s3a;

Review comment:
       asf-license




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] mikedias commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mikedias commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r693317652



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
##########
@@ -420,7 +420,12 @@ private Constants() {
       "fs.s3a.multipart.purge.age";
   public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400;
 
-  // s3 server-side encryption, see S3AEncryptionMethods for valid options
+  /**
+   * s3 server-side encryption or s3 client side encryption method, see

Review comment:
       Hi, apologies if this had been discussed before but wouldn't be better to define the client-side encryption in a property called `fs.s3a.client-side-encryption-algorithm`? Or perhaps deprecate the current property and move to just `fs.s3a.encryption-algorithm`?




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672037371



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       I was thinking about this myself but thought shouldn't we keep the non-CSE behavior as it was, that is pre this patch, we weren't setting the lastPart=true for non-CSE multi-part uploads, and what would be the implications with multiple threads as well? But it does make sense to have it true since it would always be the last part uploaded... @steveloughran thoughts??




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r665189654



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -307,29 +311,31 @@ public synchronized void write(byte[] source, int offset, int len)
       // of capacity
       // Trigger an upload then process the remainder.
       LOG.debug("writing more data than block has capacity -triggering upload");
-      uploadCurrentBlock();
+      uploadCurrentBlock(false);
       // tail recursion is mildly expensive, but given buffer sizes must be MB.
       // it's unlikely to recurse very deeply.
       this.write(source, offset + written, len - written);
     } else {
-      if (remainingCapacity == 0) {
+      if (remainingCapacity == 0 && !isCSEEnabled) {
         // the whole buffer is done, trigger an upload
-        uploadCurrentBlock();
+        uploadCurrentBlock(false);
       }
     }
   }
 
   /**
    * Start an asynchronous upload of the current block.
+   * @param isLast true, if part being uploaded is last and client side

Review comment:
       and or 'or'??

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -94,9 +105,81 @@ public AmazonS3 createS3Client(
       awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix());
     }
 
-    return buildAmazonS3Client(
-        awsConf,
-        parameters);
+    if (conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) == null) {
+      return buildAmazonS3Client(
+          awsConf,
+          parameters);
+    } else {
+      return newAmazonS3EncryptionClient(
+          awsConf,
+          parameters);
+    }
+  }
+
+  /**
+   * Create an {@link AmazonS3} client of type
+   * {@link AmazonS3EncryptionV2} if CSE is enabled.
+   *
+   * @param awsConf    AWS configuration.
+   * @param parameters parameters
+   *
+   * @return new AmazonS3 client.
+   */
+  protected AmazonS3 newAmazonS3EncryptionClient(
+      final ClientConfiguration awsConf,
+      final S3ClientCreationParameters parameters){
+
+    AmazonS3 client;
+    AmazonS3EncryptionClientV2Builder builder =
+        new AmazonS3EncryptionClientV2Builder();
+    Configuration conf = getConf();
+
+    //CSE-KMS Method
+    String kmsKeyId = conf.get(CLIENT_SIDE_ENCRYPTION_KMS_KEY_ID);
+    // Check if kmsKeyID is not null
+    Preconditions.checkArgument(kmsKeyId != null, "CSE-KMS method "
+        + "requires KMS key ID. Use " + CLIENT_SIDE_ENCRYPTION_KMS_KEY_ID
+        + " property to set it. ");
+
+    EncryptionMaterialsProvider materialsProvider =
+        new KMSEncryptionMaterialsProvider(kmsKeyId);
+
+    builder.withEncryptionMaterialsProvider(materialsProvider);
+    builder.withCredentials(parameters.getCredentialSet())
+        .withClientConfiguration(awsConf)
+        .withPathStyleAccessEnabled(parameters.isPathStyleAccess());
+
+    // if metrics are not null, then add in the builder.
+    if (parameters.getMetrics() != null) {
+      LOG.debug("Creating Amazon client with AWS metrics");
+      builder.withMetricsCollector(
+          new AwsStatisticsCollector(parameters.getMetrics()));
+    }
+
+    // Create cryptoConfig
+    CryptoConfigurationV2 cryptoConfigurationV2 =
+        new CryptoConfigurationV2(CryptoMode.AuthenticatedEncryption)
+            .withRangeGetMode(CryptoRangeGetMode.ALL);
+
+    // Setting the endpoint and KMS region in cryptoConfig
+    AmazonS3EncryptionClientV2Builder.EndpointConfiguration epr
+        = createEndpointConfiguration(parameters.getEndpoint(), awsConf, getConf().getTrimmed(AWS_REGION));
+    if (epr != null) {
+      LOG.debug(
+          "Building the AmazonS3 Encryption client with endpoint configs");
+      builder.withEndpointConfiguration(epr);
+      cryptoConfigurationV2
+          .withAwsKmsRegion(RegionUtils.getRegion(epr.getSigningRegion()));
+      LOG.debug("KMS region used: {}",
+          cryptoConfigurationV2.getAwsKmsRegion());
+    } else {
+      // forcefully look for the region; extra HEAD call required.
+      builder.setForceGlobalBucketAccessEnabled(true);
+    }
+    builder.withCryptoConfiguration(cryptoConfigurationV2);

Review comment:
       Yes agree with refactoring common methods by steve.

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE;
+
+/**
+ * Tests to verify S3 Client-Side Encryption (CSE).
+ */
+public abstract class ITestS3AClientSideEncryption extends AbstractS3ATestBase {
+
+  private static final List<Integer> SIZES =
+      new ArrayList<>(Arrays.asList(0, 1, 255, 4095));
+
+  private static final int BIG_FILE_SIZE = 15 * 1024 * 1024;
+
+  /**
+   * Testing S3 CSE on different file sizes.
+   */
+  @Test
+  public void testEncryption() throws Throwable {
+    describe("Test to verify client-side encryption for different file sizes.");
+    for (int size : SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  /**
+   * Testing the S3 client side encryption over rename operation.
+   */
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    describe("Test for AWS CSE on Rename Operation.");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path src = path(getMethodName());
+    byte[] data = dataset(1024, 'a', 'z');
+    LOG.info("Region used: {}", fs.getAmazonS3Client().getRegionName());
+    writeDataset(fs, src, data, data.length, 1024 * 1024,
+        true, false);
+
+    //ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  /**
+   * Test to verify if we get same content length of files in S3 CSE using
+   * listStatus and listFiles on the parent directory.
+   */
+  @Test
+  public void testDirectoryListingFileLengths() throws IOException {
+    describe("Test to verify directory listing calls gives correct content "
+        + "lengths");
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path parentDir = path(getMethodName());
+
+    // Creating files in the parent directory that will be used to assert
+    // content length.
+    for (int i : SIZES) {
+      Path child = new Path(parentDir, getMethodName() + i);
+      writeThenReadFile(child, i);
+    }
+
+    // Getting the content lengths of files inside the directory via FileStatus.
+    List<Integer> fileLengthDirListing = new ArrayList<>();
+    for (FileStatus fileStatus : fs.listStatus(parentDir)) {
+      fileLengthDirListing.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // ListStatus.
+    Assertions.assertThat(fileLengthDirListing)
+        .describedAs("File lengths isn't same "
+            + "as expected from FileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+
+    // Getting the content lengths of files inside the directory via ListFiles.
+    RemoteIterator<LocatedFileStatus> listDir = fs.listFiles(parentDir, true);
+    List<Integer> fileLengthListLocated = new ArrayList<>();
+    while(listDir.hasNext()) {
+      LocatedFileStatus fileStatus = listDir.next();
+      fileLengthListLocated.add((int) fileStatus.getLen());
+    }
+    // Assert the file length we got against expected file length for
+    // LocatedFileStatus.
+    Assertions.assertThat(fileLengthListLocated)
+        .describedAs("File lengths isn't same "
+            + "as expected from LocatedFileStatus dir. listing")
+        .containsExactlyInAnyOrderElementsOf(SIZES);
+
+  }
+
+  /**
+   * Test to verify multipart upload through S3ABlockOutputStream and
+   * verifying the contents of the uploaded file.
+   */
+  @Test
+  public void testBigFilePutAndGet() throws IOException {
+    skipTest();
+    S3AFileSystem fs = getFileSystem();
+    Path filePath = path(getMethodName());
+    byte[] fileContent = new byte[BIG_FILE_SIZE];
+    // PUT a 15MB file using CSE to simulate multipart in CSE.
+    createFile(fs, filePath, true, fileContent);
+    LOG.info("Multi-part upload successful...");
+
+    try(FSDataInputStream in = fs.open(filePath)) {
+      in.seek(BIG_FILE_SIZE);
+      in.seek(0);
+      in.readFully(0, fileContent);
+
+      verifyFileContents(fs, filePath, fileContent);
+    }
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+
+    // To simulate multi part put and get in small files, we'll set the
+    // threshold and part size to 5MB.
+    conf.set(Constants.MULTIPART_SIZE, String.valueOf(MULTIPART_MIN_SIZE));
+    conf.set(Constants.MIN_MULTIPART_THRESHOLD, String.valueOf(MULTIPART_MIN_SIZE));
+    return conf;
+  }
+
+  /**
+   * Method to validate CSE for different file sizes.
+   *
+   * @param len length of the file.
+   */
+  protected void validateEncryptionForFilesize(int len) throws IOException {
+    skipTest();
+    describe("Create an encrypted file of size " + len);
+    // Creating a unique path by adding file length in file name.
+    Path path = writeThenReadFile(getMethodName() + len, len);
+    assertEncrypted(path);
+    rm(getFileSystem(), path, false, false);

Review comment:
       Do we have to explicitly do this? Won't this be done as a part of test cleanup and teardown?

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
##########
@@ -90,16 +90,18 @@
 public class Listing extends AbstractStoreOperation {
 
   private static final Logger LOG = S3AFileSystem.LOG;
+  private final boolean isCSEEnabled;
 
   static final FileStatusAcceptor ACCEPT_ALL_BUT_S3N =
       new AcceptAllButS3nDirs();
 
   private final ListingOperationCallbacks listingOperationCallbacks;
 
   public Listing(ListingOperationCallbacks listingOperationCallbacks,
-                 StoreContext storeContext) {
+                 StoreContext storeContext, boolean isCSEEnabled) {

Review comment:
       Move isCSEEnabled to storeContext?

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -94,9 +105,81 @@ public AmazonS3 createS3Client(
       awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix());
     }
 
-    return buildAmazonS3Client(
-        awsConf,
-        parameters);
+    if (conf.get(CLIENT_SIDE_ENCRYPTION_METHOD) == null) {
+      return buildAmazonS3Client(
+          awsConf,
+          parameters);
+    } else {
+      return newAmazonS3EncryptionClient(

Review comment:
       nit: buildAmazonS3EncryptionClient




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] bogthe commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
bogthe commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841812391


   > Had merge conflicts so had to force push.
   > Tests:
   > 
   > ```
   > [ERROR] Tests run: 1430, Failures: 1, Errors: 34, Skipped: 538
   > ```
   > 
   > Scale:
   > 
   > ```
   > [ERROR] Tests run: 151, Failures: 3, Errors: 21, Skipped: 29
   > ```
   > 
   > Most errors are MultiPart upload related:
   > 
   > ```
   > com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   > ```
   > 
   > Simply adding 16(Padding length) to multipart upload block size won't work. The part sizes need to be a multiple of 16, so it has that restriction for CSE. Also, one more thing to note here is that it assumes the last part to be an exception, which makes me believe that multipart upload in CSE has to be sequential(or can we parallel upload the starting parts and then upload the last part?)? So, potentially another constraint while uploading could have performance impacts here apart from the HEAD calls being required while downloading/listing.
   > @steveloughran
   
   Hi @mehakmeet , regarding multipart uploads. The last part is always an exception with regular multi part uploads too! You can do parallel uploads and even upload the last part first and it would still work (for regular multi-part). My assumption is that for multi part uploads with CSE enabled the same functionality holds (except for cipher block size, but the minimum part size for regular multi-part is 5MB = 5 * 1024 * 1024 which is still a multiple of 16 :D ). 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-841880071


   Hey @bogthe, 
   > The last part is always an exception with regular multi part uploads too! You can do parallel uploads and even upload the last part first and it would still work (for regular multi-part).
   
   Ah, I see, even I was thinking that with CSE it should still be able to find which part was last since during the parts upload step we provide part numbers and complete it in ascending order, But, I ran some tests with CSE enabled and I was facing these issues:
   
   AbstractContractMultipartUploaderTest#testMultipartUpload()
   T1: partSize: 5242880bytes(5MB) + 1byte = 5242881 bytes
   ```
   2021-05-17 02:43:43,998 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Put part 1 (size 5242881) s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:43:44,002 [s3a-transfer-shared-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(146)) - upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: Retried 0: org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   2021-05-17 02:43:44,824 [s3a-transfer-shared-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(146)) - upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: Retried 1: org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   2021-05-17 02:43:46,184 [s3a-transfer-shared-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(146)) - upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: Retried 2: org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   2021-05-17 02:43:50,537 [s3a-transfer-shared-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(146)) - upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: Retried 3: org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   2021-05-17 02:44:00,768 [s3a-transfer-shared-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(146)) - upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: Retried 4: org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   ```
   This retries a couple of times and fails with the exception:
   ```
   org.apache.hadoop.fs.s3a.AWSClientIOException: upload part #1 upload ID cFOhefvaRWyUGkB_U6zV2Mhs8RMC3u55_WOASIRCRuv1hVIeGciyQkvs5lA7gvZrdb8W5mCGwSQLsGmg9K9QbsPP1lcBF30vEVaUwbyfq0PjBxehxEeHyMklZE8hhYo_ on test/testMultipartUpload: com.amazonaws.SdkClientException: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.: Invalid part size: part sizes for encrypted multipart uploads must be multiples of the cipher block size (16) with the exception of the last part.
   ```
   
   T2: partSize: 5242880bytes(5MB)
   ```
   2021-05-17 02:46:22,270 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Put part 1 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:46:22,907 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:close(98)) - Put part 1 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload: duration 0:00.637s
   2021-05-17 02:46:22,910 [JUnit-testMultipartUpload] INFO  contract.ContractTestUtils (ContractTestUtils.java:end(1924)) - Duration of Uploaded part 1: 637,220,364 nS
   2021-05-17 02:46:22,911 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (AbstractContractMultipartUploaderTest.java:putPart(352)) - Upload bandwidth 7.846579 MB/s
   2021-05-17 02:46:22,934 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Put part 2 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:46:23,254 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:close(98)) - Put part 2 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload: duration 0:00.320s
   2021-05-17 02:46:23,254 [JUnit-testMultipartUpload] INFO  contract.ContractTestUtils (ContractTestUtils.java:end(1924)) - Duration of Uploaded part 2: 319,980,951 nS
   2021-05-17 02:46:23,255 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (AbstractContractMultipartUploaderTest.java:putPart(352)) - Upload bandwidth 15.625930 MB/s
   2021-05-17 02:46:23,275 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Put part 3 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:46:23,990 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:close(98)) - Put part 3 (size 5242880) s3a://mehakmeet-singh-data/test/testMultipartUpload: duration 0:00.715s
   2021-05-17 02:46:23,990 [JUnit-testMultipartUpload] INFO  contract.ContractTestUtils (ContractTestUtils.java:end(1924)) - Duration of Uploaded part 3: 715,353,661 nS
   2021-05-17 02:46:23,990 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (AbstractContractMultipartUploaderTest.java:putPart(352)) - Upload bandwidth 6.989550 MB/s
   2021-05-17 02:46:23,991 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Complete upload to s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:47:49,055 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:close(98)) - Complete upload to s3a://mehakmeet-singh-data/test/testMultipartUpload: duration 1:25.064s
   2021-05-17 02:47:49,056 [JUnit-testMultipartUpload] INFO  contract.AbstractContractMultipartUploaderTest (DurationInfo.java:<init>(77)) - Starting: Abort upload to s3a://mehakmeet-singh-data/test/testMultipartUpload
   2021-05-17 02:47:49,058 [s3a-transfer-shared-pool1-t6] INFO  s3a.S3AFileSystem (S3AFileSystem.java:abortMultipartUpload(4703)) - Aborting multipart upload l0UfFfsZXE8ogO8ojviT6D8iJo3oEM052apJu.txB1b5j1KPD4F8LQWWYHmOru4G1mu.uPPtGZhIYoT0P2S3g.k10ROOP7uXOiX7czPpmXzlA.67xB7YoN2_IczirQDL to test/testMultipartUpload
   ```
   Eventually fails with the exception:
   ```
   org.apache.hadoop.fs.s3a.AWSClientIOException: Completing multipart upload on test/testMultipartUpload: com.amazonaws.SdkClientException: Unable to complete an encrypted multipart upload without being told which part was the last.  Without knowing which part was the last, the encrypted data in Amazon S3 is incomplete and corrupt.: Unable to complete an encrypted multipart upload without being told which part was the last.  Without knowing which part was the last, the encrypted data in Amazon S3 is incomplete and corrupt.
   ```
   Both of these passes without CSE. 
   
   So, basically, we have a restriction to use only multiple of 16 as partSizes, even though min size of parts is 5MB and anything which is a multiple of MB, would be a multiple of 16, but we can't set any custom bytes(not multiple of 16) as partSize in CSE. 
   And, even after we set it to a multiple of 16, I am seeing the exception regarding last part. So, is the logic of part numbers not applicable in CSE? Maybe I am missing something here?
   
   CC: @steveloughran 


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran merged pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran merged pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706


   


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-783802582


   How, exactly do we get the content length without doing a head call? isn’t content length in the metadata as well?
   Also on that note, I figured something in the aws-sdk while doing a put request we have this in S3CryptoModuleBase.java:
   ```
   // Record the original, unencrypted content-length so it can be accessed
           // later
           final long plaintextLength = plaintextLength(request, metadata);
           if (plaintextLength >= 0) {
               metadata.addUserMetadata(Headers.UNENCRYPTED_CONTENT_LENGTH,
                                        Long.toString(plaintextLength));
               // Put the ciphertext length in the metadata
               metadata.setContentLength(ciphertextLength(plaintextLength));
           }
           request.setMetadata(metadata);
   ```
   Every put request in AmazonS3EncryptionClientV2, is wrapped like this. but it seems this isn’t the issue, in LIST ops, we don’t use these, right? so, how do we get file length in LIST calls?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r672610832



##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
##########
@@ -242,6 +242,7 @@ protected void logTimePerIOP(String operation,
 
   @Test
   public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
+    skipIfClientSideEncryption();

Review comment:
       ok

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -43,9 +52,12 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       losing battle here. I've got it wrong in some of the audit work and then it complicates backporting

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
##########
@@ -389,8 +397,10 @@ public void close() throws IOException {
         // PUT the final block
         if (hasBlock &&
             (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
-          //send last part
-          uploadCurrentBlock();
+          // send last part and set the value of isLastPart to true in case of
+          // CSE being enabled, since we are sure it is last part as parts
+          // are being uploaded serially in CSE.
+          uploadCurrentBlock(isCSEEnabled);

Review comment:
       good point. Looking at source there's no indication this is bad news if other uploads are in progress, and it would simplify code.
   
   why not set it?

##########
File path: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
##########
@@ -70,6 +76,53 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
 * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
 to encrypt and decrypt the data.
 
+Encryption options
+
+|  type | encryption | config on write | config on read |
+|-------|---------|-----------------|----------------|
+| `SSE-S3` | server side, AES256 | encryption algorithm | none |
+| `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none |
+| `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret |
+| `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm |
+
+With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS
+encryption channel).
+The data is dynamically encrypted and decrypted in the S3 Store, as needed.
+
+A server side algorithm can be enabled by default for a bucket, so that
+whenever data is uploaded unencrypted a default encryption algorithm is added.
+When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients
+downloading the data.
+SSE-C is different in that every client must know the secret key needed to decypt the data.
+
+Working with SSE-C data hard because every client must be configured to use the
+algorithm and supply the key. In particular, it is very hard to mix SSE-C
+encrypted objects in the same S3 bucket with objects encrypted with other
+algorithms or unencrypted; The S3A client
+(and other applications) get very confused.
+
+KMS-based key encryption is powerful as access to a key can be restricted to
+specific users/IAM roles. However, use of the key is billed and can be
+throttled. Furthermore as a client seeks around a file, the KMS key *may* be

Review comment:
       we've not done anything with client side keys. probably relevant on random IO where we do many GET's across a single input stream, so could cache it there.




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on a change in pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on a change in pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#discussion_r665556483



##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -3655,7 +3676,14 @@ S3AFileStatus s3GetFileStatus(final Path path,
         // look for the simple file
         ObjectMetadata meta = getObjectMetadata(key);
         LOG.debug("Found exact file: normal file {}", key);
-        return new S3AFileStatus(meta.getContentLength(),
+        long contentLength = meta.getContentLength();
+        // check if CSE is enabled, then strip padded length.
+        if (isCSEEnabled
+            && meta.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
+            && contentLength >= CSE_PADDING_LENGTH) {

Review comment:
       The header isn't present for multipart uploads, so to be consistent, thought we should just subtract the value instead. 




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] mukund-thakur commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-878924563


   Ran the raw s3 tests without setting up CSE. 
   Seeing just two failures and I think they are known issues.
   `[ERROR] Failures: 
   [ERROR]   ITestS3AMiscOperations.testEmptyFileChecksums:177->Assert.assertEquals:120->Assert.failNotEquals:835->Assert.fail:89 checksums of empty files expected:<etag: "a85fa2b141e24a8c0359c1541d7e2dcd"> but was:<etag: "1d61cbdc5bd4e70a8d93aefad3cf9d0e">
   [ERROR]   ITestS3AMiscOperations.testNonEmptyFileChecksumsUnencrypted:253->Assert.assertEquals:120->Assert.failNotEquals:835->Assert.fail:89 checksums expected:<etag: "e9fbb793667d00b7f362aa05ac4ccaeb"> but was:<etag: "05863e693027ce0f49048759006a0a40">`


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [hadoop] bogthe edited a comment on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
bogthe edited a comment on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-843269836


   Did some digging and found out some more context: 
   - For CSE `uploadPart` has to have parts [uploaded sequentially](https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/AmazonS3EncryptionClient.html#uploadPart-com.amazonaws.services.s3.model.UploadPartRequest-);
   - There was a request to support them parallel but it [doesn't look like they're going to be added to V1](https://github.com/aws/aws-sdk-java/issues/2248) and [it's on the feature request for V2](https://github.com/aws/aws-sdk-java-v2/projects/1)
   
   Also make sure your multi part code sets `lastPart` to `true` for CSE otherwise you'll still get the error you mentioned above. *


-- 
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:
users@infra.apache.org



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


[GitHub] [hadoop] mehakmeet commented on pull request #2706: HADOOP-13887. Support S3 client side encryption (S3-CSE) using AWS-SDK

Posted by GitBox <gi...@apache.org>.
mehakmeet commented on pull request #2706:
URL: https://github.com/apache/hadoop/pull/2706#issuecomment-879072223


   > This should have been skipped rather that failed.
   
   I think I saw it a while ago too after making the change, what worked for me was a clean build from root of project rather than hadoop-aws module, and then running the tests. 
   refer to Steve's comment: https://github.com/apache/hadoop/pull/2706#issuecomment-867746180


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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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