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

ASF GitHub Bot commented on HADOOP-18708:
-----------------------------------------

shameersss1 commented on code in PR #6884:
URL: https://github.com/apache/hadoop/pull/6884#discussion_r1673994265


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/EncryptionS3ClientFactory.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.encryption.s3.S3AsyncEncryptionClient;
+import software.amazon.encryption.s3.S3EncryptionClient;
+import software.amazon.encryption.s3.materials.CryptographicMaterialsManager;
+import software.amazon.encryption.s3.materials.DefaultCryptoMaterialsManager;
+import software.amazon.encryption.s3.materials.Keyring;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.impl.CSEMaterials;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import static 
org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unavailable;
+
+public class EncryptionS3ClientFactory extends DefaultS3ClientFactory {
+
+  private static final String ENCRYPTION_CLIENT_CLASSNAME =
+      "software.amazon.encryption.s3.S3EncryptionClient";
+
+  /**
+   * Encryption client availability.
+   */
+  private static final boolean ENCRYPTION_CLIENT_FOUND = 
checkForEncryptionClient();
+
+  /**
+   * S3Client to be wrapped by encryption client.
+   */
+  private S3Client s3Client;
+
+  /**
+   * S3AsyncClient to be wrapped by encryption client.
+   */
+  private S3AsyncClient s3AsyncClient;
+
+  private static boolean checkForEncryptionClient() {
+    try {
+      ClassLoader cl = EncryptionS3ClientFactory.class.getClassLoader();
+      cl.loadClass(ENCRYPTION_CLIENT_CLASSNAME);
+      LOG.debug("encryption client class {} found", 
ENCRYPTION_CLIENT_CLASSNAME);
+      return true;
+    } catch (Exception e) {
+      LOG.debug("encryption client class {} not found", 
ENCRYPTION_CLIENT_CLASSNAME, e);
+      return false;
+    }
+  }
+
+  /**
+   * Is the Encryption client available?
+   * @return true if it was found in the classloader
+   */
+  private static synchronized boolean isEncryptionClientAvailable() {
+    return ENCRYPTION_CLIENT_FOUND;
+  }
+
+  /**
+   * Create encrypted s3 client.
+   * @param uri S3A file system URI
+   * @param parameters parameter object
+   * @return encrypted s3 client
+   * @throws IOException IO failures
+   */
+  @Override
+  public S3Client createS3Client(URI uri, S3ClientCreationParameters 
parameters)
+      throws IOException {
+    if (!isEncryptionClientAvailable()) {
+      throw unavailable(uri, ENCRYPTION_CLIENT_CLASSNAME, null,
+          "No encryption client available");
+    }
+
+    s3Client = super.createS3Client(uri, parameters);
+    s3AsyncClient = super.createS3AsyncClient(uri, parameters);
+
+    return 
createS3EncryptionClient(parameters.getClientSideEncryptionMaterials());
+  }
+
+  /**
+   * Create async encrypted s3 client.

Review Comment:
   unlike createS3EncryptionClient, createS3AsyncEncryptionClient only wraps 
async client





> AWS SDK V2 - Implement CSE
> --------------------------
>
>                 Key: HADOOP-18708
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18708
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.0
>            Reporter: Ahmar Suhail
>            Assignee: Syed Shameerur Rahman
>            Priority: Major
>              Labels: pull-request-available
>
> S3 Encryption client for SDK V2 is now available, so add client side 
> encryption back in. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to