Browse Source

HADOOP-16986. S3A to not need wildfly on the classpath. (#1948)

Contributed by Steve Loughran.

This is a successor to HADOOP-16346, which enabled the S3A connector
to load the native openssl SSL libraries for better HTTPS performance.

That patch required wildfly.jar to be on the classpath. This
update:

* Makes wildfly.jar optional except in the special case that
"fs.s3a.ssl.channel.mode" is set to "openssl"

* Retains the declaration of wildfly.jar as a compile-time
dependency in the hadoop-aws POM. This means that unless
explicitly excluded, applications importing that published
maven artifact will, transitively, add the specified
wildfly JAR into their classpath for compilation/testing/
distribution.

This is done for packaging and to offer that optional
speedup. It is not mandatory: applications importing
the hadoop-aws POM can exclude it if they choose.

Change-Id: I7ed3e5948d1e10ce21276b3508871709347e113d
Steve Loughran 5 years ago
parent
commit
de9a6b4588

+ 74 - 68
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/DelegatingSSLSocketFactory.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.security.ssl;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.Socket;
 import java.net.Socket;
-import java.net.SocketException;
 import java.security.KeyManagementException;
 import java.security.KeyManagementException;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -31,11 +30,9 @@ import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.SSLSocketFactory;
 import javax.net.ssl.SSLSocketFactory;
 
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
-import org.wildfly.openssl.OpenSSLProvider;
-import org.wildfly.openssl.SSL;
-
 
 
 /**
 /**
  * A {@link SSLSocketFactory} that can delegate to various SSL implementations.
  * A {@link SSLSocketFactory} that can delegate to various SSL implementations.
@@ -60,8 +57,8 @@ import org.wildfly.openssl.SSL;
  * </p>
  * </p>
  *
  *
  * In order to load OpenSSL, applications must ensure the wildfly-openssl
  * In order to load OpenSSL, applications must ensure the wildfly-openssl
- * artifact is on the classpath. Currently, only ABFS and S3A provide
- * wildfly-openssl as a runtime dependency.
+ * artifact is on the classpath. Currently, only ABFS declares
+ * wildfly-openssl as an explicit dependency.
  */
  */
 public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
 public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
 
 
@@ -110,7 +107,16 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
   }
   }
 
 
   /**
   /**
-   * Singletone instance of the SSLSocketFactory.
+   * For testing only: reset the socket factory.
+   */
+  @VisibleForTesting
+  public static synchronized void resetDefaultFactory() {
+    LOG.info("Resetting default SSL Socket Factory");
+    instance = null;
+  }
+
+  /**
+   * Singleton instance of the SSLSocketFactory.
    *
    *
    * SSLSocketFactory must be initialized with appropriate SSLChannelMode
    * SSLSocketFactory must be initialized with appropriate SSLChannelMode
    * using initializeDefaultFactory method.
    * using initializeDefaultFactory method.
@@ -126,9 +132,7 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
       throws IOException {
       throws IOException {
     try {
     try {
       initializeSSLContext(preferredChannelMode);
       initializeSSLContext(preferredChannelMode);
-    } catch (NoSuchAlgorithmException e) {
-      throw new IOException(e);
-    } catch (KeyManagementException e) {
+    } catch (NoSuchAlgorithmException | KeyManagementException e) {
       throw new IOException(e);
       throw new IOException(e);
     }
     }
 
 
@@ -146,42 +150,23 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
   }
   }
 
 
   private void initializeSSLContext(SSLChannelMode preferredChannelMode)
   private void initializeSSLContext(SSLChannelMode preferredChannelMode)
-      throws NoSuchAlgorithmException, KeyManagementException {
+      throws NoSuchAlgorithmException, KeyManagementException, IOException {
+    LOG.debug("Initializing SSL Context to channel mode {}",
+        preferredChannelMode);
     switch (preferredChannelMode) {
     switch (preferredChannelMode) {
     case Default:
     case Default:
-      if (!openSSLProviderRegistered) {
-        OpenSSLProvider.register();
-        openSSLProviderRegistered = true;
-      }
       try {
       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);
+        bindToOpenSSLProvider();
         channelMode = SSLChannelMode.OpenSSL;
         channelMode = SSLChannelMode.OpenSSL;
-      } catch (NoSuchAlgorithmException e) {
-        LOG.debug("Failed to load OpenSSL. Falling back to the JSSE default.");
+      } catch (LinkageError | NoSuchAlgorithmException | RuntimeException e) {
+        LOG.debug("Failed to load OpenSSL. Falling back to the JSSE default.",
+            e);
         ctx = SSLContext.getDefault();
         ctx = SSLContext.getDefault();
         channelMode = SSLChannelMode.Default_JSSE;
         channelMode = SSLChannelMode.Default_JSSE;
       }
       }
       break;
       break;
     case OpenSSL:
     case OpenSSL:
-      if (!openSSLProviderRegistered) {
-        OpenSSLProvider.register();
-        openSSLProviderRegistered = true;
-      }
-      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);
+      bindToOpenSSLProvider();
       channelMode = SSLChannelMode.OpenSSL;
       channelMode = SSLChannelMode.OpenSSL;
       break;
       break;
     case Default_JSSE:
     case Default_JSSE:
@@ -193,11 +178,38 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
       channelMode = SSLChannelMode.Default_JSSE_with_GCM;
       channelMode = SSLChannelMode.Default_JSSE_with_GCM;
       break;
       break;
     default:
     default:
-      throw new NoSuchAlgorithmException("Unknown channel mode: "
+      throw new IOException("Unknown channel mode: "
           + preferredChannelMode);
           + preferredChannelMode);
     }
     }
   }
   }
 
 
+  /**
+   * Bind to the OpenSSL provider via wildfly.
+   * This MUST be the only place where wildfly classes are referenced,
+   * so ensuring that any linkage problems only surface here where they may
+   * be caught by the initialization code.
+   */
+  private void bindToOpenSSLProvider()
+      throws NoSuchAlgorithmException, KeyManagementException {
+    if (!openSSLProviderRegistered) {
+      LOG.debug("Attempting to register OpenSSL provider");
+      org.wildfly.openssl.OpenSSLProvider.register();
+      openSSLProviderRegistered = true;
+    }
+    // Strong reference needs to be kept to logger until initialization of
+    // SSLContext finished (see HADOOP-16174):
+    java.util.logging.Logger logger = java.util.logging.Logger.getLogger(
+        "org.wildfly.openssl.SSL");
+    Level originalLevel = logger.getLevel();
+    try {
+      logger.setLevel(Level.WARNING);
+      ctx = SSLContext.getInstance("openssl.TLS");
+      ctx.init(null, null, null);
+    } finally {
+      logger.setLevel(originalLevel);
+    }
+  }
+
   public String getProviderName() {
   public String getProviderName() {
     return providerName;
     return providerName;
   }
   }
