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 2020/03/20 00:04:15 UTC

[GitHub] [pulsar] rdhabalia opened a new pull request #6566: [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

rdhabalia opened a new pull request #6566: [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566
 
 
   ### Motivation
   Implementation of [PIP-60](https://github.com/apache/pulsar/wiki/PIP-60:-Support-Proxy-server-with-SNI-routing)
   A proxy server is a go‑between or intermediary server that forwards requests from multiple clients to different servers across the Internet. The proxy server can act as a “traffic cop,” in both forward and reverse proxy scenarios, and adds various benefits in your system such as load balancing, performance, security, auto-scaling, etc.. There are already many proxy servers already available in the market which are fast, scalable and more importantly covers various essential security aspects that are needed by the large organization to securely share their confidential data over the network. Pulsar already provides proxy implementation PIP-1 which acts as a reverse proxy and creates a gateway in front of brokers. However, pulsar doesn’t provide support to use other proxies such as Apache traffic server (ATS), HAProxy, Nginx, Envoy those are more scalable and secured. Most of these proxy-servers support SNI ROUTING which can route traffic to a destination without having to terminate the SSL connection. Routing at layer 4 gives greater transparency because the outbound connection is determined by examining the destination address in the client TCP packets.
   
   ### Modification
   https://github.com/apache/pulsar/wiki/PIP-60:-Support-Proxy-server-with-SNI-routing#changes

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r408534778
 
 

 ##########
 File path: pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java
 ##########
 @@ -58,6 +60,20 @@
         example = "pulsar+ssl://pulsar.example.com:6651"
     )
     private String brokerServiceUrlTls;
+    @ApiModelProperty(
+        name = "proxyServiceUrl",
+        value = "Proxy-service url when client would like to connect to broker via proxy.",
+        example = "pulsar+ssl://ats-proxy.example.com:4443 or "
+                    + "pulsar://ats-proxy.example.com:4080"
+    )
+    private String proxyServiceUrl;
 
 Review comment:
   yes, SNI works on layer-4 so, it requires binary URL for SNI routing.  however, we added proxyUrl placeholder extensible by adding `proxyType` and right now we support SNI and in future we can have other type which can support HTTP as well. So, I think we don't want to restrict with binary url only. But I have a plan to add a separate document with examples of different proxy-solutions(ATS, Envoy) as multiple users are interested in this solution.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r405728748
 
 

 ##########
 File path: pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProxyProtocolTest.java
 ##########
 @@ -0,0 +1,105 @@
