Przeglądaj źródła

HADOOP-16050: s3a SSL connections should use OpenSSL

(cherry picked from commit aebf229c175dfa19fff3b31e9e67596f6c6124fa)
Sahil Takiar 6 lat temu
rodzic
commit
b067f8acaa

+ 10 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -334,6 +334,16 @@
       <artifactId>dnsjava</artifactId>
       <artifactId>dnsjava</artifactId>
       <scope>compile</scope>
       <scope>compile</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.wildfly.openssl</groupId>
+      <artifactId>wildfly-openssl</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   </dependencies>
 
 
   <build>
   <build>

+ 32 - 30
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/OpenSSLSocketFactory.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-package org.apache.hadoop.fs.azurebfs.utils;
+package org.apache.hadoop.security.ssl;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
@@ -42,11 +42,11 @@ import org.wildfly.openssl.SSL;
  * performance.
  * performance.
  *
  *
  */
  */
-public final class SSLSocketFactoryEx extends SSLSocketFactory {
+public final class OpenSSLSocketFactory extends SSLSocketFactory {
 
 
   /**
   /**
    * Default indicates Ordered, preferred OpenSSL, if failed to load then fall
    * Default indicates Ordered, preferred OpenSSL, if failed to load then fall
-   * back to Default_JSSE
+   * back to Default_JSSE.
    */
    */
   public enum SSLChannelMode {
   public enum SSLChannelMode {
     OpenSSL,
     OpenSSL,
@@ -54,9 +54,9 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
     Default_JSSE
     Default_JSSE
   }
   }
 
 
-  private static SSLSocketFactoryEx instance = null;
+  private static OpenSSLSocketFactory instance = null;
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
-      SSLSocketFactoryEx.class);
+      OpenSSLSocketFactory.class);
   private String providerName;
   private String providerName;
   private SSLContext ctx;
   private SSLContext ctx;
   private String[] ciphers;
   private String[] ciphers;
@@ -71,7 +71,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
   public static synchronized void initializeDefaultFactory(
   public static synchronized void initializeDefaultFactory(
       SSLChannelMode preferredMode) throws IOException {
       SSLChannelMode preferredMode) throws IOException {
     if (instance == null) {
     if (instance == null) {
-      instance = new SSLSocketFactoryEx(preferredMode);
+      instance = new OpenSSLSocketFactory(preferredMode);
     }
     }
   }
   }
 
 
@@ -84,7 +84,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
    * @return instance of the SSLSocketFactory, instance must be initialized by
    * @return instance of the SSLSocketFactory, instance must be initialized by
    * initializeDefaultFactory.
    * initializeDefaultFactory.
    */
    */
-  public static SSLSocketFactoryEx getDefaultFactory() {
+  public static OpenSSLSocketFactory getDefaultFactory() {
     return instance;
     return instance;
   }
   }
 
 
@@ -92,7 +92,7 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
     OpenSSLProvider.register();
     OpenSSLProvider.register();
   }
   }
 
 
