You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by ni...@apache.org on 2022/09/19 10:00:19 UTC

[bookkeeper] 08/22: [Client] Switch to rely on Netty for Hostname Verification (#3310)

This is an automated email from the ASF dual-hosted git repository.

nicoloboschi pushed a commit to branch ds-4.14
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git

commit 11bb11c9a52e97563fcba46434846c5fe6d4d97c
Author: Michael Marshall <mm...@apache.org>
AuthorDate: Sun Jun 5 01:39:35 2022 -0500

    [Client] Switch to rely on Netty for Hostname Verification (#3310)
    
    ### Motivation
    
    Currently, we initiate hostname verification for the Bookkeeper Client in the `PerChannelBookieClient` class. In order to simplify the code, I propose that we refactor the client so it relies on Netty, its SslHandler/SslEngine, and the JVM, to perform the hostname verification.
    
    When HTTPS is configured as the endpoint verification algorithm, it uses [RFC 2818](https://datatracker.ietf.org/doc/html/rfc2818) to perform hostname verification. This is defined by the Java Security Standard Algorithm Names documentation for JDK versions 8, 11, and 17. Here are the official docs:
    
    * https://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html
    * https://docs.oracle.com/en/java/javase/11/docs/specs/security/standard-names.html
    * https://docs.oracle.com/en/java/javase/17/docs/specs/security/standard-names.html
    
    ### Changes
    
    * Rely on Netty and the SslEngine to perform hostname verification. With this change, CN matching is now deprecated, which brings the bookkeeper client in alignment with RFC 2818.
    * Add new method to the `SecurityHandlerFactory` interface. It is named `newTLSHandler` and takes the `host` and `port` of the remote peer when creating a new SslEngine. To ensure backwards compatibility, the default implementation will call the original method. Note that the remote host and port are only needed when a client is using them for hostname verification.
    
    (cherry picked from commit 6b22f852d31c6e325e0ac5019a34c36acc271ed3)
    (cherry picked from commit 315195e7c468f82f7802d05c38ac7a961a90d446)
---
 .../org/apache/bookkeeper/proto/AuthHandler.java   | 35 ----------------------
 .../bookkeeper/proto/PerChannelBookieClient.java   | 22 +++++---------
 .../bookkeeper/tls/SecurityHandlerFactory.java     |  4 +++
 .../apache/bookkeeper/tls/TLSContextFactory.java   | 19 +++++++++++-
 .../java/org/apache/bookkeeper/tls/TestTLS.java    |  3 +-
 5 files changed, 30 insertions(+), 53 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java
index e05846e061..f13d3a4c49 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java
@@ -29,30 +29,24 @@ import io.netty.channel.ChannelDuplexHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
 import io.netty.channel.ChannelPromise;
-import io.netty.handler.ssl.SslHandler;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicLong;
 
-import javax.net.ssl.SSLSession;
-
 import org.apache.bookkeeper.auth.AuthCallbacks;
 import org.apache.bookkeeper.auth.AuthToken;
 import org.apache.bookkeeper.auth.BookieAuthProvider;
 import org.apache.bookkeeper.auth.ClientAuthProvider;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage;
-import org.apache.http.conn.ssl.DefaultHostnameVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 class AuthHandler {
     static final Logger LOG = LoggerFactory.getLogger(AuthHandler.class);
-    private static final DefaultHostnameVerifier HOSTNAME_VERIFIER = new DefaultHostnameVerifier();
 
     static class ServerSideHandler extends ChannelInboundHandlerAdapter {
         volatile boolean authenticated = false;
@@ -444,35 +438,6 @@ class AuthHandler {
                 }
             }
         }
-
-        public boolean verifyTlsHostName(Channel channel) {
-            SslHandler sslHandler = channel.pipeline().get(SslHandler.class);
-            if (sslHandler == null) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("can't perform hostname-verification on non-ssl channel {}", channel);
-                }
-                return true;
-            }
-            SSLSession sslSession = sslHandler.engine().getSession();
-            String hostname = null;
-            if (channel.remoteAddress() instanceof InetSocketAddress) {
-                hostname = ((InetSocketAddress) channel.remoteAddress()).getHostName();
-            } else {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("can't get remote hostName on ssl session {}", channel);
-                }
-                return true;
-            }
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Verifying HostName for {}, Cipher {}, Protocols {}, on {}", hostname,
-                        sslSession.getCipherSuite(), sslSession.getProtocol(), channel);
-            }
-            boolean verification = HOSTNAME_VERIFIER.verify(hostname, sslSession);
-            if (!verification) {
-                LOG.warn("Failed to validate hostname verification {} on {}", hostname, channel);
-            }
-            return verification;
-        }
     }
 
     @SuppressWarnings("serial")
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index f248a40607..12209a636d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -59,6 +59,7 @@ import io.netty.util.Recycler.Handle;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GenericFutureListener;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.security.cert.Certificate;
@@ -1480,7 +1481,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
     void initTLSHandshake() {
         // create TLS handler
         PerChannelBookieClient parentObj = PerChannelBookieClient.this;
-        SslHandler handler = parentObj.shFactory.newTLSHandler();
+        InetSocketAddress address = (InetSocketAddress) channel.remoteAddress();
+        SslHandler handler = parentObj.shFactory.newTLSHandler(address.getHostName(), address.getPort());
         channel.pipeline().addFirst(parentObj.shFactory.getHandlerName(), handler);
         handler.handshakeFuture().addListener(new GenericFutureListener<Future<Channel>>() {
                 @Override
@@ -1504,14 +1506,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
                             state = ConnectionState.CONNECTED;
                             AuthHandler.ClientSideHandler authHandler = future.get().pipeline()
                                     .get(AuthHandler.ClientSideHandler.class);
-                        if (conf.getHostnameVerificationEnabled() && !authHandler.verifyTlsHostName(channel)) {
-                            // add HostnameVerification or private classes not
-                            // for validation
-                            rc = BKException.Code.UnauthorizedAccessException;
-                        } else {
-                                authHandler.authProvider.onProtocolUpgrade();
-                                activeTlsChannelCounter.inc();
-                            }
+                            authHandler.authProvider.onProtocolUpgrade();
+                            activeTlsChannelCounter.inc();
                         } else if (future.isSuccess()
                                 && (state == ConnectionState.CLOSED || state == ConnectionState.DISCONNECTED)) {
                             LOG.warn("Closed before TLS handshake completed, clean up: {}, current state {}",
@@ -2462,12 +2458,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
                     state = ConnectionState.CONNECTED;
                     AuthHandler.ClientSideHandler authHandler = future.channel().pipeline()
                             .get(AuthHandler.ClientSideHandler.class);
-                    if (conf.getHostnameVerificationEnabled() && !authHandler.verifyTlsHostName(channel)) {
-                        rc = BKException.Code.UnauthorizedAccessException;
-                    } else {
-                        authHandler.authProvider.onProtocolUpgrade();
-                        activeTlsChannelCounter.inc();
-                    }
+                    authHandler.authProvider.onProtocolUpgrade();
+                    activeTlsChannelCounter.inc();
                 } else if (future.isSuccess() && (state == ConnectionState.CLOSED
                     || state == ConnectionState.DISCONNECTED)) {
                     LOG.warn("Closed before connection completed, clean up: {}, current state {}",
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java
index 5b43744fd1..5d4bc3b088 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/SecurityHandlerFactory.java
@@ -41,4 +41,8 @@ public interface SecurityHandlerFactory {
     void init(NodeType type, AbstractConfiguration conf, ByteBufAllocator allocator) throws SecurityException;
 
     SslHandler newTLSHandler();
+
+    default SslHandler newTLSHandler(String host, int port) {
+        return this.newTLSHandler();
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java
index fdb2d019c3..280c4dccf1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tls/TLSContextFactory.java
@@ -46,6 +46,7 @@ import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
 import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLParameters;
 import javax.net.ssl.TrustManagerFactory;
 
 import lombok.extern.slf4j.Slf4j;
@@ -143,6 +144,7 @@ public class TLSContextFactory implements SecurityHandlerFactory {
     }
 
     private static final String TLSCONTEXT_HANDLER_NAME = "tls";
+    private NodeType type;
     private String[] protocols;
     private String[] ciphers;
     private volatile SslContext sslContext;
@@ -475,6 +477,7 @@ public class TLSContextFactory implements SecurityHandlerFactory {
             throws SecurityException {
         this.allocator = allocator;
         this.config = conf;
+        this.type = type;
         final String enabledProtocols;
         final String enabledCiphers;
         certRefreshTime = TimeUnit.SECONDS.toMillis(conf.getTLSCertFilesRefreshDurationSeconds());
@@ -522,7 +525,12 @@ public class TLSContextFactory implements SecurityHandlerFactory {
 
     @Override
     public SslHandler newTLSHandler() {
-        SslHandler sslHandler = getSSLContext().newHandler(allocator);
+        return this.newTLSHandler(null, -1);
+    }
+
+    @Override
+    public SslHandler newTLSHandler(String peer, int port) {
+        SslHandler sslHandler = getSSLContext().newHandler(allocator, peer, port);
 
         if (protocols != null && protocols.length != 0) {
             sslHandler.engine().setEnabledProtocols(protocols);
@@ -538,6 +546,15 @@ public class TLSContextFactory implements SecurityHandlerFactory {
             log.debug("Enabled cipher suites: {} ", Arrays.toString(sslHandler.engine().getEnabledCipherSuites()));
         }
 
+        if (type == NodeType.Client && ((ClientConfiguration) config).getHostnameVerificationEnabled()) {
+            SSLParameters sslParameters = sslHandler.engine().getSSLParameters();
+            sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+            sslHandler.engine().setSSLParameters(sslParameters);
+            if (log.isDebugEnabled()) {
+                log.debug("Enabled endpointIdentificationAlgorithm: HTTPS");
+            }
+        }
+
         return sslHandler;
     }
 
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
index bc3fbcc3ba..2f398b0044 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/tls/TestTLS.java
@@ -44,7 +44,6 @@ import org.apache.bookkeeper.auth.AuthToken;
 import org.apache.bookkeeper.auth.BookieAuthProvider;
 import org.apache.bookkeeper.auth.ClientAuthProvider;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BKException.BKUnauthorizedAccessException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
@@ -1020,7 +1019,7 @@ public class TestTLS extends BookKeeperClusterTestCase {
         try {
             testClient(clientConf, numBookies);
             fail("should have failed with unauthorized exception");
-        } catch (BKUnauthorizedAccessException e) {
+        } catch (BKException.BKNotEnoughBookiesException e) {
             // Ok.
         }
     }