@@ -212,21 +224,26 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
     return ciphers.clone();
     return ciphers.clone();
   }
   }
 
 
+  /**
+   * Get the channel mode of this instance.
+   * @return a channel mode.
+   */
+  public SSLChannelMode getChannelMode() {
+    return channelMode;
+  }
+
   public Socket createSocket() throws IOException {
   public Socket createSocket() throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory.createSocket();
-    configureSocket(ss);
-    return ss;
+    return configureSocket(factory.createSocket());
   }
   }
 
 
   @Override
   @Override
   public Socket createSocket(Socket s, String host, int port,
   public Socket createSocket(Socket s, String host, int port,
                              boolean autoClose) throws IOException {
                              boolean autoClose) throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory.createSocket(s, host, port, autoClose);
 
 
-    configureSocket(ss);
-    return ss;
+    return configureSocket(
+        factory.createSocket(s, host, port, autoClose));
   }
   }
 
 
   @Override
   @Override
@@ -234,52 +251,41 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
                              InetAddress localAddress, int localPort)
                              InetAddress localAddress, int localPort)
       throws IOException {
       throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory
-        .createSocket(address, port, localAddress, localPort);
-
-    configureSocket(ss);
-    return ss;
+    return configureSocket(factory
+        .createSocket(address, port, localAddress, localPort));
   }
   }
 
 
   @Override
   @Override
   public Socket createSocket(String host, int port, InetAddress localHost,
   public Socket createSocket(String host, int port, InetAddress localHost,
                              int localPort) throws IOException {
                              int localPort) throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory
-        .createSocket(host, port, localHost, localPort);
 
 
-    configureSocket(ss);
-
-    return ss;
+    return configureSocket(factory
+        .createSocket(host, port, localHost, localPort));
   }
   }
 
 
   @Override
   @Override
   public Socket createSocket(InetAddress host, int port) throws IOException {
   public Socket createSocket(InetAddress host, int port) throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory.createSocket(host, port);
-
-    configureSocket(ss);
 
 
-    return ss;
+    return configureSocket(factory.createSocket(host, port));
   }
   }
 
 
   @Override
   @Override
   public Socket createSocket(String host, int port) throws IOException {
   public Socket createSocket(String host, int port) throws IOException {
     SSLSocketFactory factory = ctx.getSocketFactory();
     SSLSocketFactory factory = ctx.getSocketFactory();
-    SSLSocket ss = (SSLSocket) factory.createSocket(host, port);
-
-    configureSocket(ss);
 
 
-    return ss;
+    return configureSocket(factory.createSocket(host, port));
   }
   }
 
 