-  private SSLSocketFactoryEx(SSLChannelMode preferredChannelMode)
+  private OpenSSLSocketFactory(SSLChannelMode preferredChannelMode)
       throws IOException {
       throws IOException {
     try {
     try {
       initializeSSLContext(preferredChannelMode);
       initializeSSLContext(preferredChannelMode);
@@ -118,33 +118,35 @@ public final class SSLSocketFactoryEx extends SSLSocketFactory {
   private void initializeSSLContext(SSLChannelMode preferredChannelMode)
   private void initializeSSLContext(SSLChannelMode preferredChannelMode)
       throws NoSuchAlgorithmException, KeyManagementException {
       throws NoSuchAlgorithmException, KeyManagementException {
     switch (preferredChannelMode) {
     switch (preferredChannelMode) {
-      case Default:
-        try {
-          java.util.logging.Logger logger = java.util.logging.Logger.getLogger(SSL.class.getName());
-          logger.setLevel(Level.WARNING);
-          ctx = SSLContext.getInstance("openssl.TLS");
-          ctx.init(null, null, null);
-          // Strong reference needs to be kept to logger until initialization of SSLContext finished (see HADOOP-16174):
-          logger.setLevel(Level.INFO);
-          channelMode = SSLChannelMode.OpenSSL;
-        } catch (NoSuchAlgorithmException e) {
-          LOG.warn("Failed to load OpenSSL. Falling back to the JSSE default.");
-          ctx = SSLContext.getDefault();
-          channelMode = SSLChannelMode.Default_JSSE;
-        }
-        break;
-      case OpenSSL:
+    case Default:
+      try {
+        java.util.logging.Logger logger = java.util.logging.Logger.getLogger(
+                SSL.class.getName());
+        logger.setLevel(Level.WARNING);
         ctx = SSLContext.getInstance("openssl.TLS");
         ctx = SSLContext.getInstance("openssl.TLS");
         ctx.init(null, null, null);
         ctx.init(null, null, null);
+        // Strong reference needs to be kept to logger until initialization of
+        // SSLContext finished (see HADOOP-16174):
+        logger.setLevel(Level.INFO);
         channelMode = SSLChannelMode.OpenSSL;
         channelMode = SSLChannelMode.OpenSSL;
-        break;
-      case Default_JSSE:
+      } catch (NoSuchAlgorithmException e) {
+        LOG.warn("Failed to load OpenSSL. Falling back to the JSSE default.");
         ctx = SSLContext.getDefault();
         ctx = SSLContext.getDefault();
         channelMode = SSLChannelMode.Default_JSSE;
         channelMode = SSLChannelMode.Default_JSSE;
-        break;
-      default:
-        throw new AssertionError("Unknown channel mode: "
-            + preferredChannelMode);
+      }
+      break;
+    case OpenSSL:
+      ctx = SSLContext.getInstance("openssl.TLS");
+      ctx.init(null, null, null);
+      channelMode = SSLChannelMode.OpenSSL;
+      break;
+    case Default_JSSE:
+      ctx = SSLContext.getDefault();
+      channelMode = SSLChannelMode.Default_JSSE;
+      break;
+    default:
+      throw new AssertionError("Unknown channel mode: "
+          + preferredChannelMode);
     }
     }
   }
   }
 
 

+ 53 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestOpenSSLSocketFactory.java

@@ -0,0 +1,53 @@
+/*
+ * 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.security.ssl;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests for {@link OpenSSLSocketFactory}.
+ */
+public class TestOpenSSLSocketFactory {
+
+  @Test
+  public void testOpenSSL() throws IOException {
+    assumeTrue(NativeCodeLoader.buildSupportsOpenssl());
+    OpenSSLSocketFactory.initializeDefaultFactory(
+            OpenSSLSocketFactory.SSLChannelMode.OpenSSL);
+    assertThat(OpenSSLSocketFactory.getDefaultFactory()
+            .getProviderName()).contains("openssl");
+  }
+
+  @Test
+  public void testJSEEJava8() throws IOException {
+    assumeTrue(System.getProperty("java.version").startsWith("1.8"));
+    OpenSSLSocketFactory.initializeDefaultFactory(
+            OpenSSLSocketFactory.SSLChannelMode.Default_JSSE);
+    assertThat(Arrays.stream(OpenSSLSocketFactory.getDefaultFactory()
+            .getSupportedCipherSuites())).noneMatch("GCM"::contains);
+  }
+}

+ 5 - 0
hadoop-tools/hadoop-aws/pom.xml

@@ -417,6 +417,11 @@
       <artifactId>aws-java-sdk-bundle</artifactId>
       <artifactId>aws-java-sdk-bundle</artifactId>
       <scope>compile</scope>
       <scope>compile</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.wildfly.openssl</groupId>
+      <artifactId>wildfly-openssl</artifactId>
+      <scope>runtime</scope>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>junit</groupId>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <artifactId>junit</artifactId>

+ 6 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 
 
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
@@ -147,6 +148,11 @@ public final class Constants {
       "fs.s3a.connection.ssl.enabled";
       "fs.s3a.connection.ssl.enabled";
   public static final boolean DEFAULT_SECURE_CONNECTIONS = true;
   public static final boolean DEFAULT_SECURE_CONNECTIONS = true;
 
 
+  // use OpenSSL or JSEE for secure connections
+  public static final String SSL_CHANNEL_MODE =  "fs.s3a.ssl.channel.mode";
+  public static final OpenSSLSocketFactory.SSLChannelMode
+      DEFAULT_SSL_CHANNEL_MODE = OpenSSLSocketFactory.SSLChannelMode.Default;
+
   //use a custom endpoint?
   //use a custom endpoint?
   public static final String ENDPOINT = "fs.s3a.endpoint";
   public static final String ENDPOINT = "fs.s3a.endpoint";
 
 

+ 34 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -34,6 +34,7 @@ import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -51,6 +52,7 @@ import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -58,6 +60,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import javax.annotation.Nullable;
 import javax.annotation.Nullable;
+import javax.net.ssl.HostnameVerifier;
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
@@ -1242,14 +1245,15 @@ public final class S3AUtils {
    *
    *
    * @param conf Hadoop configuration
    * @param conf Hadoop configuration
    * @param awsConf AWS SDK configuration
    * @param awsConf AWS SDK configuration
+   *
+   * @throws IOException if there was an error initializing the protocol
+   *                     settings
    */
    */
   public static void initConnectionSettings(Configuration conf,
   public static void initConnectionSettings(Configuration conf,
-      ClientConfiguration awsConf) {
+      ClientConfiguration awsConf) throws IOException {
     awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
     awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
         DEFAULT_MAXIMUM_CONNECTIONS, 1));
         DEFAULT_MAXIMUM_CONNECTIONS, 1));
-    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
-        DEFAULT_SECURE_CONNECTIONS);
-    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
+    initProtocolSettings(conf, awsConf);
     awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
     awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
         DEFAULT_MAX_ERROR_RETRIES, 0));
         DEFAULT_MAX_ERROR_RETRIES, 0));
     awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
     awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
