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

[pulsar] branch master updated: Integration test and documentation for Proxy w/o ZK (#3352)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new aa3fd10  Integration test and documentation for Proxy w/o ZK (#3352)
aa3fd10 is described below

commit aa3fd10c87f6b2cc0a3d57dd67b4e205bb49a0d0
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Fri Jan 11 19:27:54 2019 +0100

    Integration test and documentation for Proxy w/o ZK (#3352)
    
    Adds an integration test which creates a proxy which connects to the
    brokers using broker URLs rather than service discovery.
    
    Adds documentation for configuring the proxy in this way.
    
    Issue: #2405
---
 site2/docs/administration-proxy.md                 | 49 +++++++++++++++++++---
 site2/docs/security-authorization.md               |  4 +-
 .../pulsar/tests/integration/proxy/TestProxy.java  | 42 +++++++++++++++----
 .../integration/topologies/PulsarClusterSpec.java  |  2 +
 4 files changed, 83 insertions(+), 14 deletions(-)

diff --git a/site2/docs/administration-proxy.md b/site2/docs/administration-proxy.md
index 3321e80..75efdcb 100644
--- a/site2/docs/administration-proxy.md
+++ b/site2/docs/administration-proxy.md
@@ -4,19 +4,56 @@ title: The Pulsar proxy
 sidebar_label: Pulsar proxy
 ---
 
-The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) is an optional gateway that you can run over the brokers in a Pulsar cluster. We recommend running a Pulsar proxy in cases when direction connections between clients and Pulsar brokers are either infeasible, undesirable, or both, for example when running Pulsar in a cloud environment or on [Kubernetes](https://kubernetes.io) or an analogous platform.
+The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) is an optional gateway that you can run in front of the brokers in a Pulsar cluster. We recommend running a Pulsar proxy in cases when direction connections between clients and Pulsar brokers are either infeasible, undesirable, or both, for example when running Pulsar in a cloud environment or on [Kubernetes](https://kubernetes.io) or an analogous platform.
 
-## Running the proxy
+## Configuring the proxy
 
-In order to run the Pulsar proxy, you need to have both a local [ZooKeeper](https://zookeeper.apache.org) and configuration store quorum set up for use by your Pulsar cluster. For instructions, see [this document](deploy-bare-metal.md). Once you have ZooKeeper set up and have connection strings for both ZooKeeper quorums, you can use the [`proxy`](reference-cli-tools.md#pulsar-proxy) command of the [`pulsar`](reference-cli-tools.md#pulsar) CLI tool to start up the proxy (preferably on it [...]
+The proxy must have some way to find the addresses of the brokers of the cluster. You can do this by either configuring the proxy to connect directly to service discovery or by specifying a broker URL in the configuration. 
+
+### Option 1: Using service discovery
+
+Pulsar uses [ZooKeeper](https://zookeeper.apache.org) for service discovery. To connect the proxy to ZooKeeper, specify the following in `conf/proxy.conf`.
+```properties
+zookeeperServers=zk-0,zk-1,zk-2
+configurationStoreServers=zk-0:2184,zk-remote:2184
+```
+
+> If using service discovery, the network ACL must allow the proxy to talk to the ZooKeeper nodes on the zookeeper client port, which is usually 2181, and on the configuration store client port, which is 2184 by default. Opening the network ACLs means that if someone compromises a proxy, they have full access to ZooKeeper. For this reason, it is more secure to use broker URLs to configure the proxy.
+
+### Option 2: Using broker URLs
+
+The more secure method of configuring the proxy is to specify a URL to connect to the brokers.
+
+> [Authorization](security-authorization#enabling-authorization-and-assigning-superusers) at the proxy requires access to ZooKeeper, so if you are using this broker URLs to connect to the brokers, Proxy level authorization should be disabled. Brokers will still authorize requests after the proxy forwards them.
+
+You can configure the broker URLs in `conf/proxy.conf` as follows.
+
+```properties
+brokerServiceURL=pulsar://brokers.example.com:6650
+brokerWebServiceURL=http://brokers.example.com:8080
+functionWorkerWebServiceURL=http://function-workers.example.com:8080
+```
+
+Or if using TLS:
+```properties
+brokerServiceURLTLS=pulsar+ssl://brokers.example.com:6651
+brokerWebServiceURLTLS=https://brokers.example.com:8443
+functionWorkerWebServiceURL=https://function-workers.example.com:8443
+```
+
+The hostname in the URLs provided should be a DNS entry which points to multiple brokers or a Virtual IP which is backed by multiple broker IP addresses so that the proxy does not lose connectivity to the pulsar cluster if a single broker becomes unavailable.
+
+The ports to connect to the brokers (6650 & 8080, or in the case of TLS, 6651 & 8443) should be open in the network ACLs.
+
+Note that if you are not using functions, then `functionWorkerWebServiceURL` does not need to be configured.
+
+## Starting the proxy
 
 To start the proxy:
 
 ```bash
 $ cd /path/to/pulsar/directory
-$ bin/pulsar proxy \
-  --zookeeper-servers zk-0,zk-1,zk-2 \
-  --global-zookeeper-servers zk-0,zk-1,zk-2
+$ bin/pulsar proxy
 ```
 
 > You can run as many instances of the Pulsar proxy in a cluster as you would like.
diff --git a/site2/docs/security-authorization.md b/site2/docs/security-authorization.md
index 2cd168a..fe26cd7 100644
--- a/site2/docs/security-authorization.md
+++ b/site2/docs/security-authorization.md
@@ -16,7 +16,7 @@ When a [tenant](reference-terminology.md#tenant) is created by a superuser, that
 
 ### Enabling Authorization and Assigning Superusers
 
-Authorization is enabled and superusers are assigned in the broker ([`conf/broker.conf`](reference-configuration.md#broker)) and proxy ([`conf/proxy.conf`](reference-configuration.md#proxy)) configuration files.
+Authorization is enabled and superusers are assigned in the broker ([`conf/broker.conf`](reference-configuration.md#broker)) configuration files.
 
 ```properties
 authorizationEnabled=true
@@ -28,6 +28,8 @@ superUserRoles=my-super-user-1,my-super-user-2
 
 Typically, superuser roles are used for administrators and clients but also for broker-to-broker authorization. When using [geo-replication](concepts-replication.md), every broker needs to be able to publish to all the other clusters' topics.
 
+Authorization can also be enabled for the proxy the proxy configuration file (`conf/proxy.conf`). If it is enabled on the proxy, the proxy will do an additional authorization check before forwarding the request to a broker. The broker will still check the authorization of the request when it receives the forwarded request.
+
 ### Proxy Roles
 
 By default, the broker treats the connection between a proxy and the broker as a normal user connection. The user is authenticated as the role configured in ```proxy.conf``` (see ["Enabling TLS Authentication on Proxies"](security-tls-authentication#on-proxies)). However, this is rarely the behaviour that the user desires when connecting to the cluster through a proxy. The user expects to be able to interact with the cluster as the role for which they have authenticated with the proxy.
diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java
index aa2a5f6..bb181cc 100644
--- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java
+++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/proxy/TestProxy.java
@@ -30,25 +30,43 @@ import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.Schema;
 import org.apache.pulsar.common.policies.data.TenantInfo;
 import org.apache.pulsar.common.policies.data.TopicStats;
+import org.apache.pulsar.tests.integration.containers.ProxyContainer;
 import org.apache.pulsar.tests.integration.suites.PulsarTestSuite;
+import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec;
 import org.testng.annotations.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Test cases for proxy.
  */
 public class TestProxy extends PulsarTestSuite {
+    private final static Logger log = LoggerFactory.getLogger(TestProxy.class);
+    private ProxyContainer proxyViaURL;
 
-    @Test
-    public void testProxy() throws Exception {
+    @Override
+    protected PulsarClusterSpec.PulsarClusterSpecBuilder beforeSetupCluster(
+            String clusterName,
+            PulsarClusterSpec.PulsarClusterSpecBuilder specBuilder) {
+        proxyViaURL = new ProxyContainer(clusterName, "proxy-via-url")
+            .withEnv("brokerServiceURL", "pulsar://pulsar-broker-0:6650")
+            .withEnv("brokerWebServiceURL", "http://pulsar-broker-0:8080")
+            .withEnv("clusterName", clusterName);
 
-        final String tenant = "compaction-test-cli-" + randomName(4);
+        specBuilder.externalService("proxy-via-url", proxyViaURL);
+        return super.beforeSetupCluster(clusterName, specBuilder);
+    }
+
+    private void testProxy(String serviceUrl, String httpServiceUrl) throws Exception {
+        final String tenant = "proxy-test-" + randomName(10);
         final String namespace = tenant + "/ns1";
         final String topic = "persistent://" + namespace + "/topic1";
 
         @Cleanup
         PulsarAdmin admin = PulsarAdmin.builder()
-                .serviceHttpUrl(pulsarCluster.getHttpServiceUrl())
-                .build();
+            .serviceHttpUrl(httpServiceUrl)
+            .build();
 
         admin.tenants().createTenant(tenant,
                 new TenantInfo(Collections.emptySet(), Collections.singleton(pulsarCluster.getClusterName())));
@@ -57,8 +75,8 @@ public class TestProxy extends PulsarTestSuite {
 
         @Cleanup
         PulsarClient client = PulsarClient.builder()
-                .serviceUrl(pulsarCluster.getPlainTextServiceUrl())
-                .build();
+            .serviceUrl(serviceUrl)
+            .build();
 
         client.newConsumer()
                 .topic(topic)
@@ -80,4 +98,14 @@ public class TestProxy extends PulsarTestSuite {
         }
     }
 
+    @Test
+    public void testProxyWithServiceDiscovery() throws Exception {
+        testProxy(pulsarCluster.getPlainTextServiceUrl(), pulsarCluster.getHttpServiceUrl());
+    }
+
+    @Test
+    public void testProxyWithNoServiceDiscoveryProxyConnectsViaURL() throws Exception {
+        testProxy(proxyViaURL.getPlainTextServiceUrl(), proxyViaURL.getHttpServiceUrl());
+    }
+
 }
diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java
index 36110d2..43a40b8 100644
--- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java
+++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java
@@ -26,6 +26,7 @@ import lombok.Builder;
 import lombok.Builder.Default;
 import lombok.Getter;
 import lombok.Setter;
+import lombok.Singular;
 import lombok.experimental.Accessors;
 
 import org.apache.pulsar.tests.integration.containers.PulsarContainer;
@@ -101,6 +102,7 @@ public class PulsarClusterSpec {
      *
      * @return the list of external services to start with the cluster.
      */
+    @Singular
     Map<String, GenericContainer<?>> externalServices = Collections.EMPTY_MAP;
 
     /**