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/11/03 13:15:08 UTC

[GitHub] [hadoop] mariosmeim-db commented on a change in pull request #3440: ABFS: Support for Encryption Context

mariosmeim-db commented on a change in pull request #3440:
URL: https://github.com/apache/hadoop/pull/3440#discussion_r741897486



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java
##########
@@ -0,0 +1,145 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Base64;
+import javax.crypto.SecretKey;
+import javax.security.auth.DestroyFailedException;
+import javax.security.auth.Destroyable;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
+
+public class EncryptionAdapter implements Destroyable {
+  private final String path;
+  private SecretKey encryptionContext;
+  private SecretKey encryptionKey;
+  private final EncryptionContextProvider provider;
+  private String encodedKey = null;
+  private String encodedKeySHA = null;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EncryptionAdapter.class);
+
+  public EncryptionAdapter(EncryptionContextProvider provider, String path,
+      byte[] encryptionContext) throws IOException {
+    this(provider, path);
+    Preconditions.checkNotNull(encryptionContext,
+        "Encryption context should not be null.");
+    this.encryptionContext = new ABFSSecretKey(encryptionContext);
+  }
+
+  public EncryptionAdapter(EncryptionContextProvider provider, String path)
+      throws IOException {
+    this.provider = provider;
+    this.path = path;
+  }
+
+  public SecretKey getEncryptionKey() throws IOException {
+    if (encryptionKey != null) {
+      return encryptionKey;
+    }
+    encryptionKey = provider.getEncryptionKey(path, encryptionContext);
+    return encryptionKey;
+  }
+
+  public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException {

Review comment:
       Perhaps it should be more explicit in the naming that this is used to create a new encryption context.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -221,16 +223,50 @@ SharedKeyCredentials getSharedKeyCredentials() {
     return requestHeaders;
   }
 
-  private void addCustomerProvidedKeyHeaders(
-      final List<AbfsHttpHeader> requestHeaders) {
-    if (clientProvidedEncryptionKey != null) {
-      requestHeaders.add(
-          new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, clientProvidedEncryptionKey));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256,
-          clientProvidedEncryptionKeySHA));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM,
-          SERVER_SIDE_ENCRYPTION_ALGORITHM));
+  private void addEncryptionKeyRequestHeaders(String path,
+      List<AbfsHttpHeader> requestHeaders, boolean isCreateFileRequest,
+      EncryptionAdapter encryptionAdapter, TracingContext tracingContext)
+      throws IOException {
+    String encodedKey, encodedKeySHA256;
+    switch (encryptionType) {
+    case GLOBAL_KEY:
+      encodedKey = clientProvidedEncryptionKey;
+      encodedKeySHA256 = clientProvidedEncryptionKeySHA;
+      break;
+
+    case ENCRYPTION_CONTEXT:
+      if (isCreateFileRequest) {
+        // get new context for create file request
+        SecretKey encryptionContext =
+            encryptionAdapter.fetchEncryptionContextAndComputeKeys();
+        requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT,
+            new String(encryptionContext.getEncoded(),
+                StandardCharsets.UTF_8)));
+        try {
+          encryptionContext.destroy();
+        } catch (DestroyFailedException e) {
+          throw new IOException(
+              "Could not destroy encryptionContext: " + e.getMessage());
+        }
+      } else if (encryptionAdapter == null) {
+        // get encryption context from GetPathStatus response header
+        encryptionAdapter = new EncryptionAdapter(encryptionContextProvider,
+            new Path(path).toUri().getPath(),

Review comment:
       Should we add some error handling and logging around this?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java
##########
@@ -0,0 +1,145 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Base64;
+import javax.crypto.SecretKey;
+import javax.security.auth.DestroyFailedException;
+import javax.security.auth.Destroyable;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
+
+public class EncryptionAdapter implements Destroyable {
+  private final String path;
+  private SecretKey encryptionContext;
+  private SecretKey encryptionKey;
+  private final EncryptionContextProvider provider;
+  private String encodedKey = null;
+  private String encodedKeySHA = null;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EncryptionAdapter.class);
+
+  public EncryptionAdapter(EncryptionContextProvider provider, String path,
+      byte[] encryptionContext) throws IOException {
+    this(provider, path);
+    Preconditions.checkNotNull(encryptionContext,
+        "Encryption context should not be null.");
+    this.encryptionContext = new ABFSSecretKey(encryptionContext);
+  }
+
+  public EncryptionAdapter(EncryptionContextProvider provider, String path)
+      throws IOException {
+    this.provider = provider;
+    this.path = path;
+  }
+
+  public SecretKey getEncryptionKey() throws IOException {
+    if (encryptionKey != null) {
+      return encryptionKey;
+    }
+    encryptionKey = provider.getEncryptionKey(path, encryptionContext);
+    return encryptionKey;
+  }
+
+  public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException {
+    encryptionContext = provider.getEncryptionContext(path);

Review comment:
       If I understand correctly, this will only be reached when creating a new file, and will therefore make the proider create a new encryption context. Perhaps we could add a null check on the `encryptionContext` here.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -113,13 +119,14 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
     this.retryPolicy = abfsClientContext.getExponentialRetryPolicy();
     this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT));
     this.authType = abfsConfiguration.getAuthType(accountName);
+    this.encryptionContextProvider = encryptionContextProvider;
 
     String encryptionKey = this.abfsConfiguration
         .getClientProvidedEncryptionKey();
     if (encryptionKey != null) {
-      this.clientProvidedEncryptionKey = getBase64EncodedString(encryptionKey);
-      this.clientProvidedEncryptionKeySHA = getBase64EncodedString(
-          getSHA256Hash(encryptionKey));
+      this.clientProvidedEncryptionKey = EncryptionAdapter.getBase64EncodedString(encryptionKey);

Review comment:
       Shouldn't the encryption key passed in the configuration already be base 64 encoded?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -221,16 +223,50 @@ SharedKeyCredentials getSharedKeyCredentials() {
     return requestHeaders;
   }
 
-  private void addCustomerProvidedKeyHeaders(
-      final List<AbfsHttpHeader> requestHeaders) {
-    if (clientProvidedEncryptionKey != null) {
-      requestHeaders.add(
-          new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, clientProvidedEncryptionKey));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256,
-          clientProvidedEncryptionKeySHA));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM,
-          SERVER_SIDE_ENCRYPTION_ALGORITHM));
+  private void addEncryptionKeyRequestHeaders(String path,
+      List<AbfsHttpHeader> requestHeaders, boolean isCreateFileRequest,
+      EncryptionAdapter encryptionAdapter, TracingContext tracingContext)
+      throws IOException {
+    String encodedKey, encodedKeySHA256;
+    switch (encryptionType) {
+    case GLOBAL_KEY:
+      encodedKey = clientProvidedEncryptionKey;

Review comment:
       Did you consider creating an implementation of EncryptionContextProvider for the global key case, just for consistency between the two types? In that case, would the EncryptionAdapter still be required?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
##########
@@ -221,16 +223,50 @@ SharedKeyCredentials getSharedKeyCredentials() {
     return requestHeaders;
   }
 
-  private void addCustomerProvidedKeyHeaders(
-      final List<AbfsHttpHeader> requestHeaders) {
-    if (clientProvidedEncryptionKey != null) {
-      requestHeaders.add(
-          new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, clientProvidedEncryptionKey));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256,
-          clientProvidedEncryptionKeySHA));
-      requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM,
-          SERVER_SIDE_ENCRYPTION_ALGORITHM));
+  private void addEncryptionKeyRequestHeaders(String path,
+      List<AbfsHttpHeader> requestHeaders, boolean isCreateFileRequest,
+      EncryptionAdapter encryptionAdapter, TracingContext tracingContext)
+      throws IOException {
+    String encodedKey, encodedKeySHA256;
+    switch (encryptionType) {
+    case GLOBAL_KEY:
+      encodedKey = clientProvidedEncryptionKey;
+      encodedKeySHA256 = clientProvidedEncryptionKeySHA;
+      break;
+
+    case ENCRYPTION_CONTEXT:
+      if (isCreateFileRequest) {
+        // get new context for create file request
+        SecretKey encryptionContext =
+            encryptionAdapter.fetchEncryptionContextAndComputeKeys();
+        requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT,
+            new String(encryptionContext.getEncoded(),
+                StandardCharsets.UTF_8)));
+        try {
+          encryptionContext.destroy();
+        } catch (DestroyFailedException e) {
+          throw new IOException(
+              "Could not destroy encryptionContext: " + e.getMessage());
+        }
+      } else if (encryptionAdapter == null) {
+        // get encryption context from GetPathStatus response header
+        encryptionAdapter = new EncryptionAdapter(encryptionContextProvider,
+            new Path(path).toUri().getPath(),
+            getPathStatus(path, false, tracingContext).getResult()
+                .getResponseHeader(X_MS_ENCRYPTION_CONTEXT)
+                .getBytes(StandardCharsets.UTF_8));
+      }
+      // else use cached encryption keys from input/output streams
+      encodedKey = encryptionAdapter.getEncodedKey();
+      encodedKeySHA256 = encryptionAdapter.getEncodedKeySHA();
+      break;
+
+    default: return; // no client-provided encryption keys
     }
+    requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey));

Review comment:
       Should there be some kind of validation of the values of the 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.

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

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



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