You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ta...@apache.org on 2018/08/02 20:42:57 UTC

[2/2] qpid-jms git commit: QPIDJMS-391 Add support for netty OpenSSL implementations

QPIDJMS-391 Add support for netty OpenSSL implementations

Allow the client to utilize netty openssl features if available and
configured to do so.  The user will need to provide a netty-tcnative
dependency to the classpath such as the boringssl uber jar and set
the transport.useOpenSSL option to true.

The tests use the boringssl library v2.0.12.Final to validate.

Updates to Netty 4.1.28.Final which has some OpenSSL bindings fixes.


Project: http://git-wip-us.apache.org/repos/asf/qpid-jms/repo
Commit: http://git-wip-us.apache.org/repos/asf/qpid-jms/commit/d5cde3ed
Tree: http://git-wip-us.apache.org/repos/asf/qpid-jms/tree/d5cde3ed
Diff: http://git-wip-us.apache.org/repos/asf/qpid-jms/diff/d5cde3ed

Branch: refs/heads/master
Commit: d5cde3ed4de43d5c2462930a068da15afa916d1d
Parents: 994c5b4
Author: Timothy Bish <ta...@gmail.com>
Authored: Thu Aug 2 16:35:58 2018 -0400
Committer: Timothy Bish <ta...@gmail.com>
Committed: Thu Aug 2 16:35:58 2018 -0400

----------------------------------------------------------------------
 pom.xml                                         |   9 +-
 qpid-jms-client/pom.xml                         |   5 +
 .../qpid/jms/transports/TransportOptions.java   |  18 +
 .../qpid/jms/transports/TransportSupport.java   | 216 +++++--
 .../jms/transports/netty/NettyTcpTransport.java |  25 +-
 .../jms/integration/SaslIntegrationTest.java    |   4 +-
 .../jms/integration/SslIntegrationTest.java     |  97 ++-
 ...qpOpenProvidedServerListIntegrationTest.java |  14 +-
 .../jms/transports/TransportSupportTest.java    | 633 +++++++++++++++++--
 .../netty/NettyOpenSSLWssTransportTest.java     |  42 ++
 .../netty/NettyOpenSslTransportTest.java        | 213 +++++++
 .../qpid/jms/transports/netty/NettyServer.java  |   4 +-
 .../transports/netty/NettySslTransportTest.java |   5 +-
 .../apache/qpid/jms/JmsSSLConnectionTest.java   |   2 +-
 .../apache/qpid/jms/JmsWSSConnectionTest.java   |   2 +-
 15 files changed, 1129 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 91310f4..f607c00 100644
--- a/pom.xml
+++ b/pom.xml
@@ -40,7 +40,8 @@
 
     <!-- Dependency Versions for this Project -->
     <proton-version>0.28.0</proton-version>
-    <netty-version>4.1.27.Final</netty-version>
+    <netty-version>4.1.28.Final</netty-version>
+    <netty-tcnative-version>2.0.12.Final</netty-tcnative-version>
     <slf4j-version>1.7.25</slf4j-version>
     <geronimo.jms.2.spec.version>1.0-alpha-2</geronimo.jms.2.spec.version>
 
@@ -156,6 +157,12 @@
         <artifactId>netty-codec-http</artifactId>
         <version>${netty-version}</version>
       </dependency>
+      <!--  Testing only Uber Jar inclusion -->
+      <dependency>
+         <groupId>io.netty</groupId>
+         <artifactId>netty-tcnative-boringssl-static</artifactId>
+         <version>${netty-tcnative-version}</version>
+      </dependency>
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-api</artifactId>

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/pom.xml
----------------------------------------------------------------------
diff --git a/qpid-jms-client/pom.xml b/qpid-jms-client/pom.xml
index de175d0..83e8557 100644
--- a/qpid-jms-client/pom.xml
+++ b/qpid-jms-client/pom.xml
@@ -103,6 +103,11 @@
       <artifactId>hadoop-minikdc</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-tcnative-boringssl-static</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportOptions.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportOptions.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportOptions.java