@@ -1268,6 +1272,32 @@ public final class S3AUtils {
     }
     }
   }
   }
 
 
+  /**
+   * Initializes the connection protocol settings when connecting to S3 (e.g.
+   * either HTTP or HTTPS). If secure connections are enabled, this method
+   * will load the configured SSL providers.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   *
+   * @throws IOException if there is an error initializing the configured
+   *                     {@link javax.net.ssl.SSLSocketFactory}
+   */
+  private static void initProtocolSettings(Configuration conf,
+      ClientConfiguration awsConf) throws IOException {
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+        DEFAULT_SECURE_CONNECTIONS);
+    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
+    if (secureConnections) {
+      OpenSSLSocketFactory.initializeDefaultFactory(
+              conf.getEnum(SSL_CHANNEL_MODE, DEFAULT_SSL_CHANNEL_MODE));
+      awsConf.getApacheHttpClientConfig().setSslSocketFactory(
+            new SSLConnectionSocketFactory(
+                    OpenSSLSocketFactory.getDefaultFactory(),
+                    (HostnameVerifier) null));
+    }
+  }
+
   /**
   /**
    * Initializes AWS SDK proxy support in the AWS client configuration
    * Initializes AWS SDK proxy support in the AWS client configuration
    * if the S3A settings enable it.
    * if the S3A settings enable it.

+ 72 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ASSL.java

@@ -0,0 +1,72 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests non-default values for {@link Constants#SSL_CHANNEL_MODE}.
+ */
+public class ITestS3ASSL extends AbstractS3ATestBase {
+
+  @Test
+  public void testOpenSSL() throws IOException {
+    assumeTrue(NativeCodeLoader.buildSupportsOpenssl());
+    Configuration conf = new Configuration(getConfiguration());
+    conf.setEnum(Constants.SSL_CHANNEL_MODE,
+            OpenSSLSocketFactory.SSLChannelMode.OpenSSL);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      writeThenReadFile(fs, path("ITestS3ASSL/testOpenSSL"));
+    }
+  }
+
+  @Test
+  public void testJSEE() throws IOException {
+    Configuration conf = new Configuration(getConfiguration());
+    conf.setEnum(Constants.SSL_CHANNEL_MODE,
+            OpenSSLSocketFactory.SSLChannelMode.Default_JSSE);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      writeThenReadFile(fs, path("ITestS3ASSL/testJSEE"));
+    }
+  }
+
+  /**
+   * Helper function that writes and then reads a file. Unlike
+   * {@link #writeThenReadFile(Path, int)} it takes a {@link FileSystem} as a
+   * parameter.
+   */
+  private void writeThenReadFile(FileSystem fs, Path path) throws IOException {
+    byte[] data = dataset(1024, 'a', 'z');
+    writeDataset(fs, path, data, data.length, 1024, true);
+    ContractTestUtils.verifyFileContents(fs, path, data);
+  }
+}

+ 1 - 1
hadoop-tools/hadoop-azure/pom.xml

@@ -194,7 +194,7 @@
     <dependency>
     <dependency>
       <groupId>org.wildfly.openssl</groupId>
       <groupId>org.wildfly.openssl</groupId>
       <artifactId>wildfly-openssl</artifactId>
       <artifactId>wildfly-openssl</artifactId>
-      <scope>compile</scope>
+      <scope>runtime</scope>
     </dependency>
     </dependency>
 
 
     <!--com.fasterxml.jackson is used by WASB, not ABFS-->
     <!--com.fasterxml.jackson is used by WASB, not ABFS-->

+ 2 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -56,7 +56,7 @@ import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
 import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
 import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
 import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
@@ -435,7 +435,7 @@ public class AbfsConfiguration{
     return this.userAgentId;
     return this.userAgentId;
   }
   }
 
 
-  public SSLSocketFactoryEx.SSLChannelMode getPreferredSSLFactoryOption() {
+  public OpenSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption() {
     return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
     return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
   }
   }
 
 

