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/19 05:02:36 UTC

[GitHub] [hadoop] bilaharith opened a new pull request #2707: DRAFT PR: HADOOP-17536. ABFS: Supporting customer provided encryption key

bilaharith opened a new pull request #2707:
URL: https://github.com/apache/hadoop/pull/2707


   The data for a particular customer needs to be encrypted on account level. At server side the APIs will start accepting the encryption key as part of request headers. The data will be encrypted/decrypted with the given key at the server. 
   
   Since the ABFS FileSystem APIs are implementations for Hadoop FileSystem APIs there is no direct way with which customer can pass the key to ABFS driver. In this case driver should have the following capabilities so that it can accept and pass the encryption key as one of the request headers. 
   
   There should be a way to configure the encryption key for different accounts.
   If there is a key specified for a particular account, the same needs to be sent along with the request headers. 
   Config changes 
   
   They key for an account can be specified in the core-site as follows. 
   
   fs.azure.account.client-provided-encryption-key.{account name}.dfs.core.windows.net 


----------------------------------------------------------------
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -536,6 +594,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
 
   public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);

Review comment:
       When includeProperties is true, REST API GetFileProperties is invoked (which has user metadata in response) and is the case where CPK headers are needed. 
   REST API GetPathStatus doesnt consume CPK headers. Tomorrow if there is a rejection of headers not consumed, requests will fail.
   addCPK for includeProperties=true case.




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
##########
@@ -54,6 +55,9 @@
 
   public static final String FS_AZURE_TEST_APP_SECRET = "fs.azure.test.app.secret";
 
+  public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT = "fs.azure.test.cpk-enabled-secondary-account";

Review comment:
       Will update separately with a different PR once the feature is available across the regions.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName + "1");