index 7cb520b..4a0f69c 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportOptions.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportOptions.java
@@ -47,6 +47,7 @@ public class TransportOptions implements Cloneable {
     public static final boolean DEFAULT_VERIFY_HOST = true;
     public static final List<String> DEFAULT_DISABLED_PROTOCOLS = Collections.unmodifiableList(Arrays.asList(new String[]{"SSLv2Hello", "SSLv3"}));
     public static final int DEFAULT_SSL_PORT = 5671;
+    public static final boolean DEFAULT_USE_OPENSSL = false;
 
     private static final String JAVAX_NET_SSL_KEY_STORE = "javax.net.ssl.keyStore";
     private static final String JAVAX_NET_SSL_KEY_STORE_TYPE = "javax.net.ssl.keyStoreType";
@@ -67,6 +68,7 @@ public class TransportOptions implements Cloneable {
     private boolean useEpoll = DEFAULT_USE_EPOLL;
     private boolean useKQueue = DEFAULT_USE_KQUEUE;
     private boolean traceBytes = DEFAULT_TRACE_BYTES;
+    private boolean useOpenSSL = DEFAULT_USE_OPENSSL;
 
     private String keyStoreLocation;
     private String keyStorePassword;
@@ -512,6 +514,21 @@ public class TransportOptions implements Cloneable {
         return httpHeaders;
     }
 
+    /**
+     * @return true if OpenSSL support is enabled for this Transport.
+     */
+    public boolean isUseOpenSSL() {
+        return useOpenSSL;
+    }
+
+    /**
+     * @param useOpenSSL
+     * 		Configure if the transport should attempt to use OpenSSL
+     */
+    public void setUseOpenSSL(boolean useOpenSSL) {
+        this.useOpenSSL = useOpenSSL;
+    }
+
     protected TransportOptions copyOptions(TransportOptions copy) {
         copy.setConnectTimeout(getConnectTimeout());
         copy.setReceiveBufferSize(getReceiveBufferSize());
@@ -540,6 +557,7 @@ public class TransportOptions implements Cloneable {
         copy.setContextProtocol(getContextProtocol());
         copy.setDefaultSslPort(getDefaultSslPort());
         copy.setSslContextOverride(getSslContextOverride());
+        copy.setUseOpenSSL(isUseOpenSSL());
 
         return copy;
     }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportSupport.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportSupport.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportSupport.java
index cc96c78..425f2b9 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportSupport.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/TransportSupport.java
@@ -23,8 +23,6 @@ import java.net.URI;
 import java.security.KeyStore;
 import java.security.KeyStoreException;
 import java.security.SecureRandom;
-import java.security.cert.CertificateException;
-import java.security.cert.X509Certificate;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -37,12 +35,18 @@ import javax.net.ssl.SSLParameters;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 import javax.net.ssl.X509ExtendedKeyManager;
-import javax.net.ssl.X509TrustManager;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.OpenSslX509KeyManagerFactory;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
 import io.netty.handler.ssl.SslHandler;
+import io.netty.handler.ssl.SslProvider;
+import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
 
 /**
  * Static class that provides various utility methods used by Transport implementations.
@@ -52,6 +56,38 @@ public class TransportSupport {
     private static final Logger LOG = LoggerFactory.getLogger(TransportSupport.class);
 
     /**
+     * Determines if Netty OpenSSL support is available and applicable based on the configuration
+     * in the given TransportOptions instance.
+     *
+     * @param options
+     * 		  The configuration of the Transport being created.
+     *
+     * @return true if OpenSSL support is available and usable given the requested configuration.
+     */
+    public static boolean isOpenSSLPossible(TransportOptions options) {
+        boolean result = false;
+
+        if (options.isUseOpenSSL()) {
+            if (!OpenSsl.isAvailable()) {
+                LOG.debug("OpenSSL could not be enabled because a suitable implementation could not be found.", OpenSsl.unavailabilityCause());
+            } else if (options.getSslContextOverride() != null) {
+                LOG.debug("OpenSSL could not be enabled due to user SSLContext being supplied.");
+            } else if (!OpenSsl.supportsKeyManagerFactory()) {
+                LOG.debug("OpenSSL could not be enabled because the version provided doesn allow a KeyManagerFactory to be used");
+            } else if (options.isVerifyHost() && !OpenSsl.supportsHostnameValidation()) {
+                LOG.debug("OpenSSL could not be enabled due to verifyHost being enabled but not supported by the provided OpenSSL version.");
+            } else if (options.getKeyAlias() != null) {
+                LOG.debug("OpenSSL could not be enabled because a keyAlias is set and that feature is not supported for OpenSSL");
+            } else {
+                LOG.debug("OpenSSL Enabled: Version {} of OpenSSL will be used", OpenSsl.versionString());
+                result = true;
+            }
+        }
+
+        return result;
+    }
+
+    /**
      * Creates a Netty SslHandler instance for use in Transports that require
      * an SSL encoder / decoder.
      *
@@ -59,6 +95,8 @@ public class TransportSupport {
      * when creating the handler. If they do not, an SSLContext will first be created
      * using the other option values.
      *
+     * @param allocator
+     *		  The Netty Buffer Allocator to use is Netty resources need to be created.
      * @param remote
      *        The URI of the remote peer that the SslHandler will be used against.
      * @param options
@@ -68,19 +106,28 @@ public class TransportSupport {
      *
      * @throws Exception if an error occurs while creating the SslHandler instance.
      */
-    public static SslHandler createSslHandler(URI remote, TransportOptions options) throws Exception {
-        SSLContext sslContext = options.getSslContextOverride();
-        if(sslContext == null) {
-            sslContext = createSslContext(options);
-        }
+    public static SslHandler createSslHandler(ByteBufAllocator allocator, URI remote, TransportOptions options) throws Exception {
+        final SSLEngine sslEngine;
 
-        SSLEngine sslEngine = createSslEngine(remote, sslContext, options);
+        if (isOpenSSLPossible(options)) {
+            SslContext sslContext = createOpenSslContext(options);
+            sslEngine = createOpenSslEngine(allocator, remote, sslContext, options);
+        } else {
+            SSLContext sslContext = options.getSslContextOverride();
+            if (sslContext == null) {
+                sslContext = createJdkSslContext(options);
+            }
+
+            sslEngine = createJdkSslEngine(remote, sslContext, options);
+        }
 
         return new SslHandler(sslEngine);
     }
 
+    //----- JDK SSL Support Methods ------------------------------------------//
+
     /**
-     * Create a new SSLContext using the options specific in the given TransportSslOptions
+     * Create a new SSLContext using the options specific in the given TransportOptions
      * instance.
      *
      * @param options
@@ -90,12 +137,13 @@ public class TransportSupport {
      *
      * @throws Exception if an error occurs while creating the context.
      */
-    public static SSLContext createSslContext(TransportOptions options) throws Exception {
+    public static SSLContext createJdkSslContext(TransportOptions options) throws Exception {
         try {
             String contextProtocol = options.getContextProtocol();
             LOG.trace("Getting SSLContext instance using protocol: {}", contextProtocol);
 
             SSLContext context = SSLContext.getInstance(contextProtocol);
+
             KeyManager[] keyMgrs = loadKeyManagers(options);
             TrustManager[] trustManagers = loadTrustManagers(options);
 
@@ -108,9 +156,11 @@ public class TransportSupport {
     }
 
     /**
-     * Create a new SSLEngine instance in client mode from the given SSLContext and
-     * TransportSslOptions instances.
+     * Create a new JDK SSLEngine instance in client mode from the given SSLContext and
+     * TransportOptions instances.
      *
+     * @param remote
+     *        the URI of the remote peer that will be used to initialize the engine, may be null if none should.
      * @param context
      *        the SSLContext to use when creating the engine.
      * @param options
@@ -120,31 +170,95 @@ public class TransportSupport {
      *
      * @throws Exception if an error occurs while creating the new SSLEngine.
      */
-    public static SSLEngine createSslEngine(SSLContext context, TransportOptions options) throws Exception {
-        return createSslEngine(null, context, options);
+    public static SSLEngine createJdkSslEngine(URI remote, SSLContext context, TransportOptions options) throws Exception {
+        SSLEngine engine = null;
+        if (remote == null) {
+            engine = context.createSSLEngine();
+        } else {
+            engine = context.createSSLEngine(remote.getHost(), remote.getPort());
+        }
+
+        engine.setEnabledProtocols(buildEnabledProtocols(engine, options));
+        engine.setEnabledCipherSuites(buildEnabledCipherSuites(engine, options));
+        engine.setUseClientMode(true);
+
+        if (options.isVerifyHost()) {
+            SSLParameters sslParameters = engine.getSSLParameters();
+            sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+            engine.setSSLParameters(sslParameters);
+        }
+
+        return engine;
     }
 
+    //----- OpenSSL Support Methods ------------------------------------------//
+
     /**
-     * Create a new SSLEngine instance in client mode from the given SSLContext and
-     * TransportSslOptions instances.
+     * Create a new Netty SslContext using the options specific in the given TransportOptions
+     * instance.
+     *
+     * @param options
+     *        the configured options used to create the SslContext.
+     *
+     * @return a new SslContext instance.
      *
+     * @throws Exception if an error occurs while creating the context.
+     */
+    public static SslContext createOpenSslContext(TransportOptions options) throws Exception {
+        try {
+            String contextProtocol = options.getContextProtocol();
+            LOG.trace("Getting SslContext instance using protocol: {}", contextProtocol);
+
+            KeyManagerFactory keyManagerFactory = loadKeyManagerFactory(options, SslProvider.OPENSSL);
+            TrustManagerFactory trustManagerFactory = loadTrustManagerFactory(options);
+            SslContextBuilder builder = SslContextBuilder.forClient().sslProvider(SslProvider.OPENSSL);
+
+            // TODO - There is oddly no way in Netty right now to get the set of supported protocols
+            //        when creating the SslContext or really even when creating the SSLEngine.  Seems
+            //        like an oversight, for now we call it with TLSv1.2 so it looks like we did something.
+            if (options.getContextProtocol().equals(TransportOptions.DEFAULT_CONTEXT_PROTOCOL)) {
+                builder.protocols("TLSv1.2");
+            } else {
+                builder.protocols(options.getContextProtocol());
+            }
+            builder.keyManager(keyManagerFactory);
+            builder.trustManager(trustManagerFactory);
+
+            return builder.build();
+        } catch (Exception e) {
+            LOG.error("Failed to create SslContext: {}", e, e);
+            throw e;
+        }
+    }
+
+    /**
+     * Create a new OpenSSL SSLEngine instance in client mode from the given SSLContext and
+     * TransportOptions instances.
+     *
+     * @param allocator
+     *		  the Netty ByteBufAllocator to use to create the OpenSSL engine
      * @param remote
      *        the URI of the remote peer that will be used to initialize the engine, may be null if none should.
      * @param context
-     *        the SSLContext to use when creating the engine.
+     *        the Netty SslContext to use when creating the engine.
      * @param options
      *        the TransportOptions to use to configure the new SSLEngine.
      *
-     * @return a new SSLEngine instance in client mode.
+     * @return a new Netty managed SSLEngine instance in client mode.
      *
      * @throws Exception if an error occurs while creating the new SSLEngine.
      */
-    public static SSLEngine createSslEngine(URI remote, SSLContext context, TransportOptions options) throws Exception {
+    public static SSLEngine createOpenSslEngine(ByteBufAllocator allocator, URI remote, SslContext context, TransportOptions options) throws Exception {
         SSLEngine engine = null;
-        if(remote == null) {
-            engine = context.createSSLEngine();
+
+        if (allocator == null) {
+            throw new IllegalArgumentException("OpenSSL engine requires a valid ByteBufAllocator to operate");
+        }
+
+        if (remote == null) {
+            engine = context.newEngine(allocator);
         } else {
-            engine = context.createSSLEngine(remote.getHost(), remote.getPort());
+            engine = context.newEngine(allocator, remote.getHost(), remote.getPort());
         }
 
         engine.setEnabledProtocols(buildEnabledProtocols(engine, options));
@@ -160,6 +274,8 @@ public class TransportSupport {
         return engine;
     }
 
+    //----- Internal support methods -----------------------------------------//
+
     private static String[] buildEnabledProtocols(SSLEngine engine, TransportOptions options) {
         List<String> enabledProtocols = new ArrayList<String>();
 
@@ -211,8 +327,17 @@ public class TransportSupport {
     }
 
     private static TrustManager[] loadTrustManagers(TransportOptions options) throws Exception {
+        TrustManagerFactory factory = loadTrustManagerFactory(options);
+        if (factory != null) {
+            return factory.getTrustManagers();
+        } else {
+            return null;
+        }
+    }
+
+    private static TrustManagerFactory loadTrustManagerFactory(TransportOptions options) throws Exception {
         if (options.isTrustAll()) {
-            return new TrustManager[] { createTrustAllTrustManager() };
+            return InsecureTrustManagerFactory.INSTANCE;
         }
 
         if (options.getTrustStoreLocation() == null) {
@@ -230,7 +355,7 @@ public class TransportSupport {
         KeyStore trustStore = loadStore(storeLocation, storePassword, storeType);
         fact.init(trustStore);
 
-        return fact.getTrustManagers();
+        return fact;
     }
 
     private static KeyManager[] loadKeyManagers(TransportOptions options) throws Exception {
@@ -258,6 +383,30 @@ public class TransportSupport {
         }
     }
 
+    private static KeyManagerFactory loadKeyManagerFactory(TransportOptions options, SslProvider provider) throws Exception {
+        if (options.getKeyStoreLocation() == null) {
+            return null;
+        }
+
+        final KeyManagerFactory factory;
+        if (provider.equals(SslProvider.JDK)) {
+            factory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        } else {
+            factory = new OpenSslX509KeyManagerFactory();
+        }
+
+        String storeLocation = options.getKeyStoreLocation();
+        String storePassword = options.getKeyStorePassword();
+        String storeType = options.getKeyStoreType();
+
+        LOG.trace("Attempt to load KeyStore from location {} of type {}", storeLocation, storeType);
+
+        KeyStore keyStore = loadStore(storeLocation, storePassword, storeType);
+        factory.init(keyStore, storePassword != null ? storePassword.toCharArray() : null);
+
+        return factory;
+    }
+
     private static KeyManager[] wrapKeyManagers(String alias, KeyManager[] origKeyManagers) {
         KeyManager[] keyManagers = new KeyManager[origKeyManagers.length];
         for (int i = 0; i < origKeyManagers.length; i++) {
@@ -290,21 +439,4 @@ public class TransportSupport {
 
         return store;
     }
-
-    private static TrustManager createTrustAllTrustManager() {
-        return new X509TrustManager() {
-            @Override
-            public void checkClientTrusted(X509Certificate[] x509Certificates, String s) throws CertificateException {
-            }
-
-            @Override
-            public void checkServerTrusted(X509Certificate[] x509Certificates, String s) throws CertificateException {
-            }
-
-            @Override
-            public X509Certificate[] getAcceptedIssuers() {
-                return new X509Certificate[0];
-            }
-        };
-    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/netty/NettyTcpTransport.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/netty/NettyTcpTransport.java b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/netty/NettyTcpTransport.java
index 3c02cc9..5df7899 100644
--- a/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/netty/NettyTcpTransport.java
+++ b/qpid-jms-client/src/main/java/org/apache/qpid/jms/transports/netty/NettyTcpTransport.java
@@ -134,18 +134,6 @@ public class NettyTcpTransport implements Transport {
 
         getTransportOptions().setSslContextOverride(sslContextOverride);
 
-        final SslHandler sslHandler;
-        if (isSecure()) {
-            try {
-                sslHandler = TransportSupport.createSslHandler(getRemoteLocation(), getTransportOptions());
-            } catch (Exception ex) {
-                // TODO: can we stop it throwing Exception?
-                throw IOExceptionSupport.create(ex);
-            }
-        } else {
-            sslHandler = null;
-        }
-
         boolean useKQueue = getTransportOptions().isUseKQueue() && KQueue.isAvailable();
         boolean useEpoll = getTransportOptions().isUseEpoll() && Epoll.isAvailable();
 
@@ -172,7 +160,7 @@ public class NettyTcpTransport implements Transport {
         bootstrap.handler(new ChannelInitializer<Channel>() {
             @Override
             public void initChannel(Channel connectedChannel) throws Exception {
-                configureChannel(connectedChannel, sslHandler);
+                configureChannel(connectedChannel);
             }
         });
 
@@ -440,9 +428,16 @@ public class NettyTcpTransport implements Transport {
         }
     }
 
-    private void configureChannel(final Channel channel, final SslHandler sslHandler) throws Exception {
+    private void configureChannel(final Channel channel) throws Exception {
         if (isSecure()) {
-            channel.pipeline().addLast(sslHandler);
+            final SslHandler sslHandler;
+            try {
+                sslHandler = TransportSupport.createSslHandler(channel.alloc(), getRemoteLocation(), getTransportOptions());
+            } catch (Exception ex) {
+                throw IOExceptionSupport.create(ex);
+            }
+
+            channel.pipeline().addLast("ssl", sslHandler);
         }
 
         if (getTransportOptions().isTraceBytes()) {

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SaslIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SaslIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SaslIntegrationTest.java
index 2463318..2bcb363 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SaslIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SaslIntegrationTest.java
@@ -83,7 +83,7 @@ public class SaslIntegrationTest extends QpidJmsTestCase {
                              "transport.keyStoreLocation=" + CLIENT_JKS_KEYSTORE + "&" +
                              "transport.keyStorePassword=" + PASSWORD;
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, true);) {
             // Expect an EXTERNAL connection
@@ -312,7 +312,7 @@ public class SaslIntegrationTest extends QpidJmsTestCase {
             sslOptions.setTrustStorePassword(PASSWORD);
         }
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, requireClientCert);) {
             String connOptions = "?transport.trustStoreLocation=" + CLIENT_JKS_TRUSTSTORE + "&" +

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SslIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SslIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SslIntegrationTest.java
index f79ef32..fcae27d 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SslIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/integration/SslIntegrationTest.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.IOException;
 import java.net.Socket;
@@ -47,6 +48,8 @@ import org.apache.qpid.jms.transports.TransportOptions;
 import org.apache.qpid.jms.transports.TransportSupport;
 import org.junit.Test;
 
+import io.netty.handler.ssl.OpenSsl;
+
 public class SslIntegrationTest extends QpidJmsTestCase {
 
     private static final String BROKER_JKS_KEYSTORE = "src/test/resources/broker-jks.keystore";
@@ -82,17 +85,30 @@ public class SslIntegrationTest extends QpidJmsTestCase {
     private final IntegrationTestFixture testFixture = new IntegrationTestFixture();
 
     @Test(timeout = 20000)
-    public void testCreateAndCloseSslConnection() throws Exception {
+    public void testCreateAndCloseSslConnectionJDK() throws Exception {
+        testCreateAndCloseSslConnection(false);
+    }
+
+    @Test(timeout = 20000)
+    public void testCreateAndCloseSslConnectionOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        testCreateAndCloseSslConnection(true);
+    }
+
+    private void testCreateAndCloseSslConnection(boolean openSSL) throws Exception {
         TransportOptions sslOptions = new TransportOptions();
         sslOptions.setKeyStoreLocation(BROKER_JKS_KEYSTORE);
         sslOptions.setKeyStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, false);) {
             String connOptions = "?transport.trustStoreLocation=" + CLIENT_JKS_TRUSTSTORE + "&" +
-                                 "transport.trustStorePassword=" + PASSWORD;
+                                 "transport.trustStorePassword=" + PASSWORD + "&" +
+                                 "transport.useOpenSSL=" + openSSL;
             Connection connection = testFixture.establishConnecton(testPeer, true, connOptions, null, null, true);
 
             Socket socket = testPeer.getClientSocket();
@@ -104,13 +120,25 @@ public class SslIntegrationTest extends QpidJmsTestCase {
     }
 
     @Test(timeout = 20000)
-    public void testCreateSslConnectionWithServerSendingPreemptiveData() throws Exception {
+    public void testCreateSslConnectionWithServerSendingPreemptiveDataJDK() throws Exception {
+        doTestCreateSslConnectionWithServerSendingPreemptiveData(false);
+    }
+
+    @Test(timeout = 20000)
+    public void testCreateSslConnectionWithServerSendingPreemptiveDataOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        doTestCreateSslConnectionWithServerSendingPreemptiveData(true);
+    }
+
+    private void doTestCreateSslConnectionWithServerSendingPreemptiveData(boolean openSSL) throws Exception {
         TransportOptions serverSslOptions = new TransportOptions();
         serverSslOptions.setKeyStoreLocation(BROKER_JKS_KEYSTORE);
         serverSslOptions.setKeyStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         boolean sendServerSaslHeaderPreEmptively = true;
         try (TestAmqpPeer testPeer = new TestAmqpPeer(serverSslContext, false, sendServerSaslHeaderPreEmptively);) {
@@ -120,7 +148,8 @@ public class SslIntegrationTest extends QpidJmsTestCase {
             testPeer.expectBegin();
 
             String connOptions = "?transport.trustStoreLocation=" + CLIENT_JKS_TRUSTSTORE + "&" +
-                                  "transport.trustStorePassword=" + PASSWORD;
+                                  "transport.trustStorePassword=" + PASSWORD + "&" +
+                                  "transport.useOpenSSL=" + openSSL;
 
             JmsConnectionFactory factory = new JmsConnectionFactory("amqps://localhost:" + testPeer.getServerPort() + connOptions);
             Connection connection = factory.createConnection();
@@ -135,7 +164,19 @@ public class SslIntegrationTest extends QpidJmsTestCase {
     }
 
     @Test(timeout = 20000)
-    public void testCreateAndCloseSslConnectionWithClientAuth() throws Exception {
+    public void testCreateAndCloseSslConnectionWithClientAuthJDK() throws Exception {
+        doTestCreateAndCloseSslConnectionWithClientAuth(false);
+    }
+
+    @Test(timeout = 20000)
+    public void testCreateAndCloseSslConnectionWithClientAuthOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        doTestCreateAndCloseSslConnectionWithClientAuth(true);
+    }
+
+    private void doTestCreateAndCloseSslConnectionWithClientAuth(boolean openSSL) throws Exception {
         TransportOptions sslOptions = new TransportOptions();
         sslOptions.setKeyStoreLocation(BROKER_JKS_KEYSTORE);
         sslOptions.setTrustStoreLocation(BROKER_JKS_TRUSTSTORE);
@@ -143,13 +184,14 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         sslOptions.setTrustStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, true);) {
             String connOptions = "?transport.keyStoreLocation=" + CLIENT_MULTI_KEYSTORE + "&" +
                                  "transport.keyStorePassword=" + PASSWORD + "&" +
                                  "transport.trustStoreLocation=" + CLIENT_JKS_TRUSTSTORE + "&" +
-                                 "transport.trustStorePassword=" + PASSWORD;
+                                 "transport.trustStorePassword=" + PASSWORD + "&" +
+                                 "transport.useOpenSSL=" + openSSL;
             Connection connection = testFixture.establishConnecton(testPeer, true, connOptions, null, null, true);
 
             Socket socket = testPeer.getClientSocket();
@@ -162,12 +204,21 @@ public class SslIntegrationTest extends QpidJmsTestCase {
     }
 
     @Test(timeout = 20000)
-    public void testCreateAndCloseSslConnectionWithAlias() throws Exception {
-        doConnectionWithAliasTestImpl(CLIENT_KEY_ALIAS, CLIENT_DN);
-        doConnectionWithAliasTestImpl(CLIENT2_KEY_ALIAS, CLIENT2_DN);
+    public void testCreateAndCloseSslConnectionWithAliasJDK() throws Exception {
+        doConnectionWithAliasTestImpl(CLIENT_KEY_ALIAS, CLIENT_DN, false);
+        doConnectionWithAliasTestImpl(CLIENT2_KEY_ALIAS, CLIENT2_DN, false);
     }
 
-    private void doConnectionWithAliasTestImpl(String alias, String expectedDN) throws Exception, JMSException, SSLPeerUnverifiedException, IOException {
+    @Test(timeout = 20000)
+    public void testCreateAndCloseSslConnectionWithAliasOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        doConnectionWithAliasTestImpl(CLIENT_KEY_ALIAS, CLIENT_DN, true);
+        doConnectionWithAliasTestImpl(CLIENT2_KEY_ALIAS, CLIENT2_DN, true);
+    }
+
+    private void doConnectionWithAliasTestImpl(String alias, String expectedDN, boolean requestOpenSSL) throws Exception, JMSException, SSLPeerUnverifiedException, IOException {
         TransportOptions sslOptions = new TransportOptions();
         sslOptions.setKeyStoreLocation(BROKER_JKS_KEYSTORE);
         sslOptions.setTrustStoreLocation(BROKER_JKS_TRUSTSTORE);
@@ -175,14 +226,16 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         sslOptions.setTrustStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, true);) {
             String connOptions = "?transport.keyStoreLocation=" + CLIENT_MULTI_KEYSTORE + "&" +
                                  "transport.keyStorePassword=" + PASSWORD + "&" +
                                  "transport.trustStoreLocation=" + CLIENT_JKS_TRUSTSTORE + "&" +
                                  "transport.trustStorePassword=" + PASSWORD + "&" +
-                                 "transport.keyAlias=" + alias;
+                                 "transport.keyAlias=" + alias + "&" +
+                                 "transport.useOpenSSL=" + requestOpenSSL;
+
             Connection connection = testFixture.establishConnecton(testPeer, true, connOptions, null, null, true);
 
             Socket socket = testPeer.getClientSocket();
@@ -220,7 +273,7 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         sslOptions.setTrustStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext context = TransportSupport.createSslContext(sslOptions);
+        SSLContext context = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(context, true);) {
             String connOptions = "?transport.keyStoreLocation=" + CLIENT_MULTI_KEYSTORE + "&" +
@@ -279,7 +332,7 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         TransportOptions clientSslOptions = new TransportOptions();
         clientSslOptions.setKeyStoreLocation(clientKeyStorePath);
@@ -293,13 +346,13 @@ public class SslIntegrationTest extends QpidJmsTestCase {
             if (useExtension) {
                 factory.setExtension(JmsConnectionExtensions.SSL_CONTEXT.toString(), (options, uri) -> {
                     try {
-                        return TransportSupport.createSslContext(clientSslOptions);
+                        return TransportSupport.createJdkSslContext(clientSslOptions);
                     } catch (Exception e) {
                         throw new RuntimeException(e.getMessage(), e);
                     }
                 });
             } else {
-                factory.setSslContext(TransportSupport.createSslContext(clientSslOptions));
+                factory.setSslContext(TransportSupport.createJdkSslContext(clientSslOptions));
             }
 
             testPeer.expectSaslPlain("guest", "guest");
@@ -348,7 +401,7 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         clientSslOptions.setKeyStorePassword(PASSWORD);
         clientSslOptions.setTrustStorePassword(PASSWORD);
 
-        SSLContext clientContext = TransportSupport.createSslContext(clientSslOptions);
+        SSLContext clientContext = TransportSupport.createJdkSslContext(clientSslOptions);
 
         // Connect providing the Client 2 details via context override, expect Client2 DN instead.
         doConnectionWithSslContextOverrideAndURIConfig(clientContext, CLIENT2_DN);
@@ -362,7 +415,7 @@ public class SslIntegrationTest extends QpidJmsTestCase {
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(serverContext, true);) {
             String connOptions = "?transport.keyStoreLocation=" + CLIENT_JKS_KEYSTORE + "&" +
@@ -481,7 +534,7 @@ public class SslIntegrationTest extends QpidJmsTestCase {
             serverSslOptions.setVerifyHost(false);
         }
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         try (TestAmqpPeer testPeer = new TestAmqpPeer(serverSslContext, true);) {
             Connection connection = testFixture.establishConnecton(testPeer, true, null, null, null, true);

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverWithAmqpOpenProvidedServerListIntegrationTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverWithAmqpOpenProvidedServerListIntegrationTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverWithAmqpOpenProvidedServerListIntegrationTest.java
index 6006ca1..0baed85 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverWithAmqpOpenProvidedServerListIntegrationTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/provider/failover/FailoverWithAmqpOpenProvidedServerListIntegrationTest.java
@@ -406,7 +406,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         setSslSystemPropertiesForCurrentTest(CLIENT_JKS_KEYSTORE, PASSWORD, CLIENT_JKS_TRUSTSTORE, PASSWORD);
 
@@ -513,7 +513,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         sslOptions.setKeyStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(sslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer primaryPeer = new TestAmqpPeer(serverSslContext, false);
              TestAmqpPeer backupPeer = new TestAmqpPeer(serverSslContext, false);) {
@@ -632,7 +632,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         sslOptions.setKeyStorePassword(PASSWORD);
         sslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(sslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(sslOptions);
 
         try (TestAmqpPeer primaryPeer = new TestAmqpPeer(serverSslContext, false);
              TestAmqpPeer backupPeer = new TestAmqpPeer(serverSslContext, false);) {
@@ -753,7 +753,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         TransportOptions clientSslOptions = new TransportOptions();
         clientSslOptions.setKeyStoreLocation(CLIENT_JKS_KEYSTORE);
@@ -761,7 +761,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         clientSslOptions.setKeyStorePassword(PASSWORD);
         clientSslOptions.setTrustStorePassword(PASSWORD);
 
-        SSLContext clientSslContext = TransportSupport.createSslContext(clientSslOptions);
+        SSLContext clientSslContext = TransportSupport.createJdkSslContext(clientSslOptions);
 
         try (TestAmqpPeer primaryPeer = new TestAmqpPeer(serverSslContext, false);
              TestAmqpPeer backupPeer = new TestAmqpPeer(serverSslContext, false);) {
@@ -892,7 +892,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         setSslSystemPropertiesForCurrentTest(CLIENT_JKS_KEYSTORE, PASSWORD, CLIENT_JKS_TRUSTSTORE, PASSWORD);
 
@@ -999,7 +999,7 @@ public class FailoverWithAmqpOpenProvidedServerListIntegrationTest extends QpidJ
         serverSslOptions.setTrustStorePassword(PASSWORD);
         serverSslOptions.setVerifyHost(false);
 
-        SSLContext serverSslContext = TransportSupport.createSslContext(serverSslOptions);
+        SSLContext serverSslContext = TransportSupport.createJdkSslContext(serverSslOptions);
 
         setSslSystemPropertiesForCurrentTest(CLIENT_JKS_KEYSTORE, PASSWORD, CLIENT_JKS_TRUSTSTORE, PASSWORD);
 

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/TransportSupportTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/TransportSupportTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/TransportSupportTest.java
index 23b58ff..90ac437 100644
--- a/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/TransportSupportTest.java
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/TransportSupportTest.java
@@ -23,9 +23,11 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.IOException;
 import java.security.UnrecoverableKeyException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -35,6 +37,12 @@ import javax.net.ssl.SSLEngine;
 import org.apache.qpid.jms.test.QpidJmsTestCase;
 import org.junit.Test;
 
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.OpenSslEngine;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslHandler;
+
 /**
  * Tests for the TransportSupport class.
  */
@@ -63,17 +71,20 @@ public class TransportSupportTest extends QpidJmsTestCase {
 
     public static final String[] ENABLED_PROTOCOLS = new String[] { "TLSv1" };
 
+    // Currently the OpenSSL implementation cannot disable SSLv2Hello
+    public static final String[] ENABLED_OPENSSL_PROTOCOLS = new String[] { "SSLv2Hello", "TLSv1" };
+
     private static final String ALIAS_DOES_NOT_EXIST = "alias.does.not.exist";
     private static final String ALIAS_CA_CERT = "ca";
 
     @Test
-    public void testLegacySslProtocolsDisabledByDefault() throws Exception {
+    public void testLegacySslProtocolsDisabledByDefaultJDK() throws Exception {
         TransportOptions options = createJksSslOptions(null);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
         assertNotNull(engine);
 
         List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
@@ -82,97 +93,231 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslContextJksStore() throws Exception {
+    public void testLegacySslProtocolsDisabledByDefaultOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions(null);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+        assertNotNull(engine);
+
+        List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
+        assertFalse("SSLv3 should not be enabled by default", engineProtocols.contains("SSLv3"));
+
+        // TODO - Netty is currently unable to disable OpenSSL SSLv2Hello so we are stuck with it for now.
+        // assertFalse("SSLv2Hello should not be enabled by default", engineProtocols.contains("SSLv2Hello"));
+    }
+
+    @Test
+    public void testCreateSslContextJksStoreJDK() throws Exception {
         TransportOptions options = createJksSslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
         assertEquals("TLS", context.getProtocol());
     }
 
     @Test
-    public void testCreateSslContextJksStoreWithConfiguredContextProtocol() throws Exception {
+    public void testCreateSslContextJksStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions();
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        // TODO There is no means currently of getting the protocol from the netty SslContext.
+        // assertEquals("TLS", context.getProtocol());
+    }
+
+    @Test
+    public void testCreateSslContextJksStoreWithConfiguredContextProtocolJDK() throws Exception {
         TransportOptions options = createJksSslOptions();
         String contextProtocol = "TLSv1.2";
         options.setContextProtocol(contextProtocol);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
         assertEquals(contextProtocol, context.getProtocol());
     }
 
+    @Test
+    public void testCreateSslContextJksStoreWithConfiguredContextProtocolOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions();
+        String contextProtocol = "TLSv1.2";
+        options.setContextProtocol(contextProtocol);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        // TODO There is no means currently of getting the protocol from the netty SslContext.
+        // assertEquals(contextProtocol, context.getProtocol());
+    }
+
     @Test(expected = UnrecoverableKeyException.class)
-    public void testCreateSslContextNoKeyStorePassword() throws Exception {
+    public void testCreateSslContextNoKeyStorePasswordJDK() throws Exception {
+        TransportOptions options = createJksSslOptions();
+        options.setKeyStorePassword(null);
+        TransportSupport.createJdkSslContext(options);
+    }
+
+    @Test(expected = UnrecoverableKeyException.class)
+    public void testCreateSslContextNoKeyStorePasswordOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
         TransportOptions options = createJksSslOptions();
         options.setKeyStorePassword(null);
-        TransportSupport.createSslContext(options);
+        TransportSupport.createOpenSslContext(options);
     }
 
     @Test(expected = IOException.class)
-    public void testCreateSslContextWrongKeyStorePassword() throws Exception {
+    public void testCreateSslContextWrongKeyStorePasswordJDK() throws Exception {
         TransportOptions options = createJksSslOptions();
         options.setKeyStorePassword("wrong");
-        TransportSupport.createSslContext(options);
+        TransportSupport.createJdkSslContext(options);
     }
 
     @Test(expected = IOException.class)
-    public void testCreateSslContextBadPathToKeyStore() throws Exception {
+    public void testCreateSslContextWrongKeyStorePasswordOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions();
+        options.setKeyStorePassword("wrong");
+        TransportSupport.createOpenSslContext(options);
+    }
+
+    @Test(expected = IOException.class)
+    public void testCreateSslContextBadPathToKeyStoreJDK() throws Exception {
         TransportOptions options = createJksSslOptions();
         options.setKeyStoreLocation(CLIENT_JKS_KEYSTORE + ".bad");
-        TransportSupport.createSslContext(options);
+        TransportSupport.createJdkSslContext(options);
     }
 
     @Test(expected = IOException.class)
-    public void testCreateSslContextWrongTrustStorePassword() throws Exception {
+    public void testCreateSslContextBadPathToKeyStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions();
+        options.setKeyStoreLocation(CLIENT_JKS_KEYSTORE + ".bad");
+        TransportSupport.createOpenSslContext(options);
+    }
+
+    @Test(expected = IOException.class)
+    public void testCreateSslContextWrongTrustStorePasswordJDK() throws Exception {
         TransportOptions options = createJksSslOptions();
         options.setTrustStorePassword("wrong");
-        TransportSupport.createSslContext(options);
+        TransportSupport.createJdkSslContext(options);
+    }
+
+    @Test(expected = IOException.class)
+    public void testCreateSslContextWrongTrustStorePasswordOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions();
+        options.setTrustStorePassword("wrong");
+        TransportSupport.createOpenSslContext(options);
+    }
+
+    @Test(expected = IOException.class)
+    public void testCreateSslContextBadPathToTrustStoreJDK() throws Exception {
+        TransportOptions options = createJksSslOptions();
+        options.setTrustStoreLocation(CLIENT_JKS_TRUSTSTORE + ".bad");
+        TransportSupport.createJdkSslContext(options);
     }
 
     @Test(expected = IOException.class)
-    public void testCreateSslContextBadPathToTrustStore() throws Exception {
+    public void testCreateSslContextBadPathToTrustStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
         TransportOptions options = createJksSslOptions();
         options.setTrustStoreLocation(CLIENT_JKS_TRUSTSTORE + ".bad");
-        TransportSupport.createSslContext(options);
+        TransportSupport.createOpenSslContext(options);
     }
 
     @Test
-    public void testCreateSslContextJceksStore() throws Exception {
+    public void testCreateSslContextJceksStoreJDK() throws Exception {
         TransportOptions options = createJceksSslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
         assertEquals("TLS", context.getProtocol());
     }
 
     @Test
-    public void testCreateSslContextPkcs12Store() throws Exception {
+    public void testCreateSslContextJceksStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJceksSslOptions();
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+        assertTrue(context.isClient());
+    }
+
+    @Test
+    public void testCreateSslContextPkcs12StoreJDK() throws Exception {
         TransportOptions options = createPkcs12SslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
         assertEquals("TLS", context.getProtocol());
     }
 
+    @Test
+    public void testCreateSslContextPkcs12StoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createPkcs12SslOptions();
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+        assertTrue(context.isClient());
+    }
+
     @Test(expected = IOException.class)
-    public void testCreateSslContextIncorrectStoreType() throws Exception {
+    public void testCreateSslContextIncorrectStoreTypeJDK() throws Exception {
         TransportOptions options = createPkcs12SslOptions();
         options.setStoreType(KEYSTORE_JCEKS_TYPE);
-        TransportSupport.createSslContext(options);
+        TransportSupport.createJdkSslContext(options);
+    }
+
+    @Test(expected = IOException.class)
+    public void testCreateSslContextIncorrectStoreTypeOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createPkcs12SslOptions();
+        options.setStoreType(KEYSTORE_JCEKS_TYPE);
+        TransportSupport.createOpenSslContext(options);
     }
 
     @Test
-    public void testCreateSslEngineFromPkcs12Store() throws Exception {
+    public void testCreateSslEngineFromPkcs12StoreJDK() throws Exception {
         TransportOptions options = createPkcs12SslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
         assertNotNull(engine);
 
         List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
@@ -180,26 +325,76 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromPkcs12StoreWithExplicitEnabledProtocols() throws Exception {
+    public void testCreateSslEngineFromPkcs12StoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createPkcs12SslOptions();
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+        assertNotNull(engine);
+
+        List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
+        assertFalse(engineProtocols.isEmpty());
+    }
+
+    @Test
+    public void testCreateSslEngineFromPkcs12StoreWithExplicitEnabledProtocolsJDK() throws Exception {
         TransportOptions options = createPkcs12SslOptions(ENABLED_PROTOCOLS);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
         assertNotNull(engine);
 
         assertArrayEquals("Enabled protocols not as expected", ENABLED_PROTOCOLS, engine.getEnabledProtocols());
     }
 
     @Test
-    public void testCreateSslEngineFromJksStore() throws Exception {
+    public void testCreateSslEngineFromPkcs12StoreWithExplicitEnabledProtocolsOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createPkcs12SslOptions(ENABLED_PROTOCOLS);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+        assertNotNull(engine);
+
+        assertArrayEquals("Enabled protocols not as expected", ENABLED_OPENSSL_PROTOCOLS, engine.getEnabledProtocols());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreJDK() throws Exception {
+        TransportOptions options = createJksSslOptions();
+
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+        assertNotNull(engine);
+
+        List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
+        assertFalse(engineProtocols.isEmpty());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
         TransportOptions options = createJksSslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SslContext context = TransportSupport.createOpenSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
         assertNotNull(engine);
 
         List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
@@ -207,20 +402,36 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitEnabledProtocols() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledProtocolsJDK() throws Exception {
         TransportOptions options = createJksSslOptions(ENABLED_PROTOCOLS);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
         assertNotNull(engine);
 
         assertArrayEquals("Enabled protocols not as expected", ENABLED_PROTOCOLS, engine.getEnabledProtocols());
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitDisabledProtocols() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledProtocolsOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = createJksSslOptions(ENABLED_PROTOCOLS);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+        assertNotNull(engine);
+
+        assertArrayEquals("Enabled protocols not as expected", ENABLED_OPENSSL_PROTOCOLS, engine.getEnabledProtocols());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitDisabledProtocolsJDK() throws Exception {
         // Discover the default enabled protocols
         TransportOptions options = createJksSslOptions();
         SSLEngine directEngine = createSSLEngineDirectly(options);
@@ -231,8 +442,31 @@ public class TransportSupportTest extends QpidJmsTestCase {
         String[] disabledProtocol = new String[] { protocols[protocols.length - 1] };
         String[] trimmedProtocols = Arrays.copyOf(protocols, protocols.length - 1);
         options.setDisabledProtocols(disabledProtocol);
-        SSLContext context = TransportSupport.createSslContext(options);
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+
+        // verify the option took effect
+        assertNotNull(engine);
+        assertArrayEquals("Enabled protocols not as expected", trimmedProtocols, engine.getEnabledProtocols());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitDisabledProtocolsOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Discover the default enabled protocols
+        TransportOptions options = createJksSslOptions();
+        SSLEngine directEngine = createOpenSSLEngineDirectly(options);
+        String[] protocols = directEngine.getEnabledProtocols();
+        assertTrue("There were no initial protocols to choose from!", protocols.length > 0);
+
+        // Pull out one to disable specifically
+        String[] disabledProtocol = new String[] { protocols[protocols.length - 1] };
+        String[] trimmedProtocols = Arrays.copyOf(protocols, protocols.length - 1);
+        options.setDisabledProtocols(disabledProtocol);
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
 
         // verify the option took effect
         assertNotNull(engine);
@@ -240,7 +474,7 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledProtocols() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledProtocolsJDK() throws Exception {
         // Discover the default enabled protocols
         TransportOptions options = createJksSslOptions();
         SSLEngine directEngine = createSSLEngineDirectly(options);
@@ -255,8 +489,8 @@ public class TransportSupportTest extends QpidJmsTestCase {
         String[] remainingProtocols = new String[] { protocol2 };
         options.setEnabledProtocols(enabledProtocols);
         options.setDisabledProtocols(disabledProtocol);
-        SSLContext context = TransportSupport.createSslContext(options);
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
 
         // verify the option took effect, that the disabled protocols were removed from the enabled list.
         assertNotNull(engine);
@@ -264,7 +498,43 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitEnabledCiphers() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledProtocolsOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Discover the default enabled protocols
+        TransportOptions options = createJksSslOptions();
+        SSLEngine directEngine = createOpenSSLEngineDirectly(options);
+        String[] protocols = directEngine.getEnabledProtocols();
+        assertTrue("There were no initial protocols to choose from!", protocols.length > 1);
+
+        // Pull out two to enable, and one to disable specifically
+        String protocol1 = protocols[0];
+        String protocol2 = protocols[1];
+        String[] enabledProtocols = new String[] { protocol1, protocol2 };
+        String[] disabledProtocol = new String[] { protocol1 };
+        String[] remainingProtocols = new String[] { protocol2 };
+        options.setEnabledProtocols(enabledProtocols);
+        options.setDisabledProtocols(disabledProtocol);
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+
+        // Because Netty cannot currently disable SSLv2Hello in OpenSSL we need to account for it popping up.
+        ArrayList<String> remainingProtocolsList = new ArrayList<>(Arrays.asList(remainingProtocols));
+        if (!remainingProtocolsList.contains("SSLv2Hello")) {
+            remainingProtocolsList.add(0, "SSLv2Hello");
+        }
+
+        remainingProtocols = remainingProtocolsList.toArray(new String[remainingProtocolsList.size()]);
+
+        // verify the option took effect, that the disabled protocols were removed from the enabled list.
+        assertNotNull(engine);
+        assertEquals("Enabled protocols not as expected", remainingProtocolsList.size(), engine.getEnabledProtocols().length);
+        assertTrue("Enabled protocols not as expected", remainingProtocolsList.containsAll(Arrays.asList(engine.getEnabledProtocols())));
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledCiphersJDK() throws Exception {
         // Discover the default enabled ciphers
         TransportOptions options = createJksSslOptions();
         SSLEngine directEngine = createSSLEngineDirectly(options);
@@ -275,8 +545,31 @@ public class TransportSupportTest extends QpidJmsTestCase {
         String cipher = ciphers[0];
         String[] enabledCipher = new String[] { cipher };
         options.setEnabledCipherSuites(enabledCipher);
-        SSLContext context = TransportSupport.createSslContext(options);
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+
+        // verify the option took effect
+        assertNotNull(engine);
+        assertArrayEquals("Enabled ciphers not as expected", enabledCipher, engine.getEnabledCipherSuites());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledCiphersOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Discover the default enabled ciphers
+        TransportOptions options = createJksSslOptions();
+        SSLEngine directEngine = createOpenSSLEngineDirectly(options);
+        String[] ciphers = directEngine.getEnabledCipherSuites();
+        assertTrue("There were no initial ciphers to choose from!", ciphers.length > 0);
+
+        // Pull out one to enable specifically
+        String cipher = ciphers[0];
+        String[] enabledCipher = new String[] { cipher };
+        options.setEnabledCipherSuites(enabledCipher);
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
 
         // verify the option took effect
         assertNotNull(engine);
@@ -284,7 +577,7 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitDisabledCiphers() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitDisabledCiphersJDK() throws Exception {
         // Discover the default enabled ciphers
         TransportOptions options = createJksSslOptions();
         SSLEngine directEngine = createSSLEngineDirectly(options);
@@ -295,8 +588,31 @@ public class TransportSupportTest extends QpidJmsTestCase {
         String[] disabledCipher = new String[] { ciphers[ciphers.length - 1] };
         String[] trimmedCiphers = Arrays.copyOf(ciphers, ciphers.length - 1);
         options.setDisabledCipherSuites(disabledCipher);
-        SSLContext context = TransportSupport.createSslContext(options);
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+
+        // verify the option took effect
+        assertNotNull(engine);
+        assertArrayEquals("Enabled ciphers not as expected", trimmedCiphers, engine.getEnabledCipherSuites());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitDisabledCiphersOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Discover the default enabled ciphers
+        TransportOptions options = createJksSslOptions();
+        SSLEngine directEngine = createOpenSSLEngineDirectly(options);
+        String[] ciphers = directEngine.getEnabledCipherSuites();
+        assertTrue("There were no initial ciphers to choose from!", ciphers.length > 0);
+
+        // Pull out one to disable specifically
+        String[] disabledCipher = new String[] { ciphers[ciphers.length - 1] };
+        String[] trimmedCiphers = Arrays.copyOf(ciphers, ciphers.length - 1);
+        options.setDisabledCipherSuites(disabledCipher);
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
 
         // verify the option took effect
         assertNotNull(engine);
@@ -304,7 +620,7 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledCiphers() throws Exception {
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledCiphersJDK() throws Exception {
         // Discover the default enabled ciphers
         TransportOptions options = createJksSslOptions();
         SSLEngine directEngine = createSSLEngineDirectly(options);
@@ -319,8 +635,35 @@ public class TransportSupportTest extends QpidJmsTestCase {
         String[] remainingCipher = new String[] { cipher2 };
         options.setEnabledCipherSuites(enabledCiphers);
         options.setDisabledCipherSuites(disabledCipher);
-        SSLContext context = TransportSupport.createSslContext(options);
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+
+        // verify the option took effect, that the disabled ciphers were removed from the enabled list.
+        assertNotNull(engine);
+        assertArrayEquals("Enabled ciphers not as expected", remainingCipher, engine.getEnabledCipherSuites());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJksStoreWithExplicitEnabledAndDisabledCiphersOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Discover the default enabled ciphers
+        TransportOptions options = createJksSslOptions();
+        SSLEngine directEngine = createOpenSSLEngineDirectly(options);
+        String[] ciphers = directEngine.getEnabledCipherSuites();
+        assertTrue("There werent enough initial ciphers to choose from!", ciphers.length > 1);
+
+        // Pull out two to enable, and one to disable specifically
+        String cipher1 = ciphers[0];
+        String cipher2 = ciphers[1];
+        String[] enabledCiphers = new String[] { cipher1, cipher2 };
+        String[] disabledCipher = new String[] { cipher1 };
+        String[] remainingCipher = new String[] { cipher2 };
+        options.setEnabledCipherSuites(enabledCiphers);
+        options.setDisabledCipherSuites(disabledCipher);
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
 
         // verify the option took effect, that the disabled ciphers were removed from the enabled list.
         assertNotNull(engine);
@@ -328,13 +671,30 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJceksStore() throws Exception {
+    public void testCreateSslEngineFromJceksStoreJDK() throws Exception {
+        TransportOptions options = createJceksSslOptions();
+
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+        assertNotNull(engine);
+
+        List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
+        assertFalse(engineProtocols.isEmpty());
+    }
+
+    @Test
+    public void testCreateSslEngineFromJceksStoreOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
         TransportOptions options = createJceksSslOptions();
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SslContext context = TransportSupport.createOpenSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
         assertNotNull(engine);
 
         List<String> engineProtocols = Arrays.asList(engine.getEnabledProtocols());
@@ -342,41 +702,95 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     @Test
-    public void testCreateSslEngineFromJceksStoreWithExplicitEnabledProtocols() throws Exception {
+    public void testCreateSslEngineFromJceksStoreWithExplicitEnabledProtocolsJDK() throws Exception {
         TransportOptions options = createJceksSslOptions(ENABLED_PROTOCOLS);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
         assertNotNull(engine);
 
         assertArrayEquals("Enabled protocols not as expected", ENABLED_PROTOCOLS, engine.getEnabledProtocols());
     }
 
     @Test
-    public void testCreateSslEngineWithVerifyHost() throws Exception {
+    public void testCreateSslEngineFromJceksStoreWithExplicitEnabledProtocolsOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        // Try and disable all but the one we really want but for now expect that this one plus SSLv2Hello
+        // is going to come back until the netty code can successfully disable them all.
+        TransportOptions options = createJceksSslOptions(ENABLED_PROTOCOLS);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
+        assertNotNull(engine);
+
+        assertArrayEquals("Enabled protocols not as expected", ENABLED_OPENSSL_PROTOCOLS, engine.getEnabledProtocols());
+    }
+
+    @Test
+    public void testCreateSslEngineWithVerifyHostJDK() throws Exception {
+        TransportOptions options = createJksSslOptions();
+        options.setVerifyHost(true);
+
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+        assertNotNull(engine);
+
+        assertEquals("HTTPS", engine.getSSLParameters().getEndpointIdentificationAlgorithm());
+    }
+
+    @Test
+    public void testCreateSslEngineWithVerifyHostOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+        assumeTrue(OpenSsl.supportsHostnameValidation());
+
         TransportOptions options = createJksSslOptions();
         options.setVerifyHost(true);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SslContext context = TransportSupport.createOpenSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
         assertNotNull(engine);
 
         assertEquals("HTTPS", engine.getSSLParameters().getEndpointIdentificationAlgorithm());
     }
 
     @Test
-    public void testCreateSslEngineWithoutVerifyHost() throws Exception {
+    public void testCreateSslEngineWithoutVerifyHostJDK() throws Exception {
+        TransportOptions options = createJksSslOptions();
+        options.setVerifyHost(false);
+
+        SSLContext context = TransportSupport.createJdkSslContext(options);
+        assertNotNull(context);
+
+        SSLEngine engine = TransportSupport.createJdkSslEngine(null, context, options);
+        assertNotNull(engine);
+
+        assertNull(engine.getSSLParameters().getEndpointIdentificationAlgorithm());
+    }
+
+    @Test
+    public void testCreateSslEngineWithoutVerifyHostOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+        assumeTrue(OpenSsl.supportsHostnameValidation());
+
         TransportOptions options = createJksSslOptions();
         options.setVerifyHost(false);
 
-        SSLContext context = TransportSupport.createSslContext(options);
+        SslContext context = TransportSupport.createOpenSslContext(options);
         assertNotNull(context);
 
-        SSLEngine engine = TransportSupport.createSslEngine(context, options);
+        SSLEngine engine = TransportSupport.createOpenSslEngine(PooledByteBufAllocator.DEFAULT, null, context, options);
         assertNotNull(engine);
 
         assertNull(engine.getSSLParameters().getEndpointIdentificationAlgorithm());
@@ -388,7 +802,7 @@ public class TransportSupportTest extends QpidJmsTestCase {
         options.setKeyAlias(ALIAS_DOES_NOT_EXIST);
 
         try {
-            TransportSupport.createSslContext(options);
+            TransportSupport.createJdkSslContext(options);
             fail("Expected exception to be thrown");
         } catch (IllegalArgumentException iae) {
             // Expected
@@ -401,13 +815,96 @@ public class TransportSupportTest extends QpidJmsTestCase {
         options.setKeyAlias(ALIAS_CA_CERT);
 
         try {
-            TransportSupport.createSslContext(options);
+            TransportSupport.createJdkSslContext(options);
             fail("Expected exception to be thrown");
         } catch (IllegalArgumentException iae) {
             // Expected
         }
     }
 
+    @Test(timeout = 100000)
+    public void testIsOpenSSLPossible() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(false);
+        assertFalse(TransportSupport.isOpenSSLPossible(options));
+
+        options.setUseOpenSSL(true);
+        assertTrue(TransportSupport.isOpenSSLPossible(options));
+    }
+
+    @Test(timeout = 100000)
+    public void testIsOpenSSLPossibleWhenHostNameVerificationConfigured() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+        assumeTrue(OpenSsl.supportsHostnameValidation());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(true);
+
+        options.setVerifyHost(false);
+        assertTrue(TransportSupport.isOpenSSLPossible(options));
+
+        options.setVerifyHost(true);
+        assertTrue(TransportSupport.isOpenSSLPossible(options));
+    }
+
+    @Test(timeout = 100000)
+    public void testIsOpenSSLPossibleWhenKeyAliasIsSpecified() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+        assumeTrue(OpenSsl.supportsHostnameValidation());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(true);
+        options.setKeyAlias("alias");
+
+        assertFalse(TransportSupport.isOpenSSLPossible(options));
+    }
+
+    @Test(timeout = 100000)
+    public void testCreateSslHandlerJDK() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(false);
+
+        SslHandler handler = TransportSupport.createSslHandler(null, null, options);
+        assertNotNull(handler);
+        assertFalse(handler.engine() instanceof OpenSslEngine);
+    }
+
+    @Test(timeout = 100000)
+    public void testCreateSslHandlerOpenSSL() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(true);
+
+        SslHandler handler = TransportSupport.createSslHandler(PooledByteBufAllocator.DEFAULT, null, options);
+        assertNotNull(handler);
+        assertTrue(handler.engine() instanceof OpenSslEngine);
+    }
+
+    @Test(timeout = 100000)
+    public void testCreateOpenSSLEngineFailsWhenAllocatorMissing() throws Exception {
+        assumeTrue(OpenSsl.isAvailable());
+        assumeTrue(OpenSsl.supportsKeyManagerFactory());
+
+        TransportOptions options = new TransportOptions();
+        options.setUseOpenSSL(true);
+
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        try {
+            TransportSupport.createOpenSslEngine(null, null, context, options);
+            fail("Should throw IllegalArgumentException for null allocator.");
+        } catch (IllegalArgumentException iae) {}
+    }
+
     private TransportOptions createJksSslOptions() {
         return createJksSslOptions(null);
     }
@@ -466,8 +963,14 @@ public class TransportSupportTest extends QpidJmsTestCase {
     }
 
     private SSLEngine createSSLEngineDirectly(TransportOptions options) throws Exception {
-        SSLContext context = TransportSupport.createSslContext(options);
+        SSLContext context = TransportSupport.createJdkSslContext(options);
         SSLEngine engine = context.createSSLEngine();
         return engine;
     }
+
+    private SSLEngine createOpenSSLEngineDirectly(TransportOptions options) throws Exception {
+        SslContext context = TransportSupport.createOpenSslContext(options);
+        SSLEngine engine = context.newEngine(PooledByteBufAllocator.DEFAULT);
+        return engine;
+    }
 }

http://git-wip-us.apache.org/repos/asf/qpid-jms/blob/d5cde3ed/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/netty/NettyOpenSSLWssTransportTest.java
----------------------------------------------------------------------
diff --git a/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/netty/NettyOpenSSLWssTransportTest.java b/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/netty/NettyOpenSSLWssTransportTest.java
new file mode 100644
index 0000000..cd80ca1
--- /dev/null
+++ b/qpid-jms-client/src/test/java/org/apache/qpid/jms/transports/netty/NettyOpenSSLWssTransportTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.qpid.jms.transports.netty;
+
+import java.net.URI;
+
+import org.apache.qpid.jms.transports.TransportListener;
+import org.apache.qpid.jms.transports.TransportOptions;
+
+/**
+ * Test the NettyWsTransport with channel level security enabled.
+ */
+public class NettyOpenSSLWssTransportTest extends NettyOpenSslTransportTest {
+
+    @Override
+    protected NettyEchoServer createEchoServer(TransportOptions options, boolean needClientAuth) {
+        return new NettyEchoServer(options, true, needClientAuth, true);
+    }
+
+    @Override
+    protected NettyTcpTransport createTransport(URI serverLocation, TransportListener listener, TransportOptions options) {
+        if (listener == null) {
+            return new NettyWsTransport(serverLocation, options, true);
+        } else {
+            return new NettyWsTransport(listener, serverLocation, options, true);
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org