You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by rd...@apache.org on 2018/02/11 17:01:58 UTC

[incubator-pulsar] branch master updated: Support hostname verification on proxy to broker connection (#1214)

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

rdhabalia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new a27a1e2  Support hostname verification on proxy to broker connection (#1214)
a27a1e2 is described below

commit a27a1e2dd669bc2082cf8f8cefb30becf56fa616
Author: Rajan Dhabalia <rd...@apache.org>
AuthorDate: Sun Feb 11 09:01:55 2018 -0800

    Support hostname verification on proxy to broker connection (#1214)
    
    * Support hostname verification on proxy to broker connection
    
    * remove dep and rename config
---
 conf/proxy.conf                                    |  3 +
 .../pulsar/proxy/server/DirectProxyHandler.java    | 42 +++++++++++-
 .../pulsar/proxy/server/ProxyConfiguration.java    | 10 +++
 .../server/ProxyWithProxyAuthorizationTest.java    | 75 +++++++++++++++++++---
 4 files changed, 121 insertions(+), 9 deletions(-)

diff --git a/conf/proxy.conf b/conf/proxy.conf
index dafcfc4..78ad925 100644
--- a/conf/proxy.conf
+++ b/conf/proxy.conf
@@ -74,3 +74,6 @@ tlsCertificateFilePath=
 
 # Path for the TLS private key file
 tlsKeyFilePath=
+
+# Validates hostname when proxy creates tls connection with broker
+tlsHostnameVerificationEnabled=false
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
index 9fc2ff5..41bf20c 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
@@ -24,6 +24,8 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.cert.X509Certificate;
 
+import javax.net.ssl.SSLSession;
+
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
 import org.apache.pulsar.common.api.Commands;
@@ -37,16 +39,19 @@ import io.netty.buffer.ByteBuf;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslHandler;
 import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.FutureListener;
+import org.apache.http.conn.ssl.DefaultHostnameVerifier;
 
 public class DirectProxyHandler {
 
@@ -104,7 +109,7 @@ public class DirectProxyHandler {
                 }
                 ch.pipeline().addLast("frameDecoder",
                         new LengthFieldBasedFrameDecoder(PulsarDecoder.MaxFrameSize, 0, 4, 0, 4));
-                ch.pipeline().addLast(new ProxyBackendHandler());
+                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config));
             }
         });
 
@@ -124,7 +129,10 @@ public class DirectProxyHandler {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 inboundChannel.close();
+                return;
             }
+            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline().get("proxyOutboundHandler");
+            cnx.setRemoteHostName(targetBroker.getHost());
         });
     }
 