+    FSDataOutputStream oStream = fs.create(testFilePath);
+    oStream.write(fileContent);
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.flush(testFileName, 0, false, false, null);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    if (isWithCPK) {
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+          .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+        LambdaTestUtils.intercept(IOException.class, () -> {
+          abfsClient3.flush(testFileName, 0, false, false, null);
+        });
+      }
+    }
+
+    //  With correct CPK
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush(testFileName, 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha);
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        isWithCPK + "");
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    testSetPathProperties(true);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    testSetPathProperties(false);
+  }
+
+  private void testSetPathProperties(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPathProperties(testFileName,
+            convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+  }
+
+  @Test
+  public void testGetPathStatusFileWithCPK() throws Exception {
+    testGetPathStatusFile(true);
+  }
+
+  @Test
+  public void testGetPathStatusFileWithoutCPK() throws Exception {
+    testGetPathStatusFile(false);
+  }
+
+  private void testGetPathStatusFile(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus(testFileName, false);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    abfsRestOperation = abfsClient.getPathStatus(testFileName, true);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  private void testDeletePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath(testFileName, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    testSetPermission(true);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    testSetPermission(false);
+  }
+
+  private void testSetPermission(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission(testFileName, permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    testSetAcl(true);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    testSetAcl(false);
+  }
+
+  private void testSetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    testGetAcl(true);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    testGetAcl(false);
+  }
+
+  private void testGetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    testCheckAccess(true);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    testCheckAccess(false);
+  }
+
+  private void testCheckAccess(final boolean isWithCPK) throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess(testFileName, "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  private byte[] createFileAndGetContent(AzureBlobFileSystem fs,
+      String fileName, int fileSize) throws IOException {
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+    return fileContent;
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertNoCPKResponseHeadersPresent(
+      AbfsRestOperation abfsRestOperation) {
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+  }
+
+  private void assertResponseHeader(AbfsRestOperation abfsRestOperation,
+      boolean isHeaderExpected, String headerName, String expectedValue) {
+    final AbfsHttpOperation result = abfsRestOperation.getResult();
+    final String value = result.getResponseHeader(headerName);
+    if (isHeaderExpected) {
+      Assertions.assertThat(value).isEqualTo(expectedValue);
+    } else {
+      Assertions.assertThat(value).isNull();
+    }
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc =
+          "CPK header " + headerName + " is expected, but the same is absent.";
+    } else {
+      desc = "CPK header " + headerName
+          + " is not expected, but the same is present.";
+    }
+    Assertions.assertThat(header.isPresent()).describedAs(desc)
+        .isEqualTo(isCPKHeaderExpected);
+  }
+
+  private byte[] getSHA256Hash(String key) throws IOException {
+    try {
+      final MessageDigest digester = MessageDigest.getInstance("SHA-256");
+      return digester.digest(key.getBytes(StandardCharsets.UTF_8));
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException {
+    Configuration conf = abfs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    String encryptionKey = conf
+        .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    if (encryptionKey == null || encryptionKey.isEmpty()) {
+      return "";
+    }
+    return getBase64EncodedString(getSHA256Hash(encryptionKey));
+  }
+
+  private String getBase64EncodedString(byte[] bytes) {
+    return java.util.Base64.getEncoder().encodeToString(bytes);
+  }
+
+  private Path createFileWithContent(FileSystem fs, String fileName,
+      byte[] fileContent) throws IOException {
+    Path testFilePath = new Path(fileName);
+    try (FSDataOutputStream oStream = fs.create(testFilePath)) {
+      oStream.write(fileContent);
+      oStream.flush();
+    }
+    return testFilePath;
+  }
+
+  private void verifyContent(AzureBlobFileSystem fs, Path testFilePath,

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");

Review comment:
       Done




-- 
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 #2707: DRAFT PR: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   2m  8s |  |  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 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m  3s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  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 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 29s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 24s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 8 new + 5 unchanged - 0 fixed = 13 total (was 5)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 40s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  findbugs  |   1m  3s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/1/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  0s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 24s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Found reliance on default encoding in new org.apache.hadoop.fs.azurebfs.services.AbfsClient(URL, SharedKeyCredentials, AbfsConfiguration, AbfsClientContext):in new org.apache.hadoop.fs.azurebfs.services.AbfsClient(URL, SharedKeyCredentials, AbfsConfiguration, AbfsClientContext): String.getBytes()  At AbfsClient.java:[line 104] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 402b0297a4da 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 9a298d180da |
   | 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-2707/1/testReport/ |
   | Max. process+thread count | 535 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/1/console |
   | versions | git=2.25.1 maven=3.6.3 findbugs=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] hadoop-yetus commented on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  21m 41s |  |  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 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  6s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  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 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 18s |  |  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 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 7 unchanged - 0 fixed = 10 total (was 7)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  9s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  94m 50s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml |
   | uname | Linux 35ce95c5a2ad 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 88448802be00bbeef8004289c1bc515c7327cada |
   | 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-2707/7/testReport/ |
   | Max. process+thread count | 702 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
##########
@@ -103,7 +102,7 @@ public TestAbfsClient(){
   }
 
   private String getUserAgentString(AbfsConfiguration config,
-      boolean includeSSLProvider) throws MalformedURLException {
+      boolean includeSSLProvider) throws IOException {

Review comment:
       Why is this change required ?




-- 
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 removed a comment on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  37m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  9s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 30s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 48s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 7 unchanged - 0 fixed = 10 total (was 7)  |
   | +1 :green_heart: |  mvnsite  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 20s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m  0s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux bf1f7325c791 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 176eb38e50ca229ef64a69639f67d14169797c27 |
   | 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-2707/12/testReport/ |
   | Max. process+thread count | 708 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/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] steveloughran commented on a change in pull request #2707: DRAFT PR: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -88,6 +96,21 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
     this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT));
     this.authType = abfsConfiguration.getAuthType(accountName);
 
+    String encryptionKey = this.abfsConfiguration.getClientProvidedEncryptionKey();
+    if (encryptionKey != null) {
+      this.clientProvidedEncryptionKey = encryptionKey;
+      try {
+        MessageDigest digester = MessageDigest.getInstance("SHA-256");
+        digester.update(encryptionKey.getBytes());
+        this.clientProvidedEncryptionKeySHA = Base64.getEncoder().encodeToString(digester.digest());
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException(e);

Review comment:
       * will this include information about the requested algorithm? 
   * will this surface when the JVM doesn' t have the unrestricted crypto extension settings? If so, error text should mention 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.

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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore

Review comment:
       Applies to other ignored 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.

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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -61,6 +65,9 @@
  */
 public class AbfsClient implements Closeable {
   public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  private static final String SERVER_SIDE_ENCRYPTION_ALGORITHM = "AES256";

Review comment:
       Move these to FileSystemConfigurations.java




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   
   +1,  merged to trunk. Thank you!


-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -536,6 +594,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
 
   public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {

Review comment:
       Only one getPathStatus exists




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -563,6 +622,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP
   public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
                                 final int bufferLength, final String eTag, String cachedSasToken) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);

Review comment:
       Server doesn't send back the actual sha of the encryption key with which the data is encrypted if the encryption key sent is 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.

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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();

Review comment:
       Context of testing ListStatus on a dir should be that dir (a) has files created with CPK and (b) a case of mismatched CPK used between any 2 child files. Test needs to adhere to these scenarios.




-- 
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
##########
@@ -250,8 +249,7 @@ public void verifyUserAgentClusterType() throws Exception {
 
   public static AbfsClient createTestClientFromCurrentContext(
       AbfsClient baseAbfsClientInstance,
-      AbfsConfiguration abfsConfig)
-      throws AzureBlobFileSystemException {
+      AbfsConfiguration abfsConfig) throws IOException {

Review comment:
       Why is this change required ?




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -536,6 +594,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
 
   public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {

Review comment:
       Only one setPathProperties exists




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  20m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  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 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m 58s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  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 42s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  13m 50s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m  8s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  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 :green_heart: |  spotbugs  |   1m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  95m 20s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/16/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux e60ed4e0b298 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 4e5268f62343732197cc2ef9b73b0f13ee2ce36c |
   | 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-2707/16/testReport/ |
   | Max. process+thread count | 711 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/16/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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);

Review comment:
       Assertions are done on the response headers 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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 47s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  shelldocs  |   0m  1s |  |  Shelldocs 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 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 42s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 46s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 25s |  |  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 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 20s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/6/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 7 unchanged - 0 fixed = 10 total (was 7)  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  shellcheck  |   0m  0s |  |  No new issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  2s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 57s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 38s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  75m  4s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense mvnsite unit codespell shellcheck shelldocs compile javac javadoc mvninstall shadedclient spotbugs checkstyle xml |
   | uname | Linux faa57a6d0dfc 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 173bc3c7fef8543b8ab2602c1dde14c047fcb77f |
   | 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-2707/6/testReport/ |
   | Max. process+thread count | 546 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/6/console |
   | versions | git=2.25.1 maven=3.6.3 shellcheck=0.7.0 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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore

Review comment:
       Why is this present if its to be ignored.




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
##########
@@ -54,6 +55,9 @@
 
   public static final String FS_AZURE_TEST_APP_SECRET = "fs.azure.test.app.secret";
 
+  public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT = "fs.azure.test.cpk-enabled-secondary-account";

Review comment:
       Will update separately with a different PR once the feature is available across the regions.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName + "1");
+    FSDataOutputStream oStream = fs.create(testFilePath);
+    oStream.write(fileContent);
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.flush(testFileName, 0, false, false, null);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    if (isWithCPK) {
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+          .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+        LambdaTestUtils.intercept(IOException.class, () -> {
+          abfsClient3.flush(testFileName, 0, false, false, null);
+        });
+      }
+    }
+
+    //  With correct CPK
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush(testFileName, 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha);
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        isWithCPK + "");
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    testSetPathProperties(true);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    testSetPathProperties(false);
+  }
+
+  private void testSetPathProperties(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPathProperties(testFileName,
+            convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+  }
+
+  @Test
+  public void testGetPathStatusFileWithCPK() throws Exception {
+    testGetPathStatusFile(true);
+  }
+
+  @Test
+  public void testGetPathStatusFileWithoutCPK() throws Exception {
+    testGetPathStatusFile(false);
+  }
+
+  private void testGetPathStatusFile(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus(testFileName, false);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    abfsRestOperation = abfsClient.getPathStatus(testFileName, true);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  private void testDeletePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath(testFileName, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    testSetPermission(true);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    testSetPermission(false);
+  }
+
+  private void testSetPermission(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission(testFileName, permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    testSetAcl(true);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    testSetAcl(false);
+  }
+
+  private void testSetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    testGetAcl(true);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    testGetAcl(false);
+  }
+
+  private void testGetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    testCheckAccess(true);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    testCheckAccess(false);
+  }
+
+  private void testCheckAccess(final boolean isWithCPK) throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess(testFileName, "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  private byte[] createFileAndGetContent(AzureBlobFileSystem fs,
+      String fileName, int fileSize) throws IOException {
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+    return fileContent;
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertNoCPKResponseHeadersPresent(
+      AbfsRestOperation abfsRestOperation) {
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+  }
+
+  private void assertResponseHeader(AbfsRestOperation abfsRestOperation,
+      boolean isHeaderExpected, String headerName, String expectedValue) {
+    final AbfsHttpOperation result = abfsRestOperation.getResult();
+    final String value = result.getResponseHeader(headerName);
+    if (isHeaderExpected) {
+      Assertions.assertThat(value).isEqualTo(expectedValue);
+    } else {
+      Assertions.assertThat(value).isNull();
+    }
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc =
+          "CPK header " + headerName + " is expected, but the same is absent.";
+    } else {
+      desc = "CPK header " + headerName
+          + " is not expected, but the same is present.";
+    }
+    Assertions.assertThat(header.isPresent()).describedAs(desc)
+        .isEqualTo(isCPKHeaderExpected);
+  }
+
+  private byte[] getSHA256Hash(String key) throws IOException {
+    try {
+      final MessageDigest digester = MessageDigest.getInstance("SHA-256");
+      return digester.digest(key.getBytes(StandardCharsets.UTF_8));
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException {
+    Configuration conf = abfs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    String encryptionKey = conf
+        .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    if (encryptionKey == null || encryptionKey.isEmpty()) {
+      return "";
+    }
+    return getBase64EncodedString(getSHA256Hash(encryptionKey));
+  }
+
+  private String getBase64EncodedString(byte[] bytes) {
+    return java.util.Base64.getEncoder().encodeToString(bytes);
+  }
+
+  private Path createFileWithContent(FileSystem fs, String fileName,
+      byte[] fileContent) throws IOException {
+    Path testFilePath = new Path(fileName);
+    try (FSDataOutputStream oStream = fs.create(testFilePath)) {
+      oStream.write(fileContent);
+      oStream.flush();
+    }
+    return testFilePath;
+  }
+
+  private void verifyContent(AzureBlobFileSystem fs, Path testFilePath,

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");

Review comment:
       Done




-- 
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 removed a comment on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  21m 41s |  |  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 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  6s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  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 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 18s |  |  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 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 7 unchanged - 0 fixed = 10 total (was 7)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  9s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  94m 50s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml |
   | uname | Linux 35ce95c5a2ad 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 88448802be00bbeef8004289c1bc515c7327cada |
   | 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-2707/7/testReport/ |
   | Max. process+thread count | 702 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/7/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] bilaharith commented on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   > I can see the need for this.
   > 
   > * One thing that it's not yet covered is the need to get this into delegation tokens. Look at what was done for the S3A DT's there: the encryption settings and secret key is picked up on the client and passed into the cluster so users can submit jobs into a shared cluster, _without that shared cluster having the encryption key_. Is this needed here? If so AbfsDelegationTokenIdentifier will need to be extended.
   > * is it ever the case that different stores will have different keys?
   
   Yes, each account can have it's own key. 


-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);

Review comment:
       Done




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   > Hi @steveloughran Could you please take a look
   
   I've been on a little vacation. I did have some review which was unsubmitted...now looks out of date. Will review 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.

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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -536,6 +594,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
 
   public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);

Review comment:
       Done




-- 
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] vinaysbadami commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
##########
@@ -133,6 +133,8 @@
   public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
   /** Setting this true will make the driver use it's own RemoteIterator implementation */
   public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator";
+  /** Server side encryption key */
+  public static final String FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY = "fs.azure.client-provided-encryption-key";

Review comment:
       we should not use the dot here. The dot is used to namespace. - is fine else make it one word




-- 
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
##########
@@ -224,7 +224,7 @@ public void loadConfiguredFileSystem() throws Exception {
       this.fileSystemName = authorityParts[0];
 
       // Reset URL with configured filesystem
-      final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+      final String abfsUrl = this.getFileSystemName() + "@" + authorityParts[1];

Review comment:
       Why is this change required ?




-- 
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Ignore
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    testDeleteFileSystem(true);
+  }
+
+  @Ignore
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    testDeleteFileSystem(false);
+  }
+
+  private void testDeleteFileSystem(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Ignore
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));