+ 3 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.fs.azurebfs.constants;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 
 
 /**
 /**
  * Responsible to keep all the Azure Blob File System related configurations.
  * Responsible to keep all the Azure Blob File System related configurations.
@@ -59,8 +59,8 @@ public final class FileSystemConfigurations {
   public static final boolean DEFAULT_ENABLE_FLUSH = true;
   public static final boolean DEFAULT_ENABLE_FLUSH = true;
   public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true;
   public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true;
 
 
-  public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
-      = SSLSocketFactoryEx.SSLChannelMode.Default;
+  public static final OpenSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
+      = OpenSSLSocketFactory.SSLChannelMode.Default;
 
 
   public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false;
   public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false;
   public static final boolean DEFAULT_ENABLE_HTTPS = true;
   public static final boolean DEFAULT_ENABLE_HTTPS = true;

+ 4 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -29,7 +29,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Locale;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
@@ -79,10 +79,10 @@ public class AbfsClient implements Closeable {
 
 
     if (this.baseUrl.toString().startsWith(HTTPS_SCHEME)) {
     if (this.baseUrl.toString().startsWith(HTTPS_SCHEME)) {
       try {
       try {
-        SSLSocketFactoryEx.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption());
-        sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();
+        OpenSSLSocketFactory.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption());
+        sslProviderName = OpenSSLSocketFactory.getDefaultFactory().getProviderName();
       } catch (IOException e) {
       } catch (IOException e) {
-        // Suppress exception. Failure to init SSLSocketFactoryEx would have only performance impact.
+        // Suppress exception. Failure to init OpenSSLSocketFactory would have only performance impact.
       }
       }
     }
     }
 
 

+ 2 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java

@@ -29,7 +29,7 @@ import java.util.UUID;
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.SSLSocketFactory;
 import javax.net.ssl.SSLSocketFactory;
 
 
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonToken;
 import org.codehaus.jackson.JsonToken;
@@ -180,7 +180,7 @@ public class AbfsHttpOperation {
     this.connection = openConnection();
     this.connection = openConnection();
     if (this.connection instanceof HttpsURLConnection) {
     if (this.connection instanceof HttpsURLConnection) {
       HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
       HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
-      SSLSocketFactory sslSocketFactory = SSLSocketFactoryEx.getDefaultFactory();
+      SSLSocketFactory sslSocketFactory = OpenSSLSocketFactory.getDefaultFactory();
       if (sslSocketFactory != null) {
       if (sslSocketFactory != null) {
         secureConn.setSSLSocketFactory(sslSocketFactory);
         secureConn.setSSLSocketFactory(sslSocketFactory);
       }
       }

+ 8 - 8
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java

@@ -47,7 +47,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotEquals;
 
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -161,19 +161,19 @@ public class TestAbfsConfigurationFieldsValidation {
   @Test
   @Test
   public void testSSLSocketFactoryConfiguration()
   public void testSSLSocketFactoryConfiguration()
       throws InvalidConfigurationValueException, IllegalAccessException, IOException {
       throws InvalidConfigurationValueException, IllegalAccessException, IOException {
-    assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption());
-    assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption());
-    assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(OpenSSLSocketFactory.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertNotEquals(OpenSSLSocketFactory.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertNotEquals(OpenSSLSocketFactory.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption());
 
 
     Configuration configuration = new Configuration();
     Configuration configuration = new Configuration();
-    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.Default_JSSE);
+    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, OpenSSLSocketFactory.SSLChannelMode.Default_JSSE);
     AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
     AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(OpenSSLSocketFactory.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption());
 
 
     configuration = new Configuration();
     configuration = new Configuration();
-    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.OpenSSL);
+    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, OpenSSLSocketFactory.SSLChannelMode.OpenSSL);
     localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
     localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    assertEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, localAbfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(OpenSSLSocketFactory.SSLChannelMode.OpenSSL, localAbfsConfiguration.getPreferredSSLFactoryOption());
   }
   }
 
 
 }
 }

+ 3 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java

@@ -25,9 +25,9 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
 /**
 /**
@@ -46,7 +46,7 @@ public final class TestAbfsClient {
         config, null, null);
         config, null, null);
     String sslProviderName = null;
     String sslProviderName = null;
     if (includeSSLProvider) {
     if (includeSSLProvider) {
-      sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();
+      sslProviderName = OpenSSLSocketFactory.getDefaultFactory().getProviderName();
     }
     }
     String userAgent = client.initializeUserAgent(config, sslProviderName);
     String userAgent = client.initializeUserAgent(config, sslProviderName);
     Pattern pattern = Pattern.compile(expectedPattern);
     Pattern pattern = Pattern.compile(expectedPattern);
@@ -86,7 +86,7 @@ public final class TestAbfsClient {
     final Configuration configuration = new Configuration();
     final Configuration configuration = new Configuration();
     configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
     configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
     configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
     configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
-        SSLSocketFactoryEx.SSLChannelMode.Default_JSSE.name());
+        OpenSSLSocketFactory.SSLChannelMode.Default_JSSE.name());
     AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
     AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
     validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"),
     validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"),
         abfsConfiguration, true);
         abfsConfiguration, true);