@@ -135,9 +143,17 @@ public class DirectProxyHandler {
     public class ProxyBackendHandler extends PulsarDecoder implements FutureListener<Void> {
 
         private BackendState state = BackendState.Init;
+        private String remoteHostName;
+        protected ChannelHandlerContext ctx;
+        private ProxyConfiguration config;
+        
+        public ProxyBackendHandler(ProxyConfiguration config) {
+            this.config = config;
+        }
 
         @Override
         public void channelActive(ChannelHandlerContext ctx) throws Exception {
+            this.ctx = ctx;
             // Send the Connect command to broker
             String authData = "";
             if (authentication.getAuthData().hasDataFromCommand()) {
@@ -195,6 +211,15 @@ public class DirectProxyHandler {
             if (log.isDebugEnabled()) {
                 log.debug("[{}] [{}] Received Connected from broker", inboundChannel, outboundChannel);
             }
+            
+            if (config.isTlsHostnameVerificationEnabled() && remoteHostName != null
+                    && !verifyTlsHostName(remoteHostName, ctx)) {
+                // close the connection if host-verification failed with the broker
+                log.warn("[{}] Failed to verify hostname of {}", ctx.channel(), remoteHostName);
+                ctx.close();
+                return;
+            }
+            
             state = BackendState.HandshakeCompleted;
 
             inboundChannel.writeAndFlush(Commands.newConnected(connected.getProtocolVersion())).addListener(future -> {
@@ -220,6 +245,21 @@ public class DirectProxyHandler {
             log.warn("[{}] [{}] Caught exception: {}", inboundChannel, outboundChannel, cause.getMessage(), cause);
             ctx.close();
         }
+        
+        public void setRemoteHostName(String remoteHostName) {
+            this.remoteHostName = remoteHostName;
+        }
+
+        private boolean verifyTlsHostName(String hostname, ChannelHandlerContext ctx) {
+            ChannelHandler sslHandler = ctx.channel().pipeline().get("tls");
+
+            SSLSession sslSession = null;
+            if (sslHandler != null) {
+                sslSession = ((SslHandler) sslHandler).engine().getSession();
+                return (new DefaultHostnameVerifier()).verify(hostname, sslSession);
+            }
+            return false;
+        }
     }
 
     private static final Logger log = LoggerFactory.getLogger(DirectProxyHandler.class);
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
index 0cd2c29..e5906ba 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java
@@ -82,6 +82,8 @@ public class ProxyConfiguration implements PulsarConfiguration {
     private String tlsTrustCertsFilePath;
     // Accept untrusted TLS certificate from client
     private boolean tlsAllowInsecureConnection = false;
+    // Validates hostname when proxy creates tls connection with broker
+    private boolean tlsHostnameVerificationEnabled = false;
 
     private Properties properties = new Properties();
 
@@ -213,6 +215,14 @@ public class ProxyConfiguration implements PulsarConfiguration {
         this.tlsAllowInsecureConnection = tlsAllowInsecureConnection;
     }
 
+    public boolean isTlsHostnameVerificationEnabled() {
+        return tlsHostnameVerificationEnabled;
+    }
+
+    public void setTlsHostnameVerificationEnabled(boolean tlsHostnameVerificationEnabled) {
+        this.tlsHostnameVerificationEnabled = tlsHostnameVerificationEnabled;
+    }
+
     public String getBrokerClientAuthenticationPlugin() {
         return brokerClientAuthenticationPlugin;
     }
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithProxyAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithProxyAuthorizationTest.java
index 1619e25..b370d63 100644
--- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithProxyAuthorizationTest.java
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithProxyAuthorizationTest.java
@@ -29,6 +29,7 @@ import org.apache.bookkeeper.test.PortManager;
 import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.api.Authentication;
+import org.apache.pulsar.client.api.ClientConfiguration;
 import org.apache.pulsar.client.api.Consumer;
 import org.apache.pulsar.client.api.ConsumerConfiguration;
 import org.apache.pulsar.client.api.Message;
@@ -133,7 +134,6 @@ public class ProxyWithProxyAuthorizationTest extends ProducerConsumerBase {
  
         proxyService = Mockito.spy(new ProxyService(proxyConfig));
 
-        proxyService.start();
     }
 
     @AfterMethod
@@ -143,9 +143,13 @@ public class ProxyWithProxyAuthorizationTest extends ProducerConsumerBase {
         proxyService.close();
     }
 
+    void startProxy() throws Exception {
+        proxyService.start();
+    }
+    
     /**
      * <pre>
-     * It verifies e2e tls + Authentication + Authorization (client -> proxy -> broker>
+     * It verifies e2e tls + Authentication + Authorization (client -> proxy -> broker)
      * 
      * 1. client connects to proxy over tls and pass auth-data
      * 2. proxy authenticate client and retrieve client-role 
@@ -161,10 +165,12 @@ public class ProxyWithProxyAuthorizationTest extends ProducerConsumerBase {
     public void textProxyAuthorization() throws Exception {
         log.info("-- Starting {} test --", methodName);
 
+        startProxy();
         createAdminClient();
         final String proxyServiceUrl = "pulsar://localhost:" + proxyConfig.getServicePortTls();
         // create a client which connects to proxy over tls and pass authData
-        PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, false);
+        ClientConfiguration clientConf = new ClientConfiguration();
+        PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, clientConf);
 
         String namespaceName = "my-property/proxy-authorization/my-ns";
         
@@ -209,13 +215,68 @@ public class ProxyWithProxyAuthorizationTest extends ProducerConsumerBase {
     }
 
     @Test(dataProvider = "hostnameVerification")
-    public void textProxyAuthorizationTlsHostVerification(boolean hostnameVerificationEnabled) throws Exception {
+    public void textTlsHostVerificationProxyToClient(boolean hostnameVerificationEnabled) throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        startProxy();
+        createAdminClient();
+        final String proxyServiceUrl = "pulsar://localhost:" + proxyConfig.getServicePortTls();
+        // create a client which connects to proxy over tls and pass authData
+        ClientConfiguration clientConf = new ClientConfiguration();
+        clientConf.setTlsHostnameVerificationEnable(hostnameVerificationEnabled);
+        PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, clientConf);
+
+        String namespaceName = "my-property/proxy-authorization/my-ns";
+
+        admin.properties().createProperty("my-property",
+                new PropertyAdmin(Lists.newArrayList("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
+        admin.namespaces().createNamespace(namespaceName);
+
+        admin.namespaces().grantPermissionOnNamespace(namespaceName, "Proxy",
+                Sets.newHashSet(AuthAction.consume, AuthAction.produce));
+        admin.namespaces().grantPermissionOnNamespace(namespaceName, "Client",
+                Sets.newHashSet(AuthAction.consume, AuthAction.produce));
+
+        ConsumerConfiguration conf = new ConsumerConfiguration();
+        conf.setSubscriptionType(SubscriptionType.Exclusive);
+        try {
+            Consumer consumer = proxyClient.subscribe("persistent://my-property/proxy-authorization/my-ns/my-topic1",
+                    "my-subscriber-name", conf);
+            if (hostnameVerificationEnabled) {
+                Assert.fail("Connection should be failed due to hostnameVerification enabled");
+            }
+        } catch (PulsarClientException e) {
+            if (!hostnameVerificationEnabled) {
+                Assert.fail("Consumer should be created because hostnameverification is disabled");
+            }
+        }
+
+        log.info("-- Exiting {} test --", methodName);
+    }
+
+    /**
+     * It verifies hostname verification at proxy when proxy tries to connect with broker. Proxy performs hostname
+     * verification when broker sends its certs over tls .
+     * <pre>
+     * 1. Broker sends certs back to proxy with CN="Broker" however, proxy tries to connect with hostname=localhost
+     * 2. so, client fails to create consumer if proxy is enabled with hostname verification
+     * </pre>
+     * 
+     * @param hostnameVerificationEnabled
+     * @throws Exception
+     */
+    @Test(dataProvider = "hostnameVerification")
+    public void textTlsHostVerificationProxyToBroker(boolean hostnameVerificationEnabled) throws Exception {
         log.info("-- Starting {} test --", methodName);
 
+        proxyConfig.setTlsHostnameVerificationEnabled(hostnameVerificationEnabled);
+        startProxy();
         createAdminClient();
         final String proxyServiceUrl = "pulsar://localhost:" + proxyConfig.getServicePortTls();
         // create a client which connects to proxy over tls and pass authData
-        PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, hostnameVerificationEnabled);
+        ClientConfiguration clientConf = new ClientConfiguration();
+        clientConf.setOperationTimeout(1, TimeUnit.SECONDS);
+        PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, clientConf);
 
         String namespaceName = "my-property/proxy-authorization/my-ns";
 
@@ -261,19 +322,17 @@ public class ProxyWithProxyAuthorizationTest extends ProducerConsumerBase {
         admin = spy(new PulsarAdmin(brokerUrlTls, clientConf));
     }
     
-    private PulsarClient createPulsarClient(String proxyServiceUrl, boolean hosnameVerificationEnabled) throws PulsarClientException {
+    private PulsarClient createPulsarClient(String proxyServiceUrl, ClientConfiguration clientConf) throws PulsarClientException {
         Map<String, String> authParams = Maps.newHashMap();
         authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
         authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
         Authentication authTls = new AuthenticationTls();
         authTls.configure(authParams);
-        org.apache.pulsar.client.api.ClientConfiguration clientConf = new org.apache.pulsar.client.api.ClientConfiguration();
         clientConf.setStatsInterval(0, TimeUnit.SECONDS);
         clientConf.setTlsTrustCertsFilePath(TLS_CLIENT_TRUST_CERT_FILE_PATH);
         clientConf.setTlsAllowInsecureConnection(true);
         clientConf.setAuthentication(authTls);
         clientConf.setUseTls(true);
-        clientConf.setTlsHostnameVerificationEnable(hosnameVerificationEnabled);
         return PulsarClient.create(proxyServiceUrl, clientConf);
     }
 }

-- 
To stop receiving notification emails like this one, please contact
rdhabalia@apache.org.