Review comment:
       why is liststatus being called here ?




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }

Review comment:
       Done




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;

Review comment:
       Removed

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -159,6 +185,18 @@ SharedKeyCredentials getSharedKeyCredentials() {
     return requestHeaders;
   }
 
+  private void addServerSideEncryptionHeaders(

Review comment:
       Done




-- 
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 removed a comment on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  20m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  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 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m 58s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  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 42s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  13m 50s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m  8s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  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 :green_heart: |  spotbugs  |   1m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  95m 20s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/16/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux e60ed4e0b298 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 4e5268f62343732197cc2ef9b73b0f13ee2ce36c |
   | 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-2707/16/testReport/ |
   | Max. process+thread count | 711 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/16/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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -563,6 +622,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP
   public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
                                 final int bufferLength, final String eTag, String cachedSasToken) throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);

Review comment:
       For the purposes of debugging, add a log line if request fails and the sha256 of the key used in request and the server returned sha256 in response are different.




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 34s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 36s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  13m 55s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 14s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/11/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 9 new + 9 unchanged - 0 fixed = 18 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 53s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  72m 55s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux dcd079ba7a41 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 8da250b9c6236adb92905f24d52b4163690b36a1 |
   | 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-2707/11/testReport/ |
   | Max. process+thread count | 745 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  37m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  9s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 30s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 48s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 7 unchanged - 0 fixed = 10 total (was 7)  |
   | +1 :green_heart: |  mvnsite  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 20s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m  0s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux bf1f7325c791 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 176eb38e50ca229ef64a69639f67d14169797c27 |
   | 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-2707/12/testReport/ |
   | Max. process+thread count | 708 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/12/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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
##########
@@ -339,9 +340,10 @@ public void processResponse(final byte[] buffer, final int offset, final int len
     if (this.requestId == null) {
       this.requestId = AbfsHttpConstants.EMPTY_STRING;
     }
+    responseHeaders = AbfsIoUtils.getResponseHeaders(connection);
     // dump the headers
     AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
-        connection.getHeaderFields());
+        responseHeaders);

Review comment:
       New method is introduced to print response headers which simply accepts the list of response headers




-- 
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] vinaysbadami commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -111,19 +132,38 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
   public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
                     final AbfsConfiguration abfsConfiguration,
                     final AccessTokenProvider tokenProvider,
-                    final AbfsClientContext abfsClientContext) {
+                    final AbfsClientContext abfsClientContext)
+      throws IOException {
     this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext);
     this.tokenProvider = tokenProvider;
   }
 
   public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
                     final AbfsConfiguration abfsConfiguration,
                     final SASTokenProvider sasTokenProvider,
-                    final AbfsClientContext abfsClientContext) {
+                    final AbfsClientContext abfsClientContext)
+      throws IOException {
     this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext);
     this.sasTokenProvider = sasTokenProvider;
   }
 
