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

[GitHub] rdhabalia closed pull request #1214: Support hostname verification on proxy to broker connection

rdhabalia closed pull request #1214: Support hostname verification on proxy to broker connection
URL: https://github.com/apache/incubator-pulsar/pull/1214
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/conf/proxy.conf b/conf/proxy.conf
index dafcfc401..78ad9255d 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 9fc2ff57d..41bf20c4a 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.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.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 @@ protected void initChannel(SocketChannel ch) throws Exception {
                 }
                 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 @@ protected void initChannel(SocketChannel ch) throws Exception {
             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 @@ protected void initChannel(SocketChannel ch) throws Exception {
     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 @@ protected void handleConnected(CommandConnected connected) {
             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 void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
             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 0cd2c2936..e5906baee 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 @@
     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 void setTlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) {
         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 1619e2564..b370d63a3 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.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 @@ protected void setup() throws Exception {
  
         proxyService = Mockito.spy(new ProxyService(proxyConfig));
 
-        proxyService.start();
     }
 
     @AfterMethod
@@ -143,9 +143,13 @@ protected void cleanup() throws Exception {
         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 @@ protected void cleanup() throws Exception {
     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 void textProxyAuthorization() throws Exception {
     }
 
     @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 @@ protected final void createAdminClient() throws Exception {
         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);
     }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services