-  private void configureSocket(SSLSocket ss) throws SocketException {
-    ss.setEnabledCipherSuites(ciphers);
+  private Socket configureSocket(Socket socket) {
+    ((SSLSocket) socket).setEnabledCipherSuites(ciphers);
+    return socket;
   }
   }
 
 
   private String[] alterCipherList(String[] defaultCiphers) {
   private String[] alterCipherList(String[] defaultCiphers) {
 
 
-    ArrayList<String> preferredSuits = new ArrayList<>();
+    ArrayList<String> preferredSuites = new ArrayList<>();
 
 
     // Remove GCM mode based ciphers from the supported list.
     // Remove GCM mode based ciphers from the supported list.
     for (int i = 0; i < defaultCiphers.length; i++) {
     for (int i = 0; i < defaultCiphers.length; i++) {
@@ -287,11 +293,11 @@ public final class DelegatingSSLSocketFactory extends SSLSocketFactory {
         LOG.debug("Removed Cipher - {} from list of enabled SSLSocket ciphers",
         LOG.debug("Removed Cipher - {} from list of enabled SSLSocket ciphers",
                 defaultCiphers[i]);
                 defaultCiphers[i]);
       } else {
       } else {
-        preferredSuits.add(defaultCiphers[i]);
+        preferredSuites.add(defaultCiphers[i]);
       }
       }
     }
     }
 
 
-    ciphers = preferredSuits.toArray(new String[0]);
+    ciphers = preferredSuites.toArray(new String[0]);
     return ciphers;
     return ciphers;
   }
   }
-}
+}

+ 18 - 17
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java

