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 2023/02/22 21:23:41 UTC

[pulsar] branch branch-2.11 updated: [fix][broker] Allow proxy to pass same role for authRole and originalRole (#19557)

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

mmarshall pushed a commit to branch branch-2.11
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.11 by this push:
     new 5f5551df286 [fix][broker] Allow proxy to pass same role for authRole and originalRole (#19557)
5f5551df286 is described below

commit 5f5551df2862edc147303ff5c124edaef17773a8
Author: Michael Marshall <mm...@apache.org>
AuthorDate: Wed Feb 22 15:07:18 2023 -0600

    [fix][broker] Allow proxy to pass same role for authRole and originalRole (#19557)
    
    I broke the Pulsar Proxy with #19455 because that PR requires that when `X-Original-Principal` is supplied, the auth role must be a proxy role. This is not always the case for proxied admin requests. This PR seeks to fix that incorrect assumption by changing the way verification is done for the roles. Specifically, when the two roles are the same and they are not a proxy role, we will consider it a valid combination.
    
    Note that there is no inefficiency in this solution because When the `authenticatedPrincipal` is not a proxy role, that is the only role that is authenticated. Note also that we do not let the binary protocol authenticate this way, and that is consistent with the way the pulsar proxy forwards authentication data.
    
    Currently, we do the following when authentication is enabled in the proxy:
    
    1. Authenticate the client's http request and put the resulting role in the `X-Original-Principal` header for the call to the broker.
    https://github.com/apache/pulsar/blob/38555851359f9cfc172650c387a58c5a03809e97/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java#L370-L373
    
    2. Copy the `Authorization` header into the broker's http request:
    https://github.com/apache/pulsar/blob/38555851359f9cfc172650c387a58c5a03809e97/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java#L232-L236
    
    3. Configure the proxy's http client to use client TLS authentication (when configured):
    https://github.com/apache/pulsar/blob/38555851359f9cfc172650c387a58c5a03809e97/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java#L269-L277
    
    The problem with #19455 is that it assumes the proxy supplies its own authentication data. However, that only happens when using TLS authentication. Otherwise, the proxy forwards the client's authentication data in the `Authorization` header. As such, calls will fail because the `X-Original-Principal` header supplied without using a proxy role.
    
    * Consider the `authenticatedPrincipal` and the `originalPrincipal` a valid pair when they are equal and are not a `proxyRole` for http requests.
    
    I initially proposed that we only add the `X-Original-Principal` when we are using the proxy's authentication (see the first commit). I decided this solution is not ideal because it doesn't solve the problem, it doesn't make the brokers backwards compatible, and there isn't actually any inefficiency in passing the role as a header.
    
    When cherry-picking #19455 to branch-2.9, I discovered that `PackagesOpsWithAuthTest#testPackagesOps` was consistently failing because of the way the proxy supplies authentication data when proxying http requests. That test was removed by https://github.com/apache/pulsar/pull/12771, which explains why I didn't catch the error sooner. This PR includes a test that fails  without this change.
    
    Note that the primary issue must be that we didn't have any tests doing authentication forwarding through the proxy. Now we will have both relevant tests where the proxy is and is not authenticating.
    
    This is not a breaking change.
    
    - [x] `doc-required`
    
    PR in forked repository: https://github.com/michaeljmarshall/pulsar/pull/31
    
    (cherry picked from commit d4be954dedcc7537b3d65b9a1d7b5662e6062fdf)
---
 .../broker/authorization/AuthorizationService.java |  26 +++-
 .../apache/pulsar/broker/service/ServerCnx.java    |   2 +-
 .../pulsar/broker/auth/AuthorizationTest.java      |  30 +++--
 .../pulsar/broker/service/ServerCnxTest.java       |   2 +
 .../proxy/server/ProxyWithAuthorizationTest.java   | 141 ++++++++++++---------
 .../server/ProxyWithJwtAuthorizationTest.java      |  14 +-
 6 files changed, 129 insertions(+), 86 deletions(-)

diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
index 05f146e8953..701c9a81552 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
@@ -292,23 +292,36 @@ public class AuthorizationService {
         return provider.allowSinkOpsAsync(namespaceName, role, authenticationData);
     }
 
+    /**
+     * Whether the authenticatedPrincipal and the originalPrincipal form a valid pair. This method assumes that
+     * authenticatedPrincipal and originalPrincipal can be equal, as long as they are not a proxy role. This use
+     * case is relvant for the admin server because of the way the proxy handles authentication. The binary protocol
+     * should not use this method.
+     * @return true when roles are a valid combination and false when roles are an invalid combination
+     */
     public boolean isValidOriginalPrincipal(String authenticatedPrincipal,
                                             String originalPrincipal,
                                             AuthenticationDataSource authDataSource) {
         SocketAddress remoteAddress = authDataSource != null ? authDataSource.getPeerAddress() : null;
-        return isValidOriginalPrincipal(authenticatedPrincipal, originalPrincipal, remoteAddress);
+        return isValidOriginalPrincipal(authenticatedPrincipal, originalPrincipal, remoteAddress, true);
     }
 
     /**
      * Validates that the authenticatedPrincipal and the originalPrincipal are a valid combination.
-     * Valid combinations fulfill the following rule: the authenticatedPrincipal is in
-     * {@link ServiceConfiguration#getProxyRoles()}, if, and only if, the originalPrincipal is set to a role
-     * that is not also in {@link ServiceConfiguration#getProxyRoles()}.
+     * Valid combinations fulfill one of the following two rules:
+     * <p>
+     * 1. The authenticatedPrincipal is in {@link ServiceConfiguration#getProxyRoles()}, if, and only if,
+     * the originalPrincipal is set to a role that is not also in {@link ServiceConfiguration#getProxyRoles()}.
+     * <p>
+     * 2. The authenticatedPrincipal and the originalPrincipal are the same, but are not a proxyRole, when
+     * allowNonProxyPrincipalsToBeEqual is true.
+     *
      * @return true when roles are a valid combination and false when roles are an invalid combination
      */
     public boolean isValidOriginalPrincipal(String authenticatedPrincipal,
                                             String originalPrincipal,
-                                            SocketAddress remoteAddress) {
+                                            SocketAddress remoteAddress,
+                                            boolean allowNonProxyPrincipalsToBeEqual) {
         String errorMsg = null;
         if (conf.getProxyRoles().contains(authenticatedPrincipal)) {
             if (StringUtils.isBlank(originalPrincipal)) {
@@ -316,7 +329,8 @@ public class AuthorizationService {
             } else if (conf.getProxyRoles().contains(originalPrincipal)) {
                 errorMsg = "originalPrincipal cannot be a proxy role.";
             }
-        } else if (StringUtils.isNotBlank(originalPrincipal)) {
+        } else if (StringUtils.isNotBlank(originalPrincipal)
+                && !(allowNonProxyPrincipalsToBeEqual && originalPrincipal.equals(authenticatedPrincipal))) {
             errorMsg = "cannot specify originalPrincipal when connecting without valid proxy role.";
         }
         if (errorMsg != null) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index 7bf602f185a..b25d9514c1c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -700,7 +700,7 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
                 if (service.isAuthenticationEnabled()) {
                     if (service.isAuthorizationEnabled()) {
                         if (!service.getAuthorizationService()
-                                .isValidOriginalPrincipal(this.authRole, originalPrincipal, remoteAddress)) {
+                                .isValidOriginalPrincipal(this.authRole, originalPrincipal, remoteAddress, false)) {
                             state = State.Failed;
                             service.getPulsarStats().recordConnectionCreateFail();
                             final ByteBuf msg = Commands.newError(-1, ServerError.AuthorizationError,
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
index ad69180b236..bd6dfd872c5 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
@@ -242,27 +242,31 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
         AuthorizationService auth = new AuthorizationService(conf, Mockito.mock(PulsarResources.class));
 
         // Original principal should be supplied when authenticatedPrincipal is proxy role
-        assertTrue(auth.isValidOriginalPrincipal("proxy", "client", (SocketAddress) null));
+        assertTrue(auth.isValidOriginalPrincipal("proxy", "client", (SocketAddress) null, false));
 
         // Non proxy role should not supply originalPrincipal
-        assertTrue(auth.isValidOriginalPrincipal("client", "", (SocketAddress) null));
-        assertTrue(auth.isValidOriginalPrincipal("client", null, (SocketAddress) null));
+        assertTrue(auth.isValidOriginalPrincipal("client", "", (SocketAddress) null, false));
+        assertTrue(auth.isValidOriginalPrincipal("client", null, (SocketAddress) null, false));
+
+        // Edge cases that differ because binary protocol and http protocol have different expectations
+        assertTrue(auth.isValidOriginalPrincipal("client", "client", (SocketAddress) null, true));
+        assertFalse(auth.isValidOriginalPrincipal("client", "client", (SocketAddress) null, false));
 
         // Only likely in cases when authentication is disabled, but we still define these to be valid.
-        assertTrue(auth.isValidOriginalPrincipal(null, null, (SocketAddress) null));
-        assertTrue(auth.isValidOriginalPrincipal(null, "", (SocketAddress) null));
-        assertTrue(auth.isValidOriginalPrincipal("", null, (SocketAddress) null));
-        assertTrue(auth.isValidOriginalPrincipal("", "", (SocketAddress) null));
+        assertTrue(auth.isValidOriginalPrincipal(null, null, (SocketAddress) null, false));
+        assertTrue(auth.isValidOriginalPrincipal(null, "", (SocketAddress) null, false));
+        assertTrue(auth.isValidOriginalPrincipal("", null, (SocketAddress) null, false));
+        assertTrue(auth.isValidOriginalPrincipal("", "", (SocketAddress) null, false));
 
         // Proxy role must supply an original principal
-        assertFalse(auth.isValidOriginalPrincipal("proxy", "", (SocketAddress) null));
-        assertFalse(auth.isValidOriginalPrincipal("proxy", null, (SocketAddress) null));
+        assertFalse(auth.isValidOriginalPrincipal("proxy", "", (SocketAddress) null, false));
+        assertFalse(auth.isValidOriginalPrincipal("proxy", null, (SocketAddress) null, false));
 
         // OriginalPrincipal cannot be proxy role
-        assertFalse(auth.isValidOriginalPrincipal("proxy", "proxy", (SocketAddress) null));
-        assertFalse(auth.isValidOriginalPrincipal("client", "proxy", (SocketAddress) null));
-        assertFalse(auth.isValidOriginalPrincipal("", "proxy", (SocketAddress) null));
-        assertFalse(auth.isValidOriginalPrincipal(null, "proxy", (SocketAddress) null));
+        assertFalse(auth.isValidOriginalPrincipal("proxy", "proxy", (SocketAddress) null, false));
+        assertFalse(auth.isValidOriginalPrincipal("client", "proxy", (SocketAddress) null, false));
+        assertFalse(auth.isValidOriginalPrincipal("", "proxy", (SocketAddress) null, false));
+        assertFalse(auth.isValidOriginalPrincipal(null, "proxy", (SocketAddress) null, false));
 
         // Must gracefully handle a missing AuthenticationDataSource
         assertTrue(auth.isValidOriginalPrincipal("proxy", "client", (AuthenticationDataSource) null));
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
index 34264890a4b..cccc6eb59ed 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
@@ -657,6 +657,8 @@ public class ServerCnxTest {
         verifyAuthRoleAndOriginalPrincipalBehavior(authMethodName, "pass.client", "pass.proxy");
         // Invalid combinations where the original principal is set to a non-proxy role
         verifyAuthRoleAndOriginalPrincipalBehavior(authMethodName, "pass.client1", "pass.client");
+        verifyAuthRoleAndOriginalPrincipalBehavior(authMethodName, "pass.client", "pass.client");
+        verifyAuthRoleAndOriginalPrincipalBehavior(authMethodName, "pass.client", "pass.client1");
     }
 
     private void verifyAuthRoleAndOriginalPrincipalBehavior(String authMethodName, String authData,
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java
index deeb9dc4416..39628e0615d 100644
--- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java
@@ -84,6 +84,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
     private final String TLS_SUPERUSER_CLIENT_CERT_FILE_PATH = "./src/test/resources/authentication/tls/client-cert.pem";
 
     private ProxyService proxyService;
+    private WebServer webServer;
     private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
 
     @DataProvider(name = "hostnameVerification")
@@ -175,6 +176,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
         Set<String> superUserRoles = new HashSet<>();
         superUserRoles.add("superUser");
+        superUserRoles.add("Proxy");
         conf.setSuperUserRoles(superUserRoles);
 
         conf.setBrokerClientAuthenticationPlugin(AuthenticationTls.class.getName());
@@ -202,12 +204,11 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         proxyConfig.setForwardAuthorizationCredentials(true);
         proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl());
         proxyConfig.setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls());
+        proxyConfig.setBrokerWebServiceURLTLS(pulsar.getWebServiceAddressTls());
         proxyConfig.setAdvertisedAddress(null);
 
-        proxyConfig.setServicePort(Optional.of(0));
         proxyConfig.setBrokerProxyAllowedTargetPorts("*");
         proxyConfig.setServicePortTls(Optional.of(0));
-        proxyConfig.setWebServicePort(Optional.of(0));
         proxyConfig.setWebServicePortTls(Optional.of(0));
         proxyConfig.setTlsEnabledWithBroker(true);
 
@@ -225,9 +226,10 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
         proxyConfig.setProperties(properties);
 
-        proxyService = Mockito.spy(new ProxyService(proxyConfig,
-                                           new AuthenticationService(
-                                                   PulsarConfigurationLoader.convertFrom(proxyConfig))));
+        AuthenticationService authService =
+                new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig));
+        proxyService = Mockito.spy(new ProxyService(proxyConfig, authService));
+        webServer = new WebServer(proxyConfig, authService);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -235,10 +237,13 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
     protected void cleanup() throws Exception {
         super.internalCleanup();
         proxyService.close();
+        webServer.stop();
     }
 
     private void startProxy() throws Exception {
         proxyService.start();
+        ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null);
+        webServer.start();
     }
 
     /**
@@ -260,23 +265,15 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         startProxy();
-        createAdminClient();
+        // Skip hostname verification because the certs intentionally do not have a hostname
+        createProxyAdminClient(false);
         // create a client which connects to proxy over tls and pass authData
         @Cleanup
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(), PulsarClient.builder());
 
         String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrlTls(brokerUrlTls.toString()).build());
-
-        admin.tenants().createTenant("my-tenant",
-                new TenantInfoImpl(Sets.newHashSet("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));
+        initializeCluster(admin, namespaceName);
 
         Consumer<byte[]> consumer = proxyClient.newConsumer()
                 .topic("persistent://my-tenant/my-ns/my-topic1")
@@ -313,7 +310,8 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         startProxy();
-        createAdminClient();
+        // Testing client to proxy hostname verification, so use the dataProvider's value here
+        createProxyAdminClient(hostnameVerificationEnabled);
         // create a client which connects to proxy over tls and pass authData
         @Cleanup
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(),
@@ -321,16 +319,21 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
         String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrlTls.toString()).build());
-
-        admin.tenants().createTenant("my-tenant",
-                new TenantInfoImpl(Sets.newHashSet("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));
+        try {
+            initializeCluster(admin, namespaceName);
+            if (hostnameVerificationEnabled) {
+                Assert.fail("Connection should be failed due to hostnameVerification enabled");
+            }
+        } catch (PulsarAdminException e) {
+            if (!hostnameVerificationEnabled) {
+                Assert.fail("Cluster should initialize because hostnameverification is disabled");
+            }
+            admin.close();
+            // Need new client because the admin client to proxy is failing due to hostname verification, and we still
+            // want to test the binary protocol client fails to connect as well
+            createProxyAdminClient(false);
+            initializeCluster(admin, namespaceName);
+        }
 
         try {
             proxyClient.newConsumer().topic("persistent://my-tenant/my-ns/my-topic1")
@@ -365,7 +368,8 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
         proxyConfig.setTlsHostnameVerificationEnabled(hostnameVerificationEnabled);
         startProxy();
-        createAdminClient();
+        // This test skips hostname verification for client to proxy in order to test proxy to broker
+        createProxyAdminClient(false);
         // create a client which connects to proxy over tls and pass authData
         @Cleanup
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(),
@@ -373,16 +377,22 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
         String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrlTls(brokerUrlTls.toString()).build());
-
-        admin.tenants().createTenant("my-tenant",
-                new TenantInfoImpl(Sets.newHashSet("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));
+        try {
+            initializeCluster(admin, namespaceName);
+            if (hostnameVerificationEnabled) {
+                Assert.fail("Connection should be failed due to hostnameVerification enabled for proxy to broker");
+            }
+        } catch (PulsarAdminException.ServerSideErrorException e) {
+            if (!hostnameVerificationEnabled) {
+                Assert.fail("Cluster should initialize because hostnameverification is disabled for proxy to broker");
+            }
+            Assert.assertEquals(e.getStatusCode(), 502, "Should get bad gateway");
+            admin.close();
+            // Need to use broker's admin client because the proxy to broker is failing, and we still want to test
+            // the binary protocol client fails to connect as well
+            createBrokerAdminClient();
+            initializeCluster(admin, namespaceName);
+        }
 
         try {
             proxyClient.newConsumer().topic("persistent://my-tenant/my-ns/my-topic1")
@@ -410,18 +420,9 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
             throws Exception {
         log.info("-- Starting {} test --", methodName);
         String namespaceName = "my-tenant/my-ns";
-        createAdminClient();
+        createBrokerAdminClient();
 
-        admin.clusters().createCluster("proxy-authorization", ClusterData.builder().serviceUrl(brokerUrlTls.toString()).build());
-
-        admin.tenants().createTenant("my-tenant",
-                new TenantInfoImpl(Sets.newHashSet("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));
+        initializeCluster(admin, namespaceName);
 
         ProxyConfiguration proxyConfig = new ProxyConfiguration();
         proxyConfig.setAuthenticationEnabled(true);
@@ -508,7 +509,8 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         log.info("-- Starting {} test --", methodName);
 
         startProxy();
-        createAdminClient();
+        // Skip hostname verification because the certs intentionally do not have a hostname
+        createProxyAdminClient(false);
 
         @Cleanup
         PulsarClient proxyClient = PulsarClient.builder()
@@ -523,17 +525,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
         String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization",
-                ClusterData.builder().serviceUrlTls(brokerUrlTls.toString()).build());
-
-        admin.tenants().createTenant("my-tenant",
-                new TenantInfoImpl(Sets.newHashSet("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));
+        initializeCluster(admin, namespaceName);
 
         Consumer<byte[]> consumer = proxyClient.newConsumer()
                 .topic("persistent://my-tenant/my-ns/my-topic1")
@@ -565,7 +557,32 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         log.info("-- Exiting {} test --", methodName);
     }
 
-    private void createAdminClient() throws Exception {
+    private void initializeCluster(PulsarAdmin adminClient, String namespaceName) throws Exception {
+        adminClient.clusters().createCluster("proxy-authorization", ClusterData.builder()
+                .serviceUrlTls(brokerUrlTls.toString()).build());
+
+        adminClient.tenants().createTenant("my-tenant",
+                new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
+        adminClient.namespaces().createNamespace(namespaceName);
+
+        adminClient.namespaces().grantPermissionOnNamespace(namespaceName, "Proxy",
+                Sets.newHashSet(AuthAction.consume, AuthAction.produce));
+        adminClient.namespaces().grantPermissionOnNamespace(namespaceName, "Client",
+                Sets.newHashSet(AuthAction.consume, AuthAction.produce));
+    }
+
+    private void createProxyAdminClient(boolean enableTlsHostnameVerification) throws Exception {
+        Map<String, String> authParams = Maps.newHashMap();
+        authParams.put("tlsCertFile", TLS_SUPERUSER_CLIENT_CERT_FILE_PATH);
+        authParams.put("tlsKeyFile", TLS_SUPERUSER_CLIENT_KEY_FILE_PATH);
+
+        admin = spy(PulsarAdmin.builder().serviceHttpUrl("https://localhost:" + webServer.getListenPortHTTPS().get())
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH)
+                .enableTlsHostnameVerification(enableTlsHostnameVerification)
+                .authentication(AuthenticationTls.class.getName(), authParams).build());
+    }
+
+    private void createBrokerAdminClient() throws Exception {
         Map<String, String> authParams = Maps.newHashMap();
         authParams.put("tlsCertFile", TLS_SUPERUSER_CLIENT_CERT_FILE_PATH);
         authParams.put("tlsKeyFile", TLS_SUPERUSER_CLIENT_KEY_FILE_PATH);
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
index 9dc08d84848..2d8bfe63298 100644
--- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
@@ -71,6 +71,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase {
     private final String CLIENT_TOKEN = Jwts.builder().setSubject(CLIENT_ROLE).signWith(SECRET_KEY).compact();
 
     private ProxyService proxyService;
+    private WebServer webServer;
     private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
 
     @BeforeMethod
@@ -104,6 +105,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase {
         proxyConfig.setAuthorizationEnabled(false);
         proxyConfig.getProperties().setProperty("tokenSecretKey", "data:;base64," + Base64.getEncoder().encodeToString(SECRET_KEY.getEncoded()));
         proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl());
+        proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress());
 
         proxyConfig.setServicePort(Optional.of(0));
         proxyConfig.setBrokerProxyAllowedTargetPorts("*");
@@ -114,9 +116,10 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase {
         proxyConfig.setBrokerClientAuthenticationParameters(PROXY_TOKEN);
         proxyConfig.setAuthenticationProviders(providers);
 
-        proxyService = Mockito.spy(new ProxyService(proxyConfig,
-                new AuthenticationService(
-                        PulsarConfigurationLoader.convertFrom(proxyConfig))));
+        AuthenticationService authService =
+                new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig));
+        proxyService = Mockito.spy(new ProxyService(proxyConfig, authService));
+        webServer = new WebServer(proxyConfig, authService);
     }
 
     @AfterMethod(alwaysRun = true)
@@ -124,10 +127,13 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase {
     protected void cleanup() throws Exception {
         super.internalCleanup();
         proxyService.close();
+        webServer.stop();
     }
 
     private void startProxy() throws Exception {
         proxyService.start();
+        ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null);
+        webServer.start();
     }
 
     /**
@@ -430,7 +436,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase {
     }
 
     private void createAdminClient() throws Exception {
-        admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString())
+        admin = spy(PulsarAdmin.builder().serviceHttpUrl(webServer.getServiceUri().toString())
                 .authentication(AuthenticationFactory.token(ADMIN_TOKEN)).build());
     }