Quellcode durchsuchen

Revert "HADOOP-16050: s3a SSL connections should use OpenSSL"

This reverts commit b067f8acaa79b1230336900a5c62ba465b2adb28.

Change-Id: I584b050a56c0e6f70b11fa3f7db00d5ac46e7dd8
Steve Loughran vor 6 Jahren
Ursprung
Commit
309501c6fa

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

@@ -343,16 +343,6 @@
       <artifactId>dnsjava</artifactId>
       <scope>compile</scope>
     </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>
 
   <build>

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

@@ -1,53 +0,0 @@
-/*
- * 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);
-  }
-}

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

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

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 
 import java.util.concurrent.TimeUnit;
 
@@ -148,11 +147,6 @@ public final class Constants {
       "fs.s3a.connection.ssl.enabled";
   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?
   public static final String ENDPOINT = "fs.s3a.endpoint";
 

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

@@ -34,7 +34,6 @@ import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 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.base.Preconditions;
@@ -52,7 +51,6 @@ import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.ProviderUtils;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.util.VersionInfo;
 
 import com.google.common.collect.Lists;
@@ -60,7 +58,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
-import javax.net.ssl.HostnameVerifier;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
@@ -1254,15 +1251,14 @@ public final class S3AUtils {
    *
    * @param conf Hadoop configuration
    * @param awsConf AWS SDK configuration
-   *
-   * @throws IOException if there was an error initializing the protocol
-   *                     settings
    */
   public static void initConnectionSettings(Configuration conf,
-      ClientConfiguration awsConf) throws IOException {
+      ClientConfiguration awsConf) {
     awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
         DEFAULT_MAXIMUM_CONNECTIONS, 1));
-    initProtocolSettings(conf, awsConf);
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+        DEFAULT_SECURE_CONNECTIONS);
+    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
     awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
         DEFAULT_MAX_ERROR_RETRIES, 0));
     awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
@@ -1281,32 +1277,6 @@ 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
    * if the S3A settings enable it.

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

@@ -1,72 +0,0 @@
-/*
- * 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>
       <groupId>org.wildfly.openssl</groupId>
       <artifactId>wildfly-openssl</artifactId>
-      <scope>runtime</scope>
+      <scope>compile</scope>
     </dependency>
 
     <!--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.KeyProvider;
 import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -435,7 +435,7 @@ public class AbfsConfiguration{
     return this.userAgentId;
   }
 
-  public OpenSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption() {
+  public SSLSocketFactoryEx.SSLChannelMode getPreferredSSLFactoryOption() {
     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.InterfaceStability;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 
 /**
  * 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_AUTOTHROTTLING = true;
 
-  public static final OpenSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
-      = OpenSSLSocketFactory.SSLChannelMode.Default;
+  public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
+      = SSLSocketFactoryEx.SSLChannelMode.Default;
 
   public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false;
   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 com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
@@ -79,10 +79,10 @@ public class AbfsClient implements Closeable {
 
     if (this.baseUrl.toString().startsWith(HTTPS_SCHEME)) {
       try {
-        OpenSSLSocketFactory.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption());
-        sslProviderName = OpenSSLSocketFactory.getDefaultFactory().getProviderName();
+        SSLSocketFactoryEx.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption());
+        sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();
       } catch (IOException e) {
-        // Suppress exception. Failure to init OpenSSLSocketFactory would have only performance impact.
+        // Suppress exception. Failure to init SSLSocketFactoryEx 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.SSLSocketFactory;
 
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonToken;
@@ -180,7 +180,7 @@ public class AbfsHttpOperation {
     this.connection = openConnection();
     if (this.connection instanceof HttpsURLConnection) {
       HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
-      SSLSocketFactory sslSocketFactory = OpenSSLSocketFactory.getDefaultFactory();
+      SSLSocketFactory sslSocketFactory = SSLSocketFactoryEx.getDefaultFactory();
       if (sslSocketFactory != null) {
         secureConn.setSSLSocketFactory(sslSocketFactory);
       }

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

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

+ 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 org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.junit.Test;
 
 /**
@@ -161,19 +161,19 @@ public class TestAbfsConfigurationFieldsValidation {
   @Test
   public void testSSLSocketFactoryConfiguration()
       throws InvalidConfigurationValueException, IllegalAccessException, IOException {
-    assertEquals(OpenSSLSocketFactory.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption());
-    assertNotEquals(OpenSSLSocketFactory.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption());
-    assertNotEquals(OpenSSLSocketFactory.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption());
+    assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption());
 
     Configuration configuration = new Configuration();
-    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, OpenSSLSocketFactory.SSLChannelMode.Default_JSSE);
+    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.Default_JSSE);
     AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    assertEquals(OpenSSLSocketFactory.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption());
 
     configuration = new Configuration();
-    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, OpenSSLSocketFactory.SSLChannelMode.OpenSSL);
+    configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.OpenSSL);
     localAbfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    assertEquals(OpenSSLSocketFactory.SSLChannelMode.OpenSSL, localAbfsConfiguration.getPreferredSSLFactoryOption());
+    assertEquals(SSLSocketFactoryEx.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.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.security.ssl.OpenSSLSocketFactory;
 import org.apache.hadoop.util.VersionInfo;
 
 /**
@@ -46,7 +46,7 @@ public final class TestAbfsClient {
         config, null, null);
     String sslProviderName = null;
     if (includeSSLProvider) {
-      sslProviderName = OpenSSLSocketFactory.getDefaultFactory().getProviderName();
+      sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();
     }
     String userAgent = client.initializeUserAgent(config, sslProviderName);
     Pattern pattern = Pattern.compile(expectedPattern);
@@ -86,7 +86,7 @@ public final class TestAbfsClient {
     final Configuration configuration = new Configuration();
     configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
     configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
-        OpenSSLSocketFactory.SSLChannelMode.Default_JSSE.name());
+        SSLSocketFactoryEx.SSLChannelMode.Default_JSSE.name());
     AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
     validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"),
         abfsConfiguration, true);