@@ -66,26 +66,28 @@ public class NetworkBinding {
    * @param awsConf the {@link ClientConfiguration} to set the
    * @param awsConf the {@link ClientConfiguration} to set the
    *                SSLConnectionSocketFactory for.
    *                SSLConnectionSocketFactory for.
    * @throws IOException if there is an error while initializing the
    * @throws IOException if there is an error while initializing the
-   *                     {@link SSLSocketFactory}.
+   * {@link SSLSocketFactory} other than classloader problems.
    */
    */
   public static void bindSSLChannelMode(Configuration conf,
   public static void bindSSLChannelMode(Configuration conf,
       ClientConfiguration awsConf) throws IOException {
       ClientConfiguration awsConf) throws IOException {
-    try {
-      // Validate that SSL_CHANNEL_MODE is set to a valid value.
-      String channelModeString = conf.get(
-              SSL_CHANNEL_MODE, DEFAULT_SSL_CHANNEL_MODE.name());
-      DelegatingSSLSocketFactory.SSLChannelMode channelMode = null;
-      for (DelegatingSSLSocketFactory.SSLChannelMode mode :
-              DelegatingSSLSocketFactory.SSLChannelMode.values()) {
-        if (mode.name().equalsIgnoreCase(channelModeString)) {
-          channelMode = mode;
-        }
-      }
-      if (channelMode == null) {
-        throw new IllegalArgumentException(channelModeString +
-                " is not a valid value for " + SSL_CHANNEL_MODE);
+
+    // Validate that SSL_CHANNEL_MODE is set to a valid value.
+    String channelModeString = conf.getTrimmed(
+            SSL_CHANNEL_MODE, DEFAULT_SSL_CHANNEL_MODE.name());
+    DelegatingSSLSocketFactory.SSLChannelMode channelMode = null;
+    for (DelegatingSSLSocketFactory.SSLChannelMode mode :
+            DelegatingSSLSocketFactory.SSLChannelMode.values()) {
+      if (mode.name().equalsIgnoreCase(channelModeString)) {
+        channelMode = mode;
       }
       }
+    }
+    if (channelMode == null) {
+      throw new IllegalArgumentException(channelModeString +
+              " is not a valid value for " + SSL_CHANNEL_MODE);
+    }
 
 
+    DelegatingSSLSocketFactory.initializeDefaultFactory(channelMode);
+    try {
       // Look for AWS_SOCKET_FACTORY_CLASSNAME on the classpath and instantiate
       // Look for AWS_SOCKET_FACTORY_CLASSNAME on the classpath and instantiate
       // an instance using the DelegatingSSLSocketFactory as the
       // an instance using the DelegatingSSLSocketFactory as the
       // SSLSocketFactory.
       // SSLSocketFactory.
@@ -94,7 +96,6 @@ public class NetworkBinding {
       Constructor<?> factoryConstructor =
       Constructor<?> factoryConstructor =
               sslConnectionSocketFactory.getDeclaredConstructor(
               sslConnectionSocketFactory.getDeclaredConstructor(
                       SSLSocketFactory.class, HostnameVerifier.class);
                       SSLSocketFactory.class, HostnameVerifier.class);
-      DelegatingSSLSocketFactory.initializeDefaultFactory(channelMode);
       awsConf.getApacheHttpClientConfig().setSslSocketFactory(
       awsConf.getApacheHttpClientConfig().setSslSocketFactory(
               (com.amazonaws.thirdparty.apache.http.conn.ssl.
               (com.amazonaws.thirdparty.apache.http.conn.ssl.
                       SSLConnectionSocketFactory) factoryConstructor
                       SSLConnectionSocketFactory) factoryConstructor
@@ -103,7 +104,7 @@ public class NetworkBinding {
                               (HostnameVerifier) null));
                               (HostnameVerifier) null));
     } catch (ClassNotFoundException | NoSuchMethodException |
     } catch (ClassNotFoundException | NoSuchMethodException |
             IllegalAccessException | InstantiationException |
             IllegalAccessException | InstantiationException |
-            InvocationTargetException e) {
+            InvocationTargetException | LinkageError  e) {
       LOG.debug("Unable to create class {}, value of {} will be ignored",
       LOG.debug("Unable to create class {}, value of {} will be ignored",
               AWS_SOCKET_FACTORY_CLASSNAME, SSL_CHANNEL_MODE, e);
               AWS_SOCKET_FACTORY_CLASSNAME, SSL_CHANNEL_MODE, e);
     }
     }

+ 27 - 9
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md

@@ -539,7 +539,7 @@ in Java 9, so if `default_jsse` is specified and applications run on Java
 includes GCM in the list of cipher suites on Java 8, so it is equivalent to
 includes GCM in the list of cipher suites on Java 8, so it is equivalent to
 running with the vanilla JSSE.
 running with the vanilla JSSE.
 
 
-### OpenSSL Acceleration
+### <a name="openssl"></a> OpenSSL Acceleration
 
 
 **Experimental Feature**
 **Experimental Feature**
 
 
@@ -552,8 +552,8 @@ significant performance benefit over the JSSE.
 S3A uses the
 S3A uses the
 [WildFly OpenSSL](https://github.com/wildfly-security/wildfly-openssl) library
 [WildFly OpenSSL](https://github.com/wildfly-security/wildfly-openssl) library
 to bind OpenSSL to the Java JSSE APIs. This library allows S3A to
 to bind OpenSSL to the Java JSSE APIs. This library allows S3A to
-transparently read data using OpenSSL. The wildfly-openssl library is a
-runtime dependency of S3A and contains native libraries for binding the Java
+transparently read data using OpenSSL. The `wildfly-openssl` library is an
+optional runtime dependency of S3A and contains native libraries for binding the Java
 JSSE to OpenSSL.
 JSSE to OpenSSL.
 
 
 WildFly OpenSSL must load OpenSSL itself. This can be done using the system
 WildFly OpenSSL must load OpenSSL itself. This can be done using the system
@@ -596,19 +596,37 @@ exception and S3A initialization will fail.
 
 
 Supported values for `fs.s3a.ssl.channel.mode`:
 Supported values for `fs.s3a.ssl.channel.mode`:
 
 
-| fs.s3a.ssl.channel.mode Value | Description |
+| `fs.s3a.ssl.channel.mode` Value | Description |
 |-------------------------------|-------------|
 |-------------------------------|-------------|
-| default_jsse | Uses Java JSSE without GCM on Java 8 |
-| default_jsse_with_gcm | Uses Java JSSE |
-| default | Uses OpenSSL, falls back to default_jsse if OpenSSL cannot be loaded |
-| openssl | Uses OpenSSL, fails if OpenSSL cannot be loaded |
+| `default_jsse` | Uses Java JSSE without GCM on Java 8 |
+| `default_jsse_with_gcm` | Uses Java JSSE |
+| `default` | Uses OpenSSL, falls back to `default_jsse` if OpenSSL cannot be loaded |
+| `openssl` | Uses OpenSSL, fails if OpenSSL cannot be loaded |
 
 
 The naming convention is setup in order to preserve backwards compatibility
 The naming convention is setup in order to preserve backwards compatibility
-with HADOOP-15669.
+with the ABFS support of [HADOOP-15669](https://issues.apache.org/jira/browse/HADOOP-15669).
 
 
 Other options may be added to `fs.s3a.ssl.channel.mode` in the future as
 Other options may be added to `fs.s3a.ssl.channel.mode` in the future as
 further SSL optimizations are made.
 further SSL optimizations are made.
 
 
+### WildFly classpath requirements
+
+For OpenSSL acceleration to work, a compatible version of the
+wildfly JAR must be on the classpath. This is not explicitly declared
+in the dependencies of the published `hadoop-aws` module, as it is
+optional.
+
+If the wildfly JAR is not found, the network acceleration will fall back
+to the JVM, always.
+
+Note: there have been compatibility problems with wildfly JARs and openSSL
+releases in the past: version 1.0.4.Final is not compatible with openssl 1.1.1.
+An extra complication was older versions of the `azure-data-lake-store-sdk`
+JAR used in `hadoop-azure-datalake` contained an unshaded copy of the 1.0.4.Final
+classes, causing binding problems even when a later version was explicitly
+being placed on the classpath.
+
+
 ## Tuning FileSystem Initialization.
 ## Tuning FileSystem Initialization.
 
 
 When an S3A Filesystem instance is created and initialized, the client
 When an S3A Filesystem instance is created and initialized, the client

+ 13 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

@@ -92,6 +92,19 @@ classpath, do not add any of the `aws-sdk-` JARs.
 This happens if the `hadoop-aws` and `hadoop-common` JARs are out of sync.
 This happens if the `hadoop-aws` and `hadoop-common` JARs are out of sync.
 You can't mix them around: they have to have exactly matching version numbers.
 You can't mix them around: they have to have exactly matching version numbers.
 
 
+### `java.lang.NoClassDefFoundError: org/wildfly/openssl/OpenSSLProvider`
+
+This happens when <a href="performance.html#openssl">OpenSSL performance
+acceleration</a> has been configured by setting `fs.s3a.ssl.channel.mode`
+to `openssl` but the wildfly JAR is not on the classpath.
+
+Fixes:
+* Add it to the classpath
+* Use a different channel mode, including `default`, which will
+revert to the JVM SSL implementation when the wildfly
+or native openssl libraries cannot be loaded.
+
+
 ## <a name="authentication"></a> Authentication Failure
 ## <a name="authentication"></a> Authentication Failure
 
 
 If Hadoop cannot authenticate with the S3 service endpoint,
 If Hadoop cannot authenticate with the S3 service endpoint,

+ 152 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java

@@ -0,0 +1,152 @@
+/*
+ * 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 com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.Constants.SSL_CHANNEL_MODE;
+import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.bindSSLChannelMode;
+import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory.SSLChannelMode.Default;
+import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE;
+import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE_with_GCM;
+import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory.SSLChannelMode.OpenSSL;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
+
+/**
+ * Make sure that wildfly is not on this classpath and that we can still
+ * create connections in the default option, but that openssl fails.
+ * This test suite is designed to work whether or not wildfly JAR is on
+ * the classpath, and when openssl native libraries are/are not
+ * on the path.
+ * Some of the tests are skipped in a maven build because wildfly
+ * is always on the classpath -but they are retained as in-IDE
+ * runs may be different, and if wildfly is removed from
+ * the compile or test CP then different test cases will execute.
+ */
+public class TestWildflyAndOpenSSLBinding extends AbstractHadoopTestBase {
+
+  /** Was wildfly found. */
+  private boolean hasWildfly;
+
+  @Before
+  public void setup() throws Exception {
+    // determine whether or not wildfly is on the classpath
+    ClassLoader loader = this.getClass().getClassLoader();
+    try {
+      loader.loadClass("org.wildfly.openssl.OpenSSLProvider");
+      hasWildfly = true;
+    } catch (ClassNotFoundException e) {
+      hasWildfly = false;
+    }
+  }
+
+
+  @Test
+  public void testUnknownMode() throws Throwable {
+    DelegatingSSLSocketFactory.resetDefaultFactory();
+    Configuration conf = new Configuration(false);
+    conf.set(SSL_CHANNEL_MODE, "no-such-mode ");
+    intercept(IllegalArgumentException.class, () ->
+        bindSSLChannelMode(conf, new ClientConfiguration()));
+  }
+
+  @Test
+  public void testOpenSSLNoWildfly() throws Throwable {
+    assumeThat(hasWildfly).isFalse();
+    intercept(NoClassDefFoundError.class, "wildfly", () ->
+      bindSocketFactory(OpenSSL));
+  }
+
+  /**
+   * If there is no WF on the CP, then we always downgrade
+   * to default.
+   */
+  @Test
+  public void testDefaultDowngradesNoWildfly() throws Throwable {
+    assumeThat(hasWildfly).isFalse();
+    expectBound(Default, Default_JSSE);
+  }
+
+  /**
+   * Wildfly is on the CP; if openssl native is on the
+   * path then openssl will load, otherwise JSSE.
+   */
+  @Test
+  public void testWildflyOpenSSL() throws Throwable {
+    assumeThat(hasWildfly).isTrue();
+    assertThat(bindSocketFactory(Default))
+        .describedAs("Sockets from mode " + Default)
+        .isIn(OpenSSL, Default_JSSE);
+  }
+
+  @Test
+  public void testJSSE() throws Throwable {
+    expectBound(Default_JSSE, Default_JSSE);
+  }
+
+  @Test
+  public void testGCM() throws Throwable {
+    expectBound(Default_JSSE_with_GCM, Default_JSSE_with_GCM);
+  }
+
+  /**
+   * Bind to a socket mode and verify that the result matches
+   * that expected -which does not have to be the one requested.
+   * @param channelMode mode to use
+   * @param finalMode mode to test for
+   */
+  private void expectBound(
+      DelegatingSSLSocketFactory.SSLChannelMode channelMode,
+      DelegatingSSLSocketFactory.SSLChannelMode finalMode)
+      throws Throwable {
+    assertThat(bindSocketFactory(channelMode))
+        .describedAs("Channel mode of socket factory created with mode %s",
+            channelMode)
+        .isEqualTo(finalMode);
+  }
+
+  /**
+   * Bind the socket factory to a given channel mode.
+   * @param channelMode mode to use
+   * @return the actual channel mode.
+   */
+  private DelegatingSSLSocketFactory.SSLChannelMode bindSocketFactory(
+      final DelegatingSSLSocketFactory.SSLChannelMode channelMode)
+      throws IOException {
+    DelegatingSSLSocketFactory.resetDefaultFactory();
+    Configuration conf = new Configuration(false);
+    conf.set(SSL_CHANNEL_MODE, channelMode.name());
+    ClientConfiguration awsConf = new ClientConfiguration();
+    awsConf.setProtocol(Protocol.HTTPS);
+    bindSSLChannelMode(conf, awsConf);
+    return DelegatingSSLSocketFactory.getDefaultFactory().getChannelMode();
+  }
+
+}