+/**
+ * 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.pulsar.client.api;
+
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.pulsar.client.impl.auth.AuthenticationTls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.Test;
+
+public class ProxyProtocolTest extends TlsProducerConsumerBase {
+    private static final Logger log = LoggerFactory.getLogger(ProxyProtocolTest.class);
+
+    @Test
+    public void testSniProxyProtocol() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrlTls();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
 
 Review comment:
   Should we close the client or add `@Cleanup` annotation?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r408541595
 
 

 ##########
 File path: pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProxyProtocolTest.java
 ##########
 @@ -0,0 +1,105 @@
+/**
+ * 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.pulsar.client.api;
+
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.pulsar.client.impl.auth.AuthenticationTls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.Test;
+
+public class ProxyProtocolTest extends TlsProducerConsumerBase {
+    private static final Logger log = LoggerFactory.getLogger(ProxyProtocolTest.class);
+
+    @Test
+    public void testSniProxyProtocol() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrlTls();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        // should be able to create producer successfully
+        pulsarClient.newProducer().topic(topicName).create();
+    }
+
+    @Test
+    public void testSniProxyProtocolWithInvalidProxyUrl() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String proxyHost = "invalid-url";
+        String proxyUrl = "pulsar+ssl://" + proxyHost + ":5555";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        try {
+            pulsarClient.newProducer().topic(topicName).create();
+            fail("should have failed due to invalid url");
+        } catch (PulsarClientException e) {
+            assertTrue(e.getMessage().contains(proxyHost));
+        }
+    }
+
+    @Test
+    public void testSniProxyProtocolWithoutTls() throws Exception {
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrl();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        try {
+            pulsarClient.newProducer().topic(topicName).create();
+            fail("should have failed due to invalid url");
 
 Review comment:
   sure. fixed it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r408541468
 
 

 ##########
 File path: pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProxyProtocolTest.java
 ##########
 @@ -0,0 +1,105 @@
+/**
+ * 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.pulsar.client.api;
+
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.pulsar.client.impl.auth.AuthenticationTls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.Test;
+
+public class ProxyProtocolTest extends TlsProducerConsumerBase {
+    private static final Logger log = LoggerFactory.getLogger(ProxyProtocolTest.class);
+
+    @Test
+    public void testSniProxyProtocol() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrlTls();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
 
 Review comment:
   yes.. will add it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r408534778
 
 

 ##########
 File path: pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java
 ##########
 @@ -58,6 +60,20 @@
         example = "pulsar+ssl://pulsar.example.com:6651"
     )
     private String brokerServiceUrlTls;
+    @ApiModelProperty(
+        name = "proxyServiceUrl",
+        value = "Proxy-service url when client would like to connect to broker via proxy.",
+        example = "pulsar+ssl://ats-proxy.example.com:4443 or "
+                    + "pulsar://ats-proxy.example.com:4080"
+    )
+    private String proxyServiceUrl;
 
 Review comment:
   yes, SNI works on layer-4 so, it requires binary URL. let me add such documentation. Also, I have a plan to add a separate document with examples of different proxy types as multiple users are interested in this solution.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r405729449
 
 

 ##########
 File path: pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProxyProtocolTest.java
 ##########
 @@ -0,0 +1,105 @@
+/**
+ * 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.pulsar.client.api;
+
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.pulsar.client.impl.auth.AuthenticationTls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.Test;
+
+public class ProxyProtocolTest extends TlsProducerConsumerBase {
+    private static final Logger log = LoggerFactory.getLogger(ProxyProtocolTest.class);
+
+    @Test
+    public void testSniProxyProtocol() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrlTls();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        // should be able to create producer successfully
+        pulsarClient.newProducer().topic(topicName).create();
+    }
+
+    @Test
+    public void testSniProxyProtocolWithInvalidProxyUrl() throws Exception {
+
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String proxyHost = "invalid-url";
+        String proxyUrl = "pulsar+ssl://" + proxyHost + ":5555";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH);
+        clientBuilder.authentication(AuthenticationTls.class.getName(), authParams);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        try {
+            pulsarClient.newProducer().topic(topicName).create();
+            fail("should have failed due to invalid url");
+        } catch (PulsarClientException e) {
+            assertTrue(e.getMessage().contains(proxyHost));
+        }
+    }
+
+    @Test
+    public void testSniProxyProtocolWithoutTls() throws Exception {
+        // Client should try to connect to proxy and pass broker-url as SNI header
+        String proxyUrl = pulsar.getBrokerServiceUrl();
+        String brokerServiceUrl = "pulsar+ssl://1.1.1.1:6651";
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(brokerServiceUrl)
+                .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI).operationTimeout(1000, TimeUnit.MILLISECONDS);
+
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        try {
+            pulsarClient.newProducer().topic(topicName).create();
+            fail("should have failed due to invalid url");
 
 Review comment:
   ```suggestion
               fail("should have failed due to non-tls url");
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#discussion_r405737897
 
 

 ##########
 File path: pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java
 ##########
 @@ -58,6 +60,20 @@
         example = "pulsar+ssl://pulsar.example.com:6651"
     )
     private String brokerServiceUrlTls;
+    @ApiModelProperty(
+        name = "proxyServiceUrl",
+        value = "Proxy-service url when client would like to connect to broker via proxy.",
+        example = "pulsar+ssl://ats-proxy.example.com:4443 or "
+                    + "pulsar://ats-proxy.example.com:4080"
+    )
+    private String proxyServiceUrl;
 
 Review comment:
   based on my understanding, this service URL here can only be the binary protocol service URL correct? We can't use the HTTP-protocol service URL. If that's the case, can you document this setting in `ClusterData` and the related command-line tools? It might be good to add a validation to throw exceptions if it is not a binary protocol service url.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [pulsar] rdhabalia commented on issue #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar

Posted by GitBox <gi...@apache.org>.
rdhabalia commented on issue #6566: [PIP-60] [Proxy-Server] Support SNI routing to support various proxy-server in pulsar
URL: https://github.com/apache/pulsar/pull/6566#issuecomment-613776642
 
 
   @sijie addressed the comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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