+  private byte[] getSHA256Hash(String key) throws IOException {
+    try {
+      final MessageDigest digester = MessageDigest.getInstance("SHA-256");
+      return digester.digest(key.getBytes(StandardCharsets.UTF_8));
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getBase64EncodedString(String key) {

Review comment:
       merge these into one function

##########
File path: hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh
##########
@@ -31,17 +31,17 @@ begin
 combination=HNS-OAuth
 properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled"
 "fs.azure.account.auth.type")
-values=("{account name}.dfs.core.windows.net" "true" "OAuth")
+values=("abfsitgen2.dfs.core.windows.net" "true" "OAuth")

Review comment:
       needed?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    testDeleteFileSystem(true);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    testDeleteFileSystem(false);
+  }
+
+  private void testDeleteFileSystem(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Path testPath = new Path(testFileName);
+    fs.create(testPath);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(testPath))
+        .isInstanceOf(FileNotFoundException.class);
+
+    listStatuses = fs.listStatus(new Path(newName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush(testFileName, 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    testSetPathProperties(true);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    testSetPathProperties(false);
+  }
+
+  private void testSetPathProperties(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPathProperties(testFileName,
+            convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithCPK() throws Exception {
+    testGetPathStatus(true);
+  }
+
+  @Test
+  public void testGetPathStatusWithoutCPK() throws Exception {
+    testGetPathStatus(false);
+  }
+
+  private void testGetPathStatus(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Path testPath = new Path(testFileName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testFileName + "/aaa"));
+    fs.mkdirs(new Path(testFileName + "/bbb"));
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus(testFileName, false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    testRead(true);
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    testRead(false);
+  }
+
+  private void testRead(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, testFileName, fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(testFileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(testFileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    testDeletePathWithoutCPK(false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    testDeletePathWithoutCPK(false);
+  }
+
+  private void testDeletePathWithoutCPK(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testFileName);
+    fs.create(testPath);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath(testFileName, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(testPath))
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    testSetPermission(true);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    testSetPermission(false);
+  }
+
+  private void testSetPermission(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission(testFileName, permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    testSetAcl(true);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    testSetAcl(false);
+  }
+
+  private void testSetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    testGetAcl(true);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    testGetAcl(false);
+  }
+
+  private void testGetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    testCheckAccess(true);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    testCheckAccess(false);
+  }
+
+  private void testCheckAccess(final boolean isWithCPK) throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess(testFileName, "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc = "CPK header is expected, but the same is absent.";

Review comment:
       mention header name

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);

Review comment:
       does listing return anything more for cpk blobs that needs verification?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;

Review comment:
       cant we just import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys, HttpHeaderConfigurations? What does this specifity buy us?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -536,6 +594,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
 
   public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException {

Review comment:
       ensure all overrides have it

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }

Review comment:
       change the key and try to read the file and ensure failure

##########
File path: hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
##########
@@ -62,12 +62,29 @@
       IT IN PATCHES OR COMMITS. -->
   <!--=============================================================-->
 
-  <include xmlns="http://www.w3.org/2001/XInclude" href="azure-auth-keys.xml">
+
+  <include xmlns="http://www.w3.org/2001/XInclude"

Review comment:
       needed?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();

Review comment:
       does gfs return anything more for cpk blobs that needs verification?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -512,6 +569,7 @@ public AbfsRestOperation flush(final String path, final long position, boolean r
   public AbfsRestOperation setPathProperties(final String path, final String properties)

Review comment:
       ensure all overrides have 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.

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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = 5;
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, length, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = 5;
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, length, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath("/test1", false, INT_50, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath("/test1", false, INT_50, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath("/test", true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath("/test", true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath("/test1", "/test2", null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath("/test1", "/test2", null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush("/test1", 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush("/test1", 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient.setPathProperties("/test1",
+        convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient.setPathProperties("/test1",
+        convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus("/test1", false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus("/test1", false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, "/test1", fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus("/test1", false);
+    final String eTag = op.getResult().getResponseHeader(
+        HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read("/test1", 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, "/test1", fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus("/test1", false);
+    final String eTag = op.getResult().getResponseHeader(
+        HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read("/test1", 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath("/test1", false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath("/test1", false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission("/test1", permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission("/test1", permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl("/test1", AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl("/test1", AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus("/test1");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus("/test1");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess("/test1", "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess("/test1", "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testWriteReadAndVerify() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 2 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc = "CPK hear should be resent";

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);

Review comment:
       Done




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }

Review comment:
       The AbfsClient method which the RemoteListIterator use is tested with this testcase. I think this should suffice.




-- 
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] bilaharith commented on pull request #2707: DRAFT PR: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 93, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 541, Failures: 0, Errors: 0, Skipped: 68
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 48
   
   HNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 93, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 541, Failures: 0, Errors: 0, Skipped: 26
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 40
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 93, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 541, Failures: 0, Errors: 0, Skipped: 257
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 40
   


----------------------------------------------------------------
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Ignore
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    testDeleteFileSystem(true);
+  }
+
+  @Ignore
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    testDeleteFileSystem(false);
+  }
+
+  private void testDeleteFileSystem(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Ignore
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));

Review comment:
       Changed to gfs




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -61,6 +65,9 @@
  */
 public class AbfsClient implements Closeable {
   public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  private static final String SERVER_SIDE_ENCRYPTION_ALGORITHM = "AES256";

Review comment:
       Done




-- 
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);

Review comment:
       CreateFileSystem API can be called from the Driver if the config createRemoteFileSystemDuringInitialization is true. Rest of FileSystem/Container APIs are not accessible from HDFS APIs. They are present in code to facilitate testing. 
   As CPK doesnt apply for container APIs anyway, remove all filesystem API 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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 34s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  1s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 36s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  13m 55s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 14s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/11/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 9 new + 9 unchanged - 0 fixed = 18 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 53s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  72m 55s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux dcd079ba7a41 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 8da250b9c6236adb92905f24d52b4163690b36a1 |
   | 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-2707/11/testReport/ |
   | Max. process+thread count | 745 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 36s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  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 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 53s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  9s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 29s |  |  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 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/13/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 3 new + 5 unchanged - 0 fixed = 8 total (was 5)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 54s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  74m 19s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 4db2c454c8f1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 09b462bb6ec15cc5f4a28086ec69c6e5df33f169 |
   | 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-2707/13/testReport/ |
   | Max. process+thread count | 668 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/13/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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
##########
@@ -133,6 +133,8 @@
   public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
   /** Setting this true will make the driver use it's own RemoteIterator implementation */
   public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator";
+  /** Server side encryption key */
+  public static final String FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY = "fs.azure.client-provided-encryption-key";

Review comment:
       dot as a delimiter has been the trend. Any reason to deviate ?




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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






-- 
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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {

Review comment:
       To test CPK, a file create and append with CPK headers needs to be done first. 
   Can be the test  be structured in a way so that test methods as as below ?
   createTest()
   appendTest() -> calls the above createTest and then does an append and flush
   readTest() -> calls the above appendTest and then does a read
   getFileStatus() -> calls the above appendTest and then does a read
   setXAttrs() + getXAttrs () -> calls the above appendTest and then does an append
   regressionTest() -> a single test that calls out all API like ListStatus or an acl api to ensure CPK didnt cause request failures.
   
   Above test methods can intake params for the negative case where CPK is not sent in header while file was created with CPK. 
   
   Basically there is a lot of code repeated and seems like it can be cut short.
   




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Ignore
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);

Review comment:
       Done




-- 
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 removed a comment on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  7s |  |  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 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  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 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  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  3s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 23s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 42s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  1s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  14m 29s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  75m  2s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  Questionable use of non-short-circuit logic in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:[line 75] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation is null guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:[line 291] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 9cfcedf8b7e5 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 1ebf5ae84755769c672c2d71a6f57e2c96a19c0a |
   | 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-2707/9/testReport/ |
   | Max. process+thread count | 693 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh
##########
@@ -31,17 +31,17 @@ begin
 combination=HNS-OAuth
 properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled"
 "fs.azure.account.auth.type")
-values=("{account name}.dfs.core.windows.net" "true" "OAuth")
+values=("abfsitgen2.dfs.core.windows.net" "true" "OAuth")

Review comment:
       Removed

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -111,19 +132,38 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
   public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
                     final AbfsConfiguration abfsConfiguration,
                     final AccessTokenProvider tokenProvider,
-                    final AbfsClientContext abfsClientContext) {
+                    final AbfsClientContext abfsClientContext)
+      throws IOException {
     this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext);
     this.tokenProvider = tokenProvider;
   }
 
   public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
                     final AbfsConfiguration abfsConfiguration,
                     final SASTokenProvider sasTokenProvider,
-                    final AbfsClientContext abfsClientContext) {
+                    final AbfsClientContext abfsClientContext)
+      throws IOException {
     this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext);
     this.sasTokenProvider = sasTokenProvider;
   }
 
+  private byte[] getSHA256Hash(String key) throws IOException {
+    try {
+      final MessageDigest digester = MessageDigest.getInstance("SHA-256");
+      return digester.digest(key.getBytes(StandardCharsets.UTF_8));
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getBase64EncodedString(String key) {

Review comment:
       Both are used




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    testDeleteFileSystem(true);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    testDeleteFileSystem(false);
+  }
+
+  private void testDeleteFileSystem(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Path testPath = new Path(testFileName);
+    fs.create(testPath);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(testPath))
+        .isInstanceOf(FileNotFoundException.class);
+
+    listStatuses = fs.listStatus(new Path(newName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush(testFileName, 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    testSetPathProperties(true);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    testSetPathProperties(false);
+  }
+
+  private void testSetPathProperties(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPathProperties(testFileName,
+            convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithCPK() throws Exception {
+    testGetPathStatus(true);
+  }
+
+  @Test
+  public void testGetPathStatusWithoutCPK() throws Exception {
+    testGetPathStatus(false);
+  }
+
+  private void testGetPathStatus(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Path testPath = new Path(testFileName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testFileName + "/aaa"));
+    fs.mkdirs(new Path(testFileName + "/bbb"));
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus(testFileName, false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    testRead(true);
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    testRead(false);
+  }
+
+  private void testRead(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, testFileName, fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(testFileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(testFileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    testDeletePathWithoutCPK(false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    testDeletePathWithoutCPK(false);
+  }
+
+  private void testDeletePathWithoutCPK(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testFileName);
+    fs.create(testPath);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath(testFileName, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(testPath))
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    testSetPermission(true);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    testSetPermission(false);
+  }
+
+  private void testSetPermission(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission(testFileName, permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    testSetAcl(true);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    testSetAcl(false);
+  }
+
+  private void testSetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    testGetAcl(true);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    testGetAcl(false);
+  }
+
+  private void testGetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    testCheckAccess(true);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    testCheckAccess(false);
+  }
+
+  private void testCheckAccess(final boolean isWithCPK) throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess(testFileName, "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc = "CPK header is expected, but the same is absent.";

Review comment:
       Done




-- 
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] vinaysbadami commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);

Review comment:
       refactor

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;

Review comment:
       needed?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -159,6 +185,18 @@ SharedKeyCredentials getSharedKeyCredentials() {
     return requestHeaders;
   }
 
+  private void addServerSideEncryptionHeaders(

Review comment:
       addCustomerProvidedKeyHeaders

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = 5;
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, length, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {

Review comment:
       we should have test cases for 2 CPK enabled accounts to ensure all is good.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClientCustomerProvidedKey.java
##########
@@ -0,0 +1,623 @@
+/**
+ * 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.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestAbfsClientCustomerProvidedKey
+    extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  private final IdentityTransformerInterface identityTransformer;
+
+  public ITestAbfsClientCustomerProvidedKey() throws Exception {
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass = getRawConfiguration()
+        .getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer = identityTransformerClass
+          .getConstructor(Configuration.class)
+          .newInstance(getRawConfiguration());
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = 5;
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, length, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append("/test", buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath("/test1", false, INT_50, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath("/test1", false, INT_50, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeleteFileSystemWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.deleteFilesystem();
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath("/test", true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath("/test", true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath("/test1", "/test2", null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath("/test1", "/test2", null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush("/test1", 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush("/test1", 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient.setPathProperties("/test1",
+        convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient.setPathProperties("/test1",
+        convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus("/test1", false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testGetPathStatusWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus("/test1", false);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, "/test1", fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus("/test1", false);
+    final String eTag = op.getResult().getResponseHeader(
+        HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read("/test1", 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    int fileSize = ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    createFileWithContent(fs, "/test1", fileContent);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus("/test1", false);
+    final String eTag = op.getResult().getResponseHeader(
+        HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read("/test1", 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath("/test1", false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath("/test1", false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission("/test1", permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission("/test1", permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl("/test1", AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl("/test1", AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus("/test1");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    boolean isWithCPK = false;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus("/test1");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess("/test1", "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    boolean isWithCPK = true;
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    fs.create(new Path("/test1"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess("/test1", "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+  }
+
+  @Test
+  public void testWriteReadAndVerify() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 2 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc = "CPK hear should be resent";

Review comment:
       typo




----------------------------------------------------------------
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   


-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -512,6 +569,7 @@ public AbfsRestOperation flush(final String path, final long position, boolean r
   public AbfsRestOperation setPathProperties(final String path, final String properties)

Review comment:
       Only one setPathProperties exists




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -412,6 +466,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer,
       AppendRequestParameters reqParams, final String cachedSasToken)
       throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
     requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,

Review comment:
       Will work as a separate task
   https://issues.apache.org/jira/browse/HADOOP-17636
   




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {

Review comment:
       Here we are testing the AbfsClient mehods. AbfsClient methods are from where CPK is getting incorporated with the HTTP headers. Tests are done with the right CPK, wrong CPK and no CPK.
   Basically the different clients are the clients from different file system instances, each of those file systems were created wit the above mentioned scenarios of correct wrong and no CPK configured.




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 59s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  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 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 57s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m  5s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  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 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  6s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  14m 18s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  5s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  75m 11s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  Questionable use of non-short-circuit logic in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:[line 75] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation is null guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:[line 291] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml |
   | uname | Linux c27a14dc41c6 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 0c04e502c510afb56cb094b39d2c3649b0e93dba |
   | 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-2707/8/testReport/ |
   | Max. process+thread count | 668 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   7m  3s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  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 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 11s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  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 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  4s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  7s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 26s |  |  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 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  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.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 44s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 58s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 34s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/17/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 0f36979737b8 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 4e5268f62343732197cc2ef9b73b0f13ee2ce36c |
   | 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-2707/17/testReport/ |
   | Max. process+thread count | 539 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/17/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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -88,6 +96,21 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
     this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT));
     this.authType = abfsConfiguration.getAuthType(accountName);
 
+    String encryptionKey = this.abfsConfiguration.getClientProvidedEncryptionKey();
+    if (encryptionKey != null) {
+      this.clientProvidedEncryptionKey = encryptionKey;
+      try {
+        MessageDigest digester = MessageDigest.getInstance("SHA-256");
+        digester.update(encryptionKey.getBytes());
+        this.clientProvidedEncryptionKeySHA = Base64.getEncoder().encodeToString(digester.digest());
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException(e);

Review comment:
       Looks like this is a common Exception, do we need to mention explicitly?
   https://stackoverflow.com/questions/48040299/java-security-nosuchalgorithmexception-sha3-384-messagedigest-not-available-w




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName+"1");
+    FSDataOutputStream oStream = fs.create(testFilePath);
+    oStream.write(fileContent);
+
+    //  Trying to read with different CPK headers

Review comment:
       fs is created with a CPK. Then fs2 is created with a different CPK.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName+"1");

Review comment:
       Done




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
##########
@@ -224,7 +224,7 @@ public void loadConfiguredFileSystem() throws Exception {
       this.fileSystemName = authorityParts[0];
 
       // Reset URL with configured filesystem
-      final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
+      final String abfsUrl = this.getFileSystemName() + "@" + authorityParts[1];

Review comment:
       This was buggy. authorityParts[1] is the actual FS to be loaded, getAccountName returns the FS that is created dynamically.




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");

Review comment:
       better: use `FileSystem.newInstance(URI, conf)` to explicitly get a new instance

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName + "1");
+    FSDataOutputStream oStream = fs.create(testFilePath);
+    oStream.write(fileContent);
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.flush(testFileName, 0, false, false, null);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    if (isWithCPK) {
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+          .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+        LambdaTestUtils.intercept(IOException.class, () -> {
+          abfsClient3.flush(testFileName, 0, false, false, null);
+        });
+      }
+    }
+
+    //  With correct CPK
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .flush(testFileName, 0, false, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha);
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        isWithCPK + "");
+  }
+
+  @Test
+  public void testSetPathPropertiesWithCPK() throws Exception {
+    testSetPathProperties(true);
+  }
+
+  @Test
+  public void testSetPathPropertiesWithoutCPK() throws Exception {
+    testSetPathProperties(false);
+  }
+
+  private void testSetPathProperties(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    final Hashtable<String, String> properties = new Hashtable<>();
+    properties.put("key", "val");
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPathProperties(testFileName,
+            convertXmsPropertiesToCommaSeparatedString(properties));
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+  }
+
+  @Test
+  public void testGetPathStatusFileWithCPK() throws Exception {
+    testGetPathStatusFile(true);
+  }
+
+  @Test
+  public void testGetPathStatusFileWithoutCPK() throws Exception {
+    testGetPathStatusFile(false);
+  }
+
+  private void testGetPathStatusFile(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .getPathStatus(testFileName, false);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    abfsRestOperation = abfsClient.getPathStatus(testFileName, true);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+  }
+
+  @Test
+  public void testDeletePathWithCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  @Test
+  public void testDeletePathWithoutCPK() throws Exception {
+    testDeletePath(false);
+  }
+
+  private void testDeletePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .deletePath(testFileName, false, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.listStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+  }
+
+  @Test
+  public void testSetPermissionWithCPK() throws Exception {
+    testSetPermission(true);
+  }
+
+  @Test
+  public void testSetPermissionWithoutCPK() throws Exception {
+    testSetPermission(false);
+  }
+
+  private void testSetPermission(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setPermission(testFileName, permission.toString());
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testSetAclWithCPK() throws Exception {
+    testSetAcl(true);
+  }
+
+  @Test
+  public void testSetAclWithoutCPK() throws Exception {
+    testSetAcl(false);
+  }
+
+  private void testSetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+
+    List<AclEntry> aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL));
+    final Map<String, String> aclEntries = AbfsAclHelper
+        .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries));
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testGetAclWithCPK() throws Exception {
+    testGetAcl(true);
+  }
+
+  @Test
+  public void testGetAclWithoutCPK() throws Exception {
+    testGetAcl(false);
+  }
+
+  private void testGetAcl(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    Assume.assumeTrue(fs.getIsNamespaceEnabled());
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  @Test
+  public void testCheckAccessWithCPK() throws Exception {
+    testCheckAccess(true);
+  }
+
+  @Test
+  public void testCheckAccessWithoutCPK() throws Exception {
+    testCheckAccess(false);
+  }
+
+  private void testCheckAccess(final boolean isWithCPK) throws Exception {
+    boolean isHNSEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false);
+    Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false",
+        isHNSEnabled);
+    Assume.assumeTrue("AuthType has to be OAuth",
+        getAuthType() == AuthType.OAuth);
+
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .checkAccess(testFileName, "rwx");
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+  }
+
+  private byte[] createFileAndGetContent(AzureBlobFileSystem fs,
+      String fileName, int fileSize) throws IOException {
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+    return fileContent;
+  }
+
+  private void assertCPKHeaders(AbfsRestOperation abfsRestOperation,
+      boolean isCPKHeaderExpected) {
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256,
+        isCPKHeaderExpected);
+    assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM,
+        isCPKHeaderExpected);
+  }
+
+  private void assertNoCPKResponseHeadersPresent(
+      AbfsRestOperation abfsRestOperation) {
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+  }
+
+  private void assertResponseHeader(AbfsRestOperation abfsRestOperation,
+      boolean isHeaderExpected, String headerName, String expectedValue) {
+    final AbfsHttpOperation result = abfsRestOperation.getResult();
+    final String value = result.getResponseHeader(headerName);
+    if (isHeaderExpected) {
+      Assertions.assertThat(value).isEqualTo(expectedValue);
+    } else {
+      Assertions.assertThat(value).isNull();
+    }
+  }
+
+  private void assertHeader(AbfsRestOperation abfsRestOperation,
+      String headerName, boolean isCPKHeaderExpected) {
+    assertTrue(abfsRestOperation != null);
+    Optional<AbfsHttpHeader> header = abfsRestOperation.getRequestHeaders()
+        .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName()
+            .equalsIgnoreCase(headerName)).findFirst();
+    String desc;
+    if (isCPKHeaderExpected) {
+      desc =
+          "CPK header " + headerName + " is expected, but the same is absent.";
+    } else {
+      desc = "CPK header " + headerName
+          + " is not expected, but the same is present.";
+    }
+    Assertions.assertThat(header.isPresent()).describedAs(desc)
+        .isEqualTo(isCPKHeaderExpected);
+  }
+
+  private byte[] getSHA256Hash(String key) throws IOException {
+    try {
+      final MessageDigest digester = MessageDigest.getInstance("SHA-256");
+      return digester.digest(key.getBytes(StandardCharsets.UTF_8));
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(e);
+    }
+  }
+
+  private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException {
+    Configuration conf = abfs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    String encryptionKey = conf
+        .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    if (encryptionKey == null || encryptionKey.isEmpty()) {
+      return "";
+    }
+    return getBase64EncodedString(getSHA256Hash(encryptionKey));
+  }
+
+  private String getBase64EncodedString(byte[] bytes) {
+    return java.util.Base64.getEncoder().encodeToString(bytes);
+  }
+
+  private Path createFileWithContent(FileSystem fs, String fileName,
+      byte[] fileContent) throws IOException {
+    Path testFilePath = new Path(fileName);
+    try (FSDataOutputStream oStream = fs.create(testFilePath)) {
+      oStream.write(fileContent);
+      oStream.flush();
+    }
+    return testFilePath;
+  }
+
+  private void verifyContent(AzureBlobFileSystem fs, Path testFilePath,

Review comment:
       Use `ContractTestUtils.`verifyFileContents()` as it does exactly this (with the same signature) but with significantly detailed error reporting as developed while dealing with eventual consistency problems an openstack swift.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
##########
@@ -54,6 +55,9 @@
 
   public static final String FS_AZURE_TEST_APP_SECRET = "fs.azure.test.app.secret";
 
+  public static final String FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT = "fs.azure.test.cpk-enabled-secondary-account";

Review comment:
       any changes to go in to the test docs?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3
+          .listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename = fs
+        .getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions
+        .assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))

Review comment:
       Please use intercept() over assertThatThrownBy
   
   The AssertJ one doesn't include the string value of result of the call (e.g. here, fileStatus.toString()) in the exception it raises, so loses a big piece of debuggability. See HADOOP-17535 for a recent example of this being invaluable.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,957 @@
+/**
+ * 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 ("License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled = getConfiguration()
+        .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED,
+        "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem
+        .get(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) {
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+      });
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem
+        .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       again, you can use `FileSystem.newInstance()`




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       Moved to try block

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       Moved to try block




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
##########
@@ -103,7 +102,7 @@ public TestAbfsClient(){
   }
 
   private String getUserAgentString(AbfsConfiguration config,
-      boolean includeSSLProvider) throws MalformedURLException {
+      boolean includeSSLProvider) throws IOException {

Review comment:
       AbfsClient constructor could throw IOException, wiuth the new encoding and hashing logic included

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
##########
@@ -250,8 +249,7 @@ public void verifyUserAgentClusterType() throws Exception {
 
   public static AbfsClient createTestClientFromCurrentContext(
       AbfsClient baseAbfsClientInstance,
-      AbfsConfiguration abfsConfig)
-      throws AzureBlobFileSystemException {
+      AbfsConfiguration abfsConfig) throws IOException {

Review comment:
       AbfsClient constructor could throw IOException, wiuth the new encoding and hashing logic included




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {

Review comment:
       Some of the changes are incorporated




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {

Review comment:
       Currently we are not planning to have a separate subclass exception for this. Since we don't see any special case handling specifically for this case.

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>

Review comment:
       Done




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore

Review comment:
       Made as config controlled




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");

Review comment:
       Moved to try block.
   This cannot be refactored to a common method, since the method to evaluate and tested are called on the AbfsClient instance of the respective FileSystem instance, and the method to test on the abfsclient differ. So keeping it this way.




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  7s |  |  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 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  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 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  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  3s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 23s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 42s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  1s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  14m 29s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  1s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  75m  2s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  Questionable use of non-short-circuit logic in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:[line 75] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation is null guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:[line 291] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 9cfcedf8b7e5 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 1ebf5ae84755769c672c2d71a6f57e2c96a19c0a |
   | 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-2707/9/testReport/ |
   | Max. process+thread count | 693 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -412,6 +466,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer,
       AppendRequestParameters reqParams, final String cachedSasToken)
       throws AzureBlobFileSystemException {
     final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    addCustomerProvidedKeyHeaders(requestHeaders);
     // JDK7 does not support PATCH, so to workaround the issue we will use
     // PUT and specify the real method in the X-Http-Method-Override header.
     requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,

Review comment:
       while here, is this code needed any more?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

Review comment:
       Now, these do need to go up into the "non org-apache section"; relates to how backporting usually needs to revert these back to the com.google package

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {

Review comment:
       Is there a specific IOE subclass or constant error message which can be looked for?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       again, need to close in try-with-resources or similar

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {

Review comment:
       I'm trying to understand what's happening here as it seems to depend on some specific knowledge of how the client buffers things. What is actually happening here? does oStream.write() buffer internally? if so: why does calling flush() on different clients fail?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName+"1");

Review comment:
       nit: spacing

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;

Review comment:
       and try to get these imports in order

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;

Review comment:
       wrong place for the import

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       As it's not cached, use in try-with-resources to ensure closed. I think you can merge it into the one on L119

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>

Review comment:
       remove the <p>; can confuse license checker
   
   (note: this doesn't need to be a javadoc header, I think its more accident/tradition than requirement).

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);

Review comment:
       same comments about fs2 close, IOE subclass etc as above

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");

Review comment:
       and here too. There's actually a lot of duplication here isn't there? Have you thought of factoring it out into some method which takes the closure for the intercept() and then does everything else around it?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);

Review comment:
       can pull into the try-with-resources resource list (takes a whole ;-separated list of declarations)

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }

Review comment:
       add something for listStatusIterator() too, so that if/when it moves to doing async fetching, it still does the right thing here

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,937 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.apache.hadoop.fs.azurebfs.services.*;
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+  private static final int ENCRYPTION_KEY_LEN = 32;
+  private static final int FILE_SIZE = 10 * ONE_MB;
+  private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB;
+
+  public ITestCustomerProvidedKey() throws Exception {
+    boolean isCPKTestsEnabled =
+        getConfiguration().getBoolean(FS_AZURE_TEST_CPK_ENABLED, false);
+    Assume.assumeTrue(isCPKTestsEnabled);
+  }
+
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = FILE_SIZE;
+    byte[] buffer = new byte[length];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(new Path(fileName))) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    createFileAndGetContent(fs, fileName, FILE_SIZE);
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE);
+    createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    //  Trying with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    abfsRestOperation = abfsClient2
+        .listPath(testDirName, false, INT_50, null);
+    assertListstatus(fs, abfsRestOperation, testPath);
+
+    if (isWithCPK) {
+      //  Trying with no CPK headers
+      conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+      AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+      AbfsClient abfsClient3 = fs3.getAbfsClient();
+      abfsRestOperation = abfsClient3.listPath(testDirName, false, INT_50, null);
+      assertListstatus(fs, abfsRestOperation, testPath);
+    }
+  }
+
+  private void assertListstatus(AzureBlobFileSystem fs,
+      AbfsRestOperation abfsRestOperation, Path testPath) throws IOException {
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testCreatePathWithCPK() throws Exception {
+    testCreatePath(true);
+  }
+
+  @Test
+  public void testCreatePathWithoutCPK() throws Exception {
+    testCreatePath(false);
+  }
+
+  private void testCreatePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    FsPermission permission = new FsPermission(FsAction.EXECUTE,
+        FsAction.EXECUTE, FsAction.EXECUTE);
+    FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE,
+        FsAction.NONE);
+    boolean isNamespaceEnabled = fs.getIsNamespaceEnabled();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .createPath(testFileName, true, true,
+            isNamespaceEnabled ? getOctalNotation(permission) : null,
+            isNamespaceEnabled ? getOctalNotation(umask) : null, false, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+    assertResponseHeader(abfsRestOperation, isWithCPK,
+        X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    FileStatus[] listStatuses = fs.listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName));
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 1 entry").isEqualTo(1);
+  }
+
+  @Test
+  public void testRenamePathWithCPK() throws Exception {
+    testRenamePath(true);
+  }
+
+  @Test
+  public void testRenamePathWithoutCPK() throws Exception {
+    testRenamePath(false);
+  }
+
+  private void testRenamePath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    createFileAndGetContent(fs, testFileName, FILE_SIZE);
+
+    FileStatus fileStatusBeforeRename =
+        fs.getFileStatus(new Path(testFileName));
+
+    String newName = "/newName";
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .renamePath(testFileName, newName, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertNoCPKResponseHeadersPresent(abfsRestOperation);
+
+    Assertions.assertThatThrownBy(() -> fs.getFileStatus(new Path(testFileName)))
+        .isInstanceOf(FileNotFoundException.class);
+
+    FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName));
+    Assertions.assertThat(fileStatusAfterRename.getLen())
+        .describedAs("File size has to be same before and after rename")
+        .isEqualTo(fileStatusBeforeRename.getLen());
+  }
+
+  @Test
+  public void testFlushWithCPK() throws Exception {
+    testFlush(true);
+  }
+
+  @Test
+  public void testFlushWithoutCPK() throws Exception {
+    testFlush(false);
+  }
+
+  private void testFlush(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    String expectedCPKSha = getCPKSha(fs);
+
+    byte[] fileContent = getRandomBytesArray(FILE_SIZE);
+    Path testFilePath = new Path(testFileName+"1");
+    FSDataOutputStream oStream = fs.create(testFilePath);
+    oStream.write(fileContent);
+
+    //  Trying to read with different CPK headers

Review comment:
       nit: comment is 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.

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] snvijaya commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
##########
@@ -339,9 +340,10 @@ public void processResponse(final byte[] buffer, final int offset, final int len
     if (this.requestId == null) {
       this.requestId = AbfsHttpConstants.EMPTY_STRING;
     }
+    responseHeaders = AbfsIoUtils.getResponseHeaders(connection);
     // dump the headers
     AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
-        connection.getHeaderFields());
+        responseHeaders);

Review comment:
       Why is this change required ?




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,995 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ITestCustomerProvidedKey.class);
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = "/" + methodName.getMethodName();
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs2.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+
+    //  Trying to read with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(testFilePath)) {
+      int len = 8 * ONE_MB;
+      byte[] b = new byte[len];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(b, 0, len);
+      });
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testReadWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    int fileSize = 8 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = "/" + methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    verifyContent(fs, testFilePath, fileContent);
+
+    AbfsClient abfsClient = fs.getAbfsClient();
+    int length = INT_512;
+    byte[] buffer = new byte[length * 4];
+    final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false);
+    final String eTag = op.getResult()
+        .getResponseHeader(HttpHeaderConfigurations.ETAG);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .read(fileName, 0, buffer, 0, length, eTag, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED,
+        "true");
+    assertResponseHeader(abfsRestOperation, false,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "");
+
+    //  Trying to read with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append with correct CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, true);
+    assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256,
+        getCPKSha(fs));
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with different CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+
+    //  Trying to append with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient3 = fs3.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient3.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(false);
+    final String fileName = "/" + methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    //  Trying to append without CPK headers
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(fileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, false);
+    assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256,
+        "");
+    assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, "");
+    assertResponseHeader(abfsRestOperation, true,
+        X_MS_REQUEST_SERVER_ENCRYPTED, "true");
+
+    //  Trying to append with CPK headers
+    Configuration conf = fs.getConf();
+    String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "12345678901234567890123456789012");
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+    AbfsClient abfsClient2 = fs2.getAbfsClient();
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      abfsClient2.append(fileName, buffer, appendRequestParameters, null);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getAbfs(false).getXAttr(new Path(fileName), attrName);
+    });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class, () -> {
+      getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+    });
+  }
+
+  @Ignore
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class, () -> {
+        iStream.read(buffer, 0, length);
+      });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Ignore
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();

Review comment:
       Done




-- 
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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 26s |  |  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 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m  3s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  2s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 20s |  |  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 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/10/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  4s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  2s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  81m 31s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 8670aec7bb04 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 2cd36d7aaee8354f70468e6ae830b4c294ced0fa |
   | 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-2707/10/testReport/ |
   | Max. process+thread count | 566 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
+import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+
+public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest {
+
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int INT_512 = 512;
+  private static final int INT_50 = 50;
+
+  public ITestCustomerProvidedKey() throws Exception {
+  }
+
+  @Test
+  public void testWriteReadAndVerifyWithCPK() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    int fileSize = 16 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    String fileName = methodName.getMethodName();
+    Path testFilePath = createFileWithContent(fs, fileName, fileContent);
+    try (FSDataInputStream iStream = fs.open(testFilePath)) {
+      byte[] buffer = new byte[fileSize];
+      int bytesRead = iStream.read(buffer, 0, fileSize);
+      assertEquals(bytesRead, fileSize);
+      for (int i = 0; i < fileSize; i++) {
+        assertEquals(fileContent[i], buffer[i]);
+      }
+    }
+  }
+
+  @Test
+  public void testSetGetXAttr() throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(true);
+    String fileName = methodName.getMethodName();
+    fs.create(new Path(fileName));
+
+    String valSent = "testValue";
+    String attrName = "testXAttr";
+
+    //  set get and verify
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.CREATE));
+    byte[] valBytes = fs.getXAttr(new Path(fileName), attrName);
+    String valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  set new value get and verify
+    valSent = "new value";
+    fs.setXAttr(new Path(fileName), attrName,
+        valSent.getBytes(StandardCharsets.UTF_8),
+        EnumSet.of(XAttrSetFlag.REPLACE));
+    valBytes = fs.getXAttr(new Path(fileName), attrName);
+    valRecieved = new String(valBytes);
+    assertEquals(valSent, valRecieved);
+
+    //  Read without CPK header
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getAbfs(false).getXAttr(new Path(fileName), attrName);
+        });
+
+    //  Wrong CPK
+    LambdaTestUtils.intercept(IOException.class,
+        () -> {
+          getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName);
+        });
+  }
+
+  @Test
+  public void testCopyBetweenAccounts() throws Exception {
+    String accountName = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT);
+    String accountKey = getRawConfiguration()
+        .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY);
+    Assume.assumeTrue(accountName != null && !accountName.isEmpty());
+    Assume.assumeTrue(accountKey != null && !accountKey.isEmpty());
+    String fileSystemName = "cpkfs";
+
+    //  Create fs1 and a file with CPK
+    AzureBlobFileSystem fs1 = getAbfs(true);
+    int fileSize = 24 * ONE_MB;
+    byte[] fileContent = getRandomBytesArray(fileSize);
+    Path testFilePath = createFileWithContent(fs1, "fs1-file.txt", fileContent);
+
+    //  Create fs2 with different CPK
+    Configuration conf = new Configuration();
+    conf.set("fs.abfs.impl.disable.cache", "true");
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
+    conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME);
+    conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName);
+    conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "123456789012345678901234567890ab");
+    conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName);
+    AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.get(conf);
+
+    //  Read from fs1 and write to fs2, fs1 and fs2 are having different CPK
+    Path fs2DestFilePath = new Path("fs2-dest-file.txt");
+    FSDataOutputStream ops = fs2.create(fs2DestFilePath);
+    try (FSDataInputStream iStream = fs1.open(testFilePath)) {
+      long totalBytesRead = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        ops.write(buffer);
+      } while (totalBytesRead < fileContent.length);
+      ops.close();
+    }
+
+    //  Trying to read fs2DestFilePath with different CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName,
+        "different-1234567890123456789012");
+    AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs3.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Trying to read fs2DestFilePath with no CPK headers
+    conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName);
+    AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem.get(conf);
+    try (FSDataInputStream iStream = fs4.open(fs2DestFilePath)) {
+      int length = 8 * ONE_MB;
+      byte[] buffer = new byte[length];
+      LambdaTestUtils.intercept(IOException.class,
+          () -> {
+            iStream.read(buffer, 0, length);
+          });
+    }
+
+    //  Read fs2DestFilePath and verify the content with the initial random
+    //  bytes created and wrote into the source file at fs1
+    try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) {
+      long totalBytesRead = 0;
+      int pos = 0;
+      do {
+        int length = 8 * ONE_MB;
+        byte[] buffer = new byte[length];
+        int bytesRead = iStream.read(buffer, 0, length);
+        totalBytesRead += bytesRead;
+        for (int i = 0; i < bytesRead; i++) {
+          assertEquals(fileContent[pos + i], buffer[i]);
+        }
+        pos = pos + bytesRead;
+      } while (totalBytesRead < fileContent.length);
+    }
+  }
+
+  @Test
+  public void testAppendWithCPK() throws Exception {
+    testAppend(true);
+  }
+
+  @Test
+  public void testAppendWithoutCPK() throws Exception {
+    testAppend(false);
+  }
+
+  private void testAppend(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AppendRequestParameters appendRequestParameters =
+        new AppendRequestParameters(
+        0, 0, 5, Mode.APPEND_MODE, false);
+    byte[] buffer = getRandomBytesArray(5);
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .append(testFileName, buffer, appendRequestParameters, null);
+    assertCPKHeaders(abfsRestOperation, isWithCPK);
+  }
+
+  @Test
+  public void testListPathWithCPK() throws Exception {
+    testListPath(true);
+  }
+
+  @Test
+  public void testListPathWithoutCPK() throws Exception {
+    testListPath(false);
+  }
+
+  private void testListPath(final boolean isWithCPK) throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    String testDirName = "/" + methodName.getMethodName();
+    final Path testPath = new Path(testDirName);
+    fs.mkdirs(testPath);
+    fs.mkdirs(new Path(testDirName + "/aaa"));
+    fs.mkdirs(new Path(testDirName + "/bbb"));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient
+        .listPath(testDirName, false, INT_50, null);
+
+    //  assert cpk headers are not added
+    assertCPKHeaders(abfsRestOperation, false);
+
+    FileStatus[] listStatuses = fs.listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+
+    listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath);
+    Assertions.assertThat(listStatuses.length)
+        .describedAs("listStatuses should have 2 entries").isEqualTo(2);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithCPK() throws Exception {
+    testGetFileSystemProperties(true);
+  }
+
+  @Test
+  public void testGetFileSystemPropertiesWithoutCPK() throws Exception {
+    testGetFileSystemProperties(false);
+  }
+
+  private void testGetFileSystemProperties(final boolean isWithCPK)
+      throws Exception {
+    final AzureBlobFileSystem fs = getAbfs(isWithCPK);
+    final String testFileName = "/" + methodName.getMethodName();
+    fs.create(new Path(testFileName));
+    AbfsClient abfsClient = fs.getAbfsClient();
+    AbfsRestOperation abfsRestOperation = abfsClient.getFilesystemProperties();

Review comment:
       Assertions are done on the response headers as well

##########
File path: hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
##########
@@ -62,12 +62,29 @@
       IT IN PATCHES OR COMMITS. -->
   <!--=============================================================-->
 
-  <include xmlns="http://www.w3.org/2001/XInclude" href="azure-auth-keys.xml">
+
+  <include xmlns="http://www.w3.org/2001/XInclude"

Review comment:
       No, removed




-- 
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] bilaharith commented on a change in pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java
##########
@@ -0,0 +1,741 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;

Review comment:
       Done




-- 
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 removed a comment on pull request #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 26s |  |  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 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m  3s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  2s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 20s |  |  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 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/10/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  4s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  2s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  81m 31s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 8670aec7bb04 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 2cd36d7aaee8354f70468e6ae830b4c294ced0fa |
   | 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-2707/10/testReport/ |
   | Max. process+thread count | 566 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/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.

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 #2707: HADOOP-17536. ABFS: Supporting customer provided encryption key

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 59s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  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 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 57s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m  5s |  |  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 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  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 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 9 unchanged - 0 fixed = 13 total (was 9)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  spotbugs  |   1m  6s | [/new-spotbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/new-spotbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  14m 18s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  5s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  75m 11s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-tools/hadoop-azure |
   |  |  Questionable use of non-short-circuit logic in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:in org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils.dumpHeadersToDebugLog(String, List)  At AbfsIoUtils.java:[line 75] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation could be null and is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:is guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int)  Dereferenced at AbfsRestOperation.java:[line 291] |
   |  |  httpOperation is null guaranteed to be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:be dereferenced in org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.executeHttpOperation(int) on exception path  Dereferenced at AbfsRestOperation.java:[line 291] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2707 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml |
   | uname | Linux c27a14dc41c6 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 0c04e502c510afb56cb094b39d2c3649b0e93dba |
   | 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-2707/8/testReport/ |
   | Max. process+thread count | 668 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2707/8/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