You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by lh...@apache.org on 2022/06/01 14:43:07 UTC

[pulsar] branch branch-2.7 updated (6a744def026 -> e5d035ae413)

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

lhotari pushed a change to branch branch-2.7
in repository https://gitbox.apache.org/repos/asf/pulsar.git


    from 6a744def026 [fix][python]Fix generated Python protobuf code not compatible with latest protobuf package (#15846)
     new 03fb6edde16 [Proxy] Log warning when opening connection to broker fails #14710
     new 6bafef1beb0 Fail proxy startup if brokerServiceURL is missing scheme (#14682)
     new 58ad46722c2 [refactor][proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode (#14713)
     new 837cfbd9a7b [Proxy & Client] Configure Netty DNS resolver to match JDK DNS caching setting, share DNS resolver instance in Proxy (#15219)
     new 8115076c511 [Proxy] Fix proxy connection leak when inbound connection closes while connecting is in progress (#15366)
     new 7c0eab26a6d [Proxy/Client] Fix DNS server denial-of-service issue when DNS entry expires (#15403)
     new 462d44f906f [Proxy] Remove unnecessary blocking DNS lookup in LookupProxyHandler (#15415)
     new dbe0518554b [Proxy] Prevent leak of unreleased lookupRequestSemaphore permits (#13812)
     new 0348502a206 Switch to rely on Netty for Hostname Verification (#15824)
     new e5d035ae413 Configure DLog Bookie, Pulsar, and Admin clients via pass through config (#15818)

The 10 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../resources/authentication/tls/broker-cert.pem   |  74 +++---
 .../test/resources/authentication/tls/cacert.pem   | 109 ++++----
 .../resources/authentication/tls/client-cert.pem   |  87 ++++---
 build/regenerate_certs_for_tests.sh                |  70 +++++
 conf/broker.conf                                   |  14 +-
 conf/functions_worker.yml                          |  20 ++
 conf/proxy.conf                                    |   7 +-
 conf/websocket.conf                                |   4 +
 .../pulsar/broker/BookKeeperClientFactoryImpl.java |  16 +-
 .../org/apache/pulsar/broker/PulsarService.java    |  32 ++-
 .../pulsar/broker/namespace/NamespaceService.java  |  10 +-
 .../pulsar/broker/service/BrokerService.java       |  21 +-
 .../apache/pulsar/compaction/CompactorTool.java    |   6 +
 ...kerInternalClientConfigurationOverrideTest.java | 103 ++++++++
 .../AuthenticationTlsHostnameVerificationTest.java |  58 ++---
 .../pulsar/client/impl/ConnectionPoolTest.java     |  38 +--
 .../PulsarClientConfigurationOverrideTest.java     |  56 ++++
 .../websocket/proxy/ProxyConfigurationTest.java    |   8 +-
 .../pulsar/client/admin/PulsarAdminBuilder.java    |  23 ++
 .../admin/internal/PulsarAdminBuilderImpl.java     |   9 +-
 .../admin/internal/http/AsyncHttpConnector.java    |   9 +-
 .../pulsar/client/internal/PropertiesUtils.java    |  64 +++++
 .../org/apache/pulsar/client/impl/ClientCnx.java   |  48 ----
 .../apache/pulsar/client/impl/ConnectionPool.java  |  66 +++--
 .../org/apache/pulsar/client/impl/HttpClient.java  |   1 +
 .../client/impl/PulsarChannelInitializer.java      |   7 +
 pulsar-common/pom.xml                              |   5 +
 .../util/NettyClientSslContextRefresher.java       |   3 +-
 .../apache/pulsar/common/util/SecurityUtility.java |  10 +
 .../pulsar/common/util/netty/DnsResolverUtil.java  |  75 ++++++
 .../src/test/resources/test_worker_config.yml      |   3 +
 .../pulsar/functions/worker/WorkerService.java     |   7 +-
 .../pulsar/functions/worker/WorkerUtils.java       |  45 +++-
 .../pulsar/functions/worker/WorkerUtilsTest.java   |  27 +-
 .../pulsar/proxy/server/AdminProxyHandler.java     |   7 +-
 .../pulsar/proxy/server/BrokerProxyValidator.java  |   2 +-
 .../pulsar/proxy/server/DirectProxyHandler.java    | 284 +++++++++++----------
 .../pulsar/proxy/server/LookupProxyHandler.java    | 123 +++++----
 .../pulsar/proxy/server/ParserProxyHandler.java    | 114 +++++----
 .../pulsar/proxy/server/ProxyConfiguration.java    |   4 +-
 .../pulsar/proxy/server/ProxyConnection.java       | 219 ++++++++++------
 .../pulsar/proxy/server/ProxyConnectionPool.java   |  60 -----
 .../apache/pulsar/proxy/server/ProxyService.java   |  15 +-
 .../pulsar/proxy/server/ProxyServiceStarter.java   |  10 +
 .../proxy/server/ServiceChannelInitializer.java    |  58 +----
 .../proxy/server/BrokerProxyValidatorTest.java     |  20 ++
 .../proxy/server/ProxyWithAuthorizationTest.java   |  70 ++---
 .../ProxyWithAuthorizationTest/broker-cacert.pem   | 109 ++++----
 .../tls/ProxyWithAuthorizationTest/broker-cert.pem |  86 +++----
 .../ProxyWithAuthorizationTest/client-cacert.pem   | 109 ++++----
 .../tls/ProxyWithAuthorizationTest/client-cert.pem |  86 +++----
 .../ProxyWithAuthorizationTest/proxy-cacert.pem    | 109 ++++----
 .../tls/ProxyWithAuthorizationTest/proxy-cert.pem  |  86 +++----
 .../test/resources/authentication/tls/cacert.pem   | 109 ++++----
 .../resources/authentication/tls/client-cert.pem   |  87 ++++---
 .../resources/authentication/tls/server-cert.pem   |  87 ++++---
 .../apache/pulsar/websocket/WebSocketService.java  |   7 +-
 site2/docs/reference-configuration.md              |  26 +-
 58 files changed, 1820 insertions(+), 1202 deletions(-)
 create mode 100755 build/regenerate_certs_for_tests.sh
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarClientConfigurationOverrideTest.java
 create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PropertiesUtils.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java
 copy pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PersistencePoliciesTest.java => pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/WorkerUtilsTest.java (53%)
 delete mode 100644 pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnectionPool.java


[pulsar] 05/10: [Proxy] Fix proxy connection leak when inbound connection closes while connecting is in progress (#15366)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8115076c5111f7bcddb010639d787c38cf0b8f82
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Thu Apr 28 19:43:13 2022 +0300

    [Proxy] Fix proxy connection leak when inbound connection closes while connecting is in progress (#15366)
    
    - backports https://github.com/apache/pulsar/pull/15366 to branch-2.7
    
    (cherry picked from commit 4621ca63fcaabf3a0faefd487434dbd97c1d8859)
---
 .../pulsar/proxy/server/DirectProxyHandler.java    |  19 ++--
 .../pulsar/proxy/server/ProxyConnection.java       | 103 ++++++++++++++++-----
 2 files changed, 92 insertions(+), 30 deletions(-)

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 8fa7787215d..64ce8c68b27 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
@@ -80,9 +80,7 @@ public class DirectProxyHandler {
     private final ProxyService service;
     private final Runnable onHandshakeCompleteAction;
 
-    public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection, String brokerHostAndPort,
-                              InetSocketAddress targetBrokerAddress, int protocolVersion,
-                              Supplier<SslHandler> sslHandlerSupplier) {
+    public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) {
         this.service = service;
         this.authentication = proxyConnection.getClientAuthentication();
         this.inboundChannel = proxyConnection.ctx().channel();
@@ -92,6 +90,10 @@ public class DirectProxyHandler {
         this.clientAuthData = proxyConnection.clientAuthData;
         this.clientAuthMethod = proxyConnection.clientAuthMethod;
         this.onHandshakeCompleteAction = proxyConnection::cancelKeepAliveTask;
+    }
+
+    public void connect(String brokerHostAndPort, InetSocketAddress targetBrokerAddress,
+                           int protocolVersion, Supplier<SslHandler> sslHandlerSupplier) {
         ProxyConfiguration config = service.getConfiguration();
 
         // Start the connection attempt.
@@ -208,6 +210,12 @@ public class DirectProxyHandler {
             (byte) 'Y',
     };
 
+    public void close() {
+        if (outboundChannel != null) {
+            outboundChannel.close();
+        }
+    }
+
     enum BackendState {
         Init, HandshakeCompleted
     }
@@ -344,10 +352,7 @@ public class DirectProxyHandler {
             onHandshakeCompleteAction.run();
             startDirectProxying(connected);
 
-            int maxMessageSize =
-                    connected.hasMaxMessageSize() ? connected.getMaxMessageSize() : Commands.INVALID_MAX_MESSAGE_SIZE;
-            inboundChannel.writeAndFlush(Commands.newConnected(connected.getProtocolVersion(), maxMessageSize))
-                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+            proxyConnection.brokerConnected(DirectProxyHandler.this, connected);
         }
 
         private void startDirectProxying(CommandConnected connected) {
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index 395d16bbbd8..fbf604866b8 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -19,14 +19,16 @@
 package org.apache.pulsar.proxy.server;
 
 import static com.google.common.base.Preconditions.checkArgument;
-
+import static com.google.common.base.Preconditions.checkState;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.handler.codec.haproxy.HAProxyMessage;
 import io.netty.resolver.dns.DnsNameResolver;
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -51,6 +53,7 @@ import org.apache.pulsar.common.protocol.PulsarHandler;
 import org.apache.pulsar.common.api.proto.PulsarApi;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect;
+import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema;
@@ -113,6 +116,9 @@ public class ProxyConnection extends PulsarHandler {
         // Follow redirects
         ProxyLookupRequests,
 
+        // Connecting to the broker
+        ProxyConnectingToBroker,
+
         // If we are proxying a connection to a specific broker, we
         // are just forwarding data between the 2 connections, without
         // looking into it
@@ -166,8 +172,8 @@ public class ProxyConnection extends PulsarHandler {
     public synchronized void channelInactive(ChannelHandlerContext ctx) throws Exception {
         super.channelInactive(ctx);
 
-        if (directProxyHandler != null && directProxyHandler.outboundChannel != null) {
-            directProxyHandler.outboundChannel.close();
+        if (directProxyHandler != null) {
+            directProxyHandler.close();
             directProxyHandler = null;
         }
 
@@ -188,11 +194,22 @@ public class ProxyConnection extends PulsarHandler {
 
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-        state = State.Closing;
         super.exceptionCaught(ctx, cause);
-        LOG.warn("[{}] Got exception {} : {} {}", remoteAddress, cause.getClass().getSimpleName(), cause.getMessage(),
+        LOG.warn("[{}] Got exception {} : Message: {} State: {}", remoteAddress, cause.getClass().getSimpleName(),
+                cause.getMessage(), state,
                 ClientCnx.isKnownException(cause) ? null : cause);
-        ctx.close();
+        if (state != State.Closed) {
+            state = State.Closing;
+        }
+        if (ctx.channel().isOpen()) {
+            ctx.close();
+        } else {
+            // close connection to broker if that is present
+            if (directProxyHandler != null) {
+                directProxyHandler.close();
+                directProxyHandler = null;
+            }
+        }
     }
 
     @Override
@@ -221,18 +238,26 @@ public class ProxyConnection extends PulsarHandler {
             break;
 
         case ProxyConnectionToBroker:
-            // Pass the buffer to the outbound connection and schedule next read
-            // only if we can write on the connection
-            ProxyService.opsCounter.inc();
-            if (msg instanceof ByteBuf) {
-                int bytes = ((ByteBuf) msg).readableBytes();
-                directProxyHandler.getInboundChannelRequestsRate().recordEvent(bytes);
-                ProxyService.bytesCounter.inc(bytes);
+            if (directProxyHandler != null) {
+                ProxyService.opsCounter.inc();
+                if (msg instanceof ByteBuf) {
+                    int bytes = ((ByteBuf) msg).readableBytes();
+                    directProxyHandler.getInboundChannelRequestsRate().recordEvent(bytes);
+                    ProxyService.bytesCounter.inc(bytes);
+                }
+                directProxyHandler.outboundChannel.writeAndFlush(msg)
+                        .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+            } else {
+                LOG.warn("Received message of type {} while connection to broker is missing in state {}. "
+                                + "Dropping the input message (readable bytes={}).", msg.getClass(), state,
+                        msg instanceof ByteBuf ? ((ByteBuf) msg).readableBytes() : -1);
             }
-            directProxyHandler.outboundChannel.writeAndFlush(msg)
-                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
             break;
-
+        case ProxyConnectingToBroker:
+            LOG.warn("Received message of type {} while connecting to broker. "
+                            + "Dropping the input message (readable bytes={}).", msg.getClass(),
+                    msg instanceof ByteBuf ? ((ByteBuf) msg).readableBytes() : -1);
+            break;
         default:
             break;
         }
@@ -278,14 +303,9 @@ public class ProxyConnection extends PulsarHandler {
                 return;
             }
 
+            state = State.ProxyConnectingToBroker;
             brokerProxyValidator.resolveAndCheckTargetAddress(proxyToBrokerUrl)
-                    .thenAcceptAsync(address -> {
-                        // Client already knows which broker to connect. Let's open a
-                        // connection there and just pass bytes in both directions
-                        state = State.ProxyConnectionToBroker;
-                        directProxyHandler = new DirectProxyHandler(service, this, proxyToBrokerUrl, address,
-                                protocolVersionToAdvertise, sslHandlerSupplier);
-                    }, ctx.executor())
+                    .thenAcceptAsync(this::connectToBroker, ctx.executor())
                     .exceptionally(throwable -> {
                         if (throwable instanceof TargetAddressDeniedException
                                 || throwable.getCause() instanceof TargetAddressDeniedException) {
@@ -318,6 +338,43 @@ public class ProxyConnection extends PulsarHandler {
         }
     }
 
+    private void handleBrokerConnected(DirectProxyHandler directProxyHandler, CommandConnected connected) {
+        checkState(ctx.executor().inEventLoop(), "This method should be called in the event loop");
+        if (state == State.ProxyConnectingToBroker && ctx.channel().isOpen() && this.directProxyHandler == null) {
+            this.directProxyHandler = directProxyHandler;
+            state = State.ProxyConnectionToBroker;
+            int maxMessageSize =
+                    connected.hasMaxMessageSize() ? connected.getMaxMessageSize() : Commands.INVALID_MAX_MESSAGE_SIZE;
+            ctx.writeAndFlush(Commands.newConnected(connected.getProtocolVersion(), maxMessageSize))
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+        } else {
+            LOG.warn("[{}] Channel is {}. ProxyConnection is in {}. "
+                            + "Closing connection to broker '{}'.",
+                    remoteAddress, ctx.channel().isOpen() ? "open" : "already closed",
+                    state != State.ProxyConnectingToBroker ? "invalid state " + state : "state " + state,
+                    proxyToBrokerUrl);
+            directProxyHandler.close();
+            ctx.close();
+        }
+    }
+
+    private void connectToBroker(InetSocketAddress brokerAddress) {
+        checkState(ctx.executor().inEventLoop(), "This method should be called in the event loop");
+        DirectProxyHandler directProxyHandler = new DirectProxyHandler(service, this);
+        directProxyHandler.connect(proxyToBrokerUrl, brokerAddress,
+                protocolVersionToAdvertise, sslHandlerSupplier);
+    }
+
+    public void brokerConnected(DirectProxyHandler directProxyHandler, CommandConnected connected) {
+        try {
+            final CommandConnected finalConnected = connected.toBuilder().build();
+            ctx.executor().submit(() -> handleBrokerConnected(directProxyHandler, finalConnected));
+        } catch (RejectedExecutionException e) {
+            LOG.error("Event loop was already closed. Closing broker connection.", e);
+            directProxyHandler.close();
+        }
+    }
+
     // According to auth result, send newConnected or newAuthChallenge command.
     private void doAuthentication(AuthData clientData) throws Exception {
         AuthData brokerData = authState.authenticate(clientData);


[pulsar] 06/10: [Proxy/Client] Fix DNS server denial-of-service issue when DNS entry expires (#15403)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 7c0eab26a6dbf573f2dc41cd6454af07e0c9e4a9
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Sat Apr 30 23:04:30 2022 +0300

    [Proxy/Client] Fix DNS server denial-of-service issue when DNS entry expires (#15403)
    
    - DnsNameResolver doesn't coordinate concurrency and this leads to DNS server DoS
      under high load
    - In Netty, DnsAddressResolverGroup internally uses internal InflightNameResolver
      class to address the problem
      - The solution is to use DnsAddressResolverGroup instead of instantiating DnsNameResolver
        directly
    
    (cherry picked from commit fe789088c181e8545bd0af9de12d497fa179dc05)
---
 .../pulsar/client/impl/ConnectionPoolTest.java     | 38 +++++++--------
 .../apache/pulsar/client/impl/ConnectionPool.java  | 54 +++++++++++-----------
 .../pulsar/proxy/server/ProxyConnection.java       | 11 +++--
 .../apache/pulsar/proxy/server/ProxyService.java   | 12 ++---
 .../proxy/server/ServiceChannelInitializer.java    |  2 +-
 5 files changed, 60 insertions(+), 57 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java
index a375eb852cb..9ba703ce494 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java
@@ -21,6 +21,10 @@ package org.apache.pulsar.client.impl;
 import com.google.common.collect.Lists;
 import io.netty.channel.EventLoopGroup;
 import io.netty.util.concurrent.DefaultThreadFactory;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.IntStream;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.util.netty.EventLoopUtil;
@@ -30,21 +34,17 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.IntStream;
-
 public class ConnectionPoolTest extends MockedPulsarServiceBaseTest {
 
     String serviceUrl;
+    int brokerPort;
 
     @BeforeClass
     @Override
     protected void setup() throws Exception {
         super.internalSetup();
-        serviceUrl = "pulsar://non-existing-dns-name:" + pulsar.getBrokerListenPort().get();
+        brokerPort = pulsar.getBrokerListenPort().get();
+        serviceUrl = "pulsar://non-existing-dns-name:" + brokerPort;
     }
 
     @AfterClass(alwaysRun = true)
@@ -61,9 +61,11 @@ public class ConnectionPoolTest extends MockedPulsarServiceBaseTest {
         conf.setServiceUrl(serviceUrl);
         PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool);
 
-        List<InetAddress> result = Lists.newArrayList();
-        result.add(InetAddress.getByName("127.0.0.1"));
-        Mockito.when(pool.resolveName("non-existing-dns-name")).thenReturn(CompletableFuture.completedFuture(result));
+        List<InetSocketAddress> result = Lists.newArrayList();
+        result.add(new InetSocketAddress("127.0.0.1", brokerPort));
+        Mockito.when(pool.resolveName(InetSocketAddress.createUnresolved("non-existing-dns-name",
+                brokerPort)))
+                .thenReturn(CompletableFuture.completedFuture(result));
 
         client.newProducer().topic("persistent://sample/standalone/ns/my-topic").create();
 
@@ -73,20 +75,20 @@ public class ConnectionPoolTest extends MockedPulsarServiceBaseTest {
 
     @Test
     public void testDoubleIpAddress() throws Exception {
-        String serviceUrl = "pulsar://non-existing-dns-name:" + pulsar.getBrokerListenPort().get();
-
         ClientConfigurationData conf = new ClientConfigurationData();
         EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, new DefaultThreadFactory("test"));
         ConnectionPool pool = Mockito.spy(new ConnectionPool(conf, eventLoop));
         conf.setServiceUrl(serviceUrl);
         PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool);
 
-        List<InetAddress> result = Lists.newArrayList();
+        List<InetSocketAddress> result = Lists.newArrayList();
 
         // Add a non existent IP to the response to check that we're trying the 2nd address as well
-        result.add(InetAddress.getByName("127.0.0.99"));
-        result.add(InetAddress.getByName("127.0.0.1"));
-        Mockito.when(pool.resolveName("non-existing-dns-name")).thenReturn(CompletableFuture.completedFuture(result));
+        result.add(new InetSocketAddress("127.0.0.99", brokerPort));
+        result.add(new InetSocketAddress("127.0.0.1", brokerPort));
+        Mockito.when(pool.resolveName(InetSocketAddress.createUnresolved("non-existing-dns-name",
+                        brokerPort)))
+                .thenReturn(CompletableFuture.completedFuture(result));
 
         // Create producer should succeed by trying the 2nd IP
         client.newProducer().topic("persistent://sample/standalone/ns/my-topic").create();
@@ -103,7 +105,7 @@ public class ConnectionPoolTest extends MockedPulsarServiceBaseTest {
         ConnectionPool pool = Mockito.spy(new ConnectionPool(conf, eventLoop));
 
         InetSocketAddress brokerAddress =
-            InetSocketAddress.createUnresolved("127.0.0.1", pulsar.getBrokerListenPort().get());
+            InetSocketAddress.createUnresolved("127.0.0.1", brokerPort);
         IntStream.range(1, 5).forEach(i -> {
             pool.getConnection(brokerAddress).thenAccept(cnx -> {
                 Assert.assertTrue(cnx.channel().isActive());
@@ -125,7 +127,7 @@ public class ConnectionPoolTest extends MockedPulsarServiceBaseTest {
         ConnectionPool pool = Mockito.spy(new ConnectionPool(conf, eventLoop));
 
         InetSocketAddress brokerAddress =
-            InetSocketAddress.createUnresolved("127.0.0.1", pulsar.getBrokerListenPort().get());
+            InetSocketAddress.createUnresolved("127.0.0.1", brokerPort);
         IntStream.range(1, 10).forEach(i -> {
             pool.getConnection(brokerAddress).thenAccept(cnx -> {
                 Assert.assertTrue(cnx.channel().isActive());
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
index 99492784e5f..8d09ee6776b 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
@@ -25,10 +25,10 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelException;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
-import io.netty.resolver.dns.DnsNameResolver;
+import io.netty.resolver.AddressResolver;
+import io.netty.resolver.dns.DnsAddressResolverGroup;
 import io.netty.resolver.dns.DnsNameResolverBuilder;
 import io.netty.util.concurrent.Future;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -62,7 +62,7 @@ public class ConnectionPool implements AutoCloseable {
     private final int maxConnectionsPerHosts;
     private final boolean isSniProxy;
 
-    protected final DnsNameResolver dnsResolver;
+    protected final AddressResolver<InetSocketAddress> addressResolver;
     private final boolean shouldCloseDnsResolver;
 
     public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException {
@@ -75,7 +75,8 @@ public class ConnectionPool implements AutoCloseable {
     }
 
     public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup,
-                             Supplier<ClientCnx> clientCnxSupplier, Optional<DnsNameResolver> dnsNameResolver)
+                          Supplier<ClientCnx> clientCnxSupplier,
+                          Optional<AddressResolver<InetSocketAddress>> addressResolver)
             throws PulsarClientException {
         this.eventLoopGroup = eventLoopGroup;
         this.clientConfig = conf;
@@ -100,15 +101,19 @@ public class ConnectionPool implements AutoCloseable {
             throw new PulsarClientException(e);
         }
 
-        this.shouldCloseDnsResolver = !dnsNameResolver.isPresent();
-        this.dnsResolver = dnsNameResolver.orElseGet(() -> createDnsNameResolver(conf, eventLoopGroup));
+        this.shouldCloseDnsResolver = !addressResolver.isPresent();
+        this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
     }
 
-    private static DnsNameResolver createDnsNameResolver(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) {
-        DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder(eventLoopGroup.next())
+    private static AddressResolver<InetSocketAddress> createAddressResolver(ClientConfigurationData conf,
+                                                                            EventLoopGroup eventLoopGroup) {
+        DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder()
                 .traceEnabled(true).channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup));
         DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
-        return dnsNameResolverBuilder.build();
+        // use DnsAddressResolverGroup to create the AddressResolver since it contains a solution
+        // to prevent cache stampede / thundering herds problem when a DNS entry expires while the system
+        // is under high load
+        return new DnsAddressResolverGroup(dnsNameResolverBuilder).getResolver(eventLoopGroup.next());
     }
 
     private static final Random random = new Random();
@@ -235,19 +240,17 @@ public class ConnectionPool implements AutoCloseable {
      * Resolve DNS asynchronously and attempt to connect to any IP address returned by DNS server.
      */
     private CompletableFuture<Channel> createConnection(InetSocketAddress unresolvedAddress) {
-        int port;
-        CompletableFuture<List<InetAddress>> resolvedAddress = null;
+        CompletableFuture<List<InetSocketAddress>> resolvedAddress;
         try {
             if (isSniProxy) {
                 URI proxyURI = new URI(clientConfig.getProxyServiceUrl());
-                port = proxyURI.getPort();
-                resolvedAddress = resolveName(proxyURI.getHost());
+                resolvedAddress =
+                        resolveName(InetSocketAddress.createUnresolved(proxyURI.getHost(), proxyURI.getPort()));
             } else {
-                port = unresolvedAddress.getPort();
-                resolvedAddress = resolveName(unresolvedAddress.getHostString());
+                resolvedAddress = resolveName(unresolvedAddress);
             }
             return resolvedAddress.thenCompose(
-                    inetAddresses -> connectToResolvedAddresses(inetAddresses.iterator(), port,
+                    inetAddresses -> connectToResolvedAddresses(inetAddresses.iterator(),
                             isSniProxy ? unresolvedAddress : null));
         } catch (URISyntaxException e) {
             log.error("Invalid Proxy url {}", clientConfig.getProxyServiceUrl(), e);
@@ -260,18 +263,17 @@ public class ConnectionPool implements AutoCloseable {
      * Try to connect to a sequence of IP addresses until a successful connection can be made, or fail if no
      * address is working.
      */
-    private CompletableFuture<Channel> connectToResolvedAddresses(Iterator<InetAddress> unresolvedAddresses,
-                                                                  int port,
+    private CompletableFuture<Channel> connectToResolvedAddresses(Iterator<InetSocketAddress> unresolvedAddresses,
                                                                   InetSocketAddress sniHost) {
         CompletableFuture<Channel> future = new CompletableFuture<>();
 
         // Successfully connected to server
-        connectToAddress(unresolvedAddresses.next(), port, sniHost)
+        connectToAddress(unresolvedAddresses.next(), sniHost)
                 .thenAccept(future::complete)
                 .exceptionally(exception -> {
                     if (unresolvedAddresses.hasNext()) {
                         // Try next IP address
-                        connectToResolvedAddresses(unresolvedAddresses, port, sniHost).thenAccept(future::complete)
+                        connectToResolvedAddresses(unresolvedAddresses, sniHost).thenAccept(future::complete)
                                 .exceptionally(ex -> {
                                     // This is already unwinding the recursive call
                                     future.completeExceptionally(ex);
@@ -287,10 +289,9 @@ public class ConnectionPool implements AutoCloseable {
         return future;
     }
 
-    @VisibleForTesting
-    CompletableFuture<List<InetAddress>> resolveName(String hostname) {
-        CompletableFuture<List<InetAddress>> future = new CompletableFuture<>();
-        dnsResolver.resolveAll(hostname).addListener((Future<List<InetAddress>> resolveFuture) -> {
+    CompletableFuture<List<InetSocketAddress>> resolveName(InetSocketAddress unresolvedAddress) {
+        CompletableFuture<List<InetSocketAddress>> future = new CompletableFuture<>();
+        addressResolver.resolveAll(unresolvedAddress).addListener((Future<List<InetSocketAddress>> resolveFuture) -> {
             if (resolveFuture.isSuccess()) {
                 future.complete(resolveFuture.get());
             } else {
@@ -303,8 +304,7 @@ public class ConnectionPool implements AutoCloseable {
     /**
      * Attempt to establish a TCP connection to an already resolved single IP address.
      */
-    private CompletableFuture<Channel> connectToAddress(InetAddress ipAddress, int port, InetSocketAddress sniHost) {
-        InetSocketAddress remoteAddress = new InetSocketAddress(ipAddress, port);
+    private CompletableFuture<Channel> connectToAddress(InetSocketAddress remoteAddress, InetSocketAddress sniHost) {
         if (clientConfig.isUseTls()) {
             return toCompletableFuture(bootstrap.register())
                     .thenCompose(channel -> channelInitializerHandler
@@ -331,7 +331,7 @@ public class ConnectionPool implements AutoCloseable {
     public void close() throws Exception {
         closeAllConnections();
         if (shouldCloseDnsResolver) {
-            dnsResolver.close();
+            addressResolver.close();
         }
     }
 
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index fbf604866b8..b7b6e359a11 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -22,7 +22,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.handler.codec.haproxy.HAProxyMessage;
-import io.netty.resolver.dns.DnsNameResolver;
+import io.netty.resolver.dns.DnsAddressResolverGroup;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Collections;
@@ -82,7 +82,7 @@ public class ProxyConnection extends PulsarHandler {
     private final AtomicLong requestIdGenerator =
             new AtomicLong(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE / 2));
     private final ProxyService service;
-    private final DnsNameResolver dnsNameResolver;
+    private final DnsAddressResolverGroup dnsAddressResolverGroup;
     private Authentication clientAuthentication;
     AuthenticationDataSource authenticationData;
     private State state;
@@ -134,10 +134,10 @@ public class ProxyConnection extends PulsarHandler {
     }
 
     public ProxyConnection(ProxyService proxyService, Supplier<SslHandler> sslHandlerSupplier,
-                           DnsNameResolver dnsNameResolver) {
+                           DnsAddressResolverGroup dnsAddressResolverGroup) {
         super(30, TimeUnit.SECONDS);
         this.service = proxyService;
-        this.dnsNameResolver = dnsNameResolver;
+        this.dnsAddressResolverGroup = dnsAddressResolverGroup;
         this.state = State.Init;
         this.sslHandlerSupplier = sslHandlerSupplier;
         this.brokerProxyValidator = service.getBrokerProxyValidator();
@@ -280,7 +280,8 @@ public class ProxyConnection extends PulsarHandler {
 
         if (this.connectionPool == null) {
             this.connectionPool = new ConnectionPool(clientConf, service.getWorkerGroup(),
-                    clientCnxSupplier, Optional.of(dnsNameResolver));
+                    clientCnxSupplier,
+                    Optional.of(dnsAddressResolverGroup.getResolver(service.getWorkerGroup().next())));
         } else {
             LOG.error("BUG! Connection Pool has already been created for proxy connection to {} state {} role {}",
                     remoteAddress, state, clientAuthRole);
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
index f24caf943b0..d1239424601 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
@@ -26,7 +26,7 @@ import io.netty.channel.AdaptiveRecvByteBufAllocator;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
-import io.netty.resolver.dns.DnsNameResolver;
+import io.netty.resolver.dns.DnsAddressResolverGroup;
 import io.netty.resolver.dns.DnsNameResolverBuilder;
 import io.netty.util.concurrent.DefaultThreadFactory;
 import io.prometheus.client.Counter;
@@ -71,7 +71,7 @@ public class ProxyService implements Closeable {
 
     private final ProxyConfiguration proxyConfig;
     @Getter
-    private final DnsNameResolver dnsNameResolver;
+    private final DnsAddressResolverGroup dnsAddressResolverGroup;
     @Getter
     private final BrokerProxyValidator brokerProxyValidator;
     private String serviceUrl;
@@ -146,13 +146,13 @@ public class ProxyService implements Closeable {
         this.workerGroup = EventLoopUtil.newEventLoopGroup(numThreads, workersThreadFactory);
         this.authenticationService = authenticationService;
 
-        DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder(workerGroup.next())
+        DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder()
                 .channelType(EventLoopUtil.getDatagramChannelClass(workerGroup));
         DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
 
-        dnsNameResolver = dnsNameResolverBuilder.build();
+        dnsAddressResolverGroup = new DnsAddressResolverGroup(dnsNameResolverBuilder);
 
-        brokerProxyValidator = new BrokerProxyValidator(dnsNameResolver.asAddressResolver(),
+        brokerProxyValidator = new BrokerProxyValidator(dnsAddressResolverGroup.getResolver(workerGroup.next()),
                 proxyConfig.getBrokerProxyAllowedHostNames(),
                 proxyConfig.getBrokerProxyAllowedIPAddresses(),
                 proxyConfig.getBrokerProxyAllowedTargetPorts());
@@ -238,7 +238,7 @@ public class ProxyService implements Closeable {
     }
 
     public void close() throws IOException {
-        dnsNameResolver.close();
+        dnsAddressResolverGroup.close();
 
         if (discoveryProvider != null) {
             discoveryProvider.close();
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
index 4aae1196fb6..a8f2287ffea 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
@@ -159,7 +159,7 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
         }
 
         ch.pipeline().addLast("handler",
-                new ProxyConnection(proxyService, sslHandlerSupplier, proxyService.getDnsNameResolver()));
+                new ProxyConnection(proxyService, sslHandlerSupplier, proxyService.getDnsAddressResolverGroup()));
 
     }
 }


[pulsar] 04/10: [Proxy & Client] Configure Netty DNS resolver to match JDK DNS caching setting, share DNS resolver instance in Proxy (#15219)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 837cfbd9a7bb9a5c6836ae851d0b8ee43b5a46f4
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Wed Apr 20 14:11:43 2022 +0300

    [Proxy & Client] Configure Netty DNS resolver to match JDK DNS caching setting, share DNS resolver instance in Proxy (#15219)
    
    - make Netty DNS resolver settings match the JDK DNS caching settings
      - with the exception that the max TTL is 60 seconds if DNS max TTL is the default (forever)
    - reuse the DNS resolver instance on the Proxy
    
    (cherry picked from commit f5adc17ed52db2da3c1c58ee4abc9b75b73464ac)
---
 .../apache/pulsar/client/impl/ConnectionPool.java  | 26 ++++++--
 pulsar-common/pom.xml                              |  5 ++
 .../pulsar/common/util/netty/DnsResolverUtil.java  | 75 ++++++++++++++++++++++
 .../pulsar/proxy/server/ProxyConnection.java       | 36 ++++++-----
 .../pulsar/proxy/server/ProxyConnectionPool.java   | 60 -----------------
 .../apache/pulsar/proxy/server/ProxyService.java   |  3 +
 .../proxy/server/ServiceChannelInitializer.java    |  2 +-
 7 files changed, 126 insertions(+), 81 deletions(-)

diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
index 92b4a3cda70..99492784e5f 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java
@@ -35,6 +35,7 @@ import java.net.URISyntaxException;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Random;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
@@ -46,6 +47,7 @@ import org.apache.pulsar.client.api.PulsarClientException.InvalidServiceURL;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.netty.DnsResolverUtil;
 import org.apache.pulsar.common.util.netty.EventLoopUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -61,13 +63,20 @@ public class ConnectionPool implements AutoCloseable {
     private final boolean isSniProxy;
 
     protected final DnsNameResolver dnsResolver;
+    private final boolean shouldCloseDnsResolver;
 
     public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException {
         this(conf, eventLoopGroup, () -> new ClientCnx(conf, eventLoopGroup));
     }
 
     public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup,
-            Supplier<ClientCnx> clientCnxSupplier) throws PulsarClientException {
+                          Supplier<ClientCnx> clientCnxSupplier) throws PulsarClientException {
+        this(conf, eventLoopGroup, clientCnxSupplier, Optional.empty());
+    }
+
+    public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup,
+                             Supplier<ClientCnx> clientCnxSupplier, Optional<DnsNameResolver> dnsNameResolver)
+            throws PulsarClientException {
         this.eventLoopGroup = eventLoopGroup;
         this.clientConfig = conf;
         this.maxConnectionsPerHosts = conf.getConnectionsPerBroker();
@@ -91,8 +100,15 @@ public class ConnectionPool implements AutoCloseable {
             throw new PulsarClientException(e);
         }
 
-        this.dnsResolver = new DnsNameResolverBuilder(eventLoopGroup.next()).traceEnabled(true)
-                .channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup)).build();
+        this.shouldCloseDnsResolver = !dnsNameResolver.isPresent();
+        this.dnsResolver = dnsNameResolver.orElseGet(() -> createDnsNameResolver(conf, eventLoopGroup));
+    }
+
+    private static DnsNameResolver createDnsNameResolver(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) {
+        DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder(eventLoopGroup.next())
+                .traceEnabled(true).channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup));
+        DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
+        return dnsNameResolverBuilder.build();
     }
 
     private static final Random random = new Random();
@@ -314,7 +330,9 @@ public class ConnectionPool implements AutoCloseable {
     @Override
     public void close() throws Exception {
         closeAllConnections();
-        dnsResolver.close();
+        if (shouldCloseDnsResolver) {
+            dnsResolver.close();
+        }
     }
 
     private void cleanupConnection(InetSocketAddress address, int connectionKey,
diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml
index e37022a3c74..0d746cd35e3 100644
--- a/pulsar-common/pom.xml
+++ b/pulsar-common/pom.xml
@@ -78,6 +78,11 @@
       <artifactId>netty-handler</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-resolver-dns</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>io.netty</groupId>
       <artifactId>netty-transport-native-epoll</artifactId>
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java
new file mode 100644
index 00000000000..8b06dbf36ec
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java
@@ -0,0 +1,75 @@
+/**
+ * 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.common.util.netty;
+
+import io.netty.resolver.dns.DnsNameResolverBuilder;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class DnsResolverUtil {
+    private static final int MIN_TTL = 0;
+    private static final int TTL;
+    private static final int NEGATIVE_TTL;
+
+    // default TTL value when JDK setting is "forever" (-1)
+    private static final int DEFAULT_TTL = 60;
+
+    // default negative TTL value when JDK setting is "forever" (-1)
+    private static final int DEFAULT_NEGATIVE_TTL = 10;
+
+    static {
+        int ttl = DEFAULT_TTL;
+        int negativeTtl = DEFAULT_NEGATIVE_TTL;
+        try {
+            // use reflection to call sun.net.InetAddressCachePolicy's get and getNegative methods for getting
+            // effective JDK settings for DNS caching
+            Class<?> inetAddressCachePolicyClass = Class.forName("sun.net.InetAddressCachePolicy");
+            Method getTTLMethod = inetAddressCachePolicyClass.getMethod("get");
+            ttl = (Integer) getTTLMethod.invoke(null);
+            Method getNegativeTTLMethod = inetAddressCachePolicyClass.getMethod("getNegative");
+            negativeTtl = (Integer) getNegativeTTLMethod.invoke(null);
+        } catch (NoSuchMethodException | ClassNotFoundException | InvocationTargetException
+                 | IllegalAccessException e) {
+            log.warn("Cannot get DNS TTL settings from sun.net.InetAddressCachePolicy class", e);
+        }
+        TTL = useDefaultTTLWhenSetToForever(ttl, DEFAULT_TTL);
+        NEGATIVE_TTL = useDefaultTTLWhenSetToForever(negativeTtl, DEFAULT_NEGATIVE_TTL);
+    }
+
+    private static int useDefaultTTLWhenSetToForever(int ttl, int defaultTtl) {
+        return ttl < 0 ? defaultTtl : ttl;
+    }
+
+    private DnsResolverUtil() {
+        // utility class with static methods, prevent instantiation
+    }
+
+    /**
+     * Configure Netty's {@link DnsNameResolverBuilder}'s ttl and negativeTtl to match the JDK's DNS caching settings.
+     * If the JDK setting for TTL is forever (-1), the TTL will be set to 60 seconds.
+     *
+     * @param dnsNameResolverBuilder The Netty {@link DnsNameResolverBuilder} instance to apply the settings
+     */
+    public static void applyJdkDnsCacheSettings(DnsNameResolverBuilder dnsNameResolverBuilder) {
+        dnsNameResolverBuilder.ttl(MIN_TTL, TTL);
+        dnsNameResolverBuilder.negativeTtl(NEGATIVE_TTL);
+    }
+}
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index 754e33f96a8..395d16bbbd8 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -22,9 +22,11 @@ import static com.google.common.base.Preconditions.checkArgument;
 
 import io.netty.channel.ChannelFutureListener;
 import io.netty.handler.codec.haproxy.HAProxyMessage;
+import io.netty.resolver.dns.DnsNameResolver;
 import java.net.SocketAddress;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -77,6 +79,7 @@ public class ProxyConnection extends PulsarHandler {
     private final AtomicLong requestIdGenerator =
             new AtomicLong(ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE / 2));
     private final ProxyService service;
+    private final DnsNameResolver dnsNameResolver;
     private Authentication clientAuthentication;
     AuthenticationDataSource authenticationData;
     private State state;
@@ -124,9 +127,11 @@ public class ProxyConnection extends PulsarHandler {
         return connectionPool;
     }
 
-    public ProxyConnection(ProxyService proxyService, Supplier<SslHandler> sslHandlerSupplier) {
+    public ProxyConnection(ProxyService proxyService, Supplier<SslHandler> sslHandlerSupplier,
+                           DnsNameResolver dnsNameResolver) {
         super(30, TimeUnit.SECONDS);
         this.service = proxyService;
+        this.dnsNameResolver = dnsNameResolver;
         this.state = State.Init;
         this.sslHandlerSupplier = sslHandlerSupplier;
         this.brokerProxyValidator = service.getBrokerProxyValidator();
@@ -234,27 +239,26 @@ public class ProxyConnection extends PulsarHandler {
     }
 
     private synchronized void completeConnect(AuthData clientData) throws PulsarClientException {
+        Supplier<ClientCnx> clientCnxSupplier;
         if (service.getConfiguration().isAuthenticationEnabled()) {
             if (service.getConfiguration().isForwardAuthorizationCredentials()) {
                 this.clientAuthData = clientData;
                 this.clientAuthMethod = authMethod;
             }
-            if (this.connectionPool == null) {
-                this.connectionPool = new ProxyConnectionPool(clientConf, service.getWorkerGroup(),
-                        () -> new ProxyClientCnx(clientConf, service.getWorkerGroup(), clientAuthRole, clientAuthData,
-                                clientAuthMethod, protocolVersionToAdvertise));
-            } else {
-                LOG.error("BUG! Connection Pool has already been created for proxy connection to {} state {} role {}",
-                        remoteAddress, state, clientAuthRole);
-            }
+            clientCnxSupplier =
+                    () -> new ProxyClientCnx(clientConf, service.getWorkerGroup(), clientAuthRole, clientAuthData,
+                            clientAuthMethod, protocolVersionToAdvertise);
         } else {
-            if (this.connectionPool == null) {
-                this.connectionPool = new ProxyConnectionPool(clientConf, service.getWorkerGroup(),
-                        () -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise));
-            } else {
-                LOG.error("BUG! Connection Pool has already been created for proxy connection to {} state {}",
-                        remoteAddress, state);
-            }
+            clientCnxSupplier =
+                    () -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise);
+        }
+
+        if (this.connectionPool == null) {
+            this.connectionPool = new ConnectionPool(clientConf, service.getWorkerGroup(),
+                    clientCnxSupplier, Optional.of(dnsNameResolver));
+        } else {
+            LOG.error("BUG! Connection Pool has already been created for proxy connection to {} state {} role {}",
+                    remoteAddress, state, clientAuthRole);
         }
 
         LOG.info("[{}] complete connection, init proxy handler. authenticated with {} role {}, hasProxyToBrokerUrl: {}",
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnectionPool.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnectionPool.java
deleted file mode 100644
index cd1b31d3434..00000000000
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnectionPool.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * 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.proxy.server;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-import java.util.function.Supplier;
-
-import org.apache.pulsar.client.api.PulsarClientException;
-import org.apache.pulsar.client.impl.ClientCnx;
-import org.apache.pulsar.client.impl.ConnectionPool;
-import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.netty.channel.EventLoopGroup;
-
-public class ProxyConnectionPool extends ConnectionPool {
-    public ProxyConnectionPool(ClientConfigurationData clientConfig, EventLoopGroup eventLoopGroup,
-            Supplier<ClientCnx> clientCnxSupplier) throws PulsarClientException {
-        super(clientConfig, eventLoopGroup, clientCnxSupplier);
-    }
-
-    @Override
-    public void close() throws IOException {
-        log.info("Closing ProxyConnectionPool.");
-        pool.forEach((address, clientCnxPool) -> {
-            if (clientCnxPool != null) {
-                clientCnxPool.forEach((identifier, clientCnx) -> {
-                    if (clientCnx != null && clientCnx.isDone()) {
-                        try {
-                            clientCnx.get().close();
-                        } catch (InterruptedException | ExecutionException e) {
-                            log.error("Unable to close get client connection future.", e);
-                        }
-                    }
-                });
-            }
-        });
-        dnsResolver.close();
-    }
-
-    private static final Logger log = LoggerFactory.getLogger(ProxyConnectionPool.class);
-}
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
index c9348837dcf..f24caf943b0 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java
@@ -52,6 +52,7 @@ import org.apache.pulsar.broker.authorization.AuthorizationService;
 import org.apache.pulsar.broker.cache.ConfigurationCacheService;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.util.netty.DnsResolverUtil;
 import org.apache.pulsar.common.util.netty.EventLoopUtil;
 import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets;
 import org.apache.pulsar.proxy.stats.TopicStats;
@@ -147,6 +148,8 @@ public class ProxyService implements Closeable {
 
         DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder(workerGroup.next())
                 .channelType(EventLoopUtil.getDatagramChannelClass(workerGroup));
+        DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
+
         dnsNameResolver = dnsNameResolverBuilder.build();
 
         brokerProxyValidator = new BrokerProxyValidator(dnsNameResolver.asAddressResolver(),
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
index a033a87912d..4aae1196fb6 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
@@ -159,7 +159,7 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
         }
 
         ch.pipeline().addLast("handler",
-                new ProxyConnection(proxyService, sslHandlerSupplier));
+                new ProxyConnection(proxyService, sslHandlerSupplier, proxyService.getDnsNameResolver()));
 
     }
 }


[pulsar] 08/10: [Proxy] Prevent leak of unreleased lookupRequestSemaphore permits (#13812)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit dbe0518554bfb4306adf23335d62eb8959b83b27
Author: Lari Hotari <lh...@users.noreply.github.com>
AuthorDate: Wed Jan 19 06:41:12 2022 +0200

    [Proxy] Prevent leak of unreleased lookupRequestSemaphore permits (#13812)
    
    * [Proxy] Prevent leak of unreleased lookupRequestSemaphore permits
    
    - should release permit in try-finally block
    
    * Cleanup code in LookupProxyHandler
    
    (cherry picked from commit 85b62e050b01b591a4b5751aab48b418ac9e4e76)
    (cherry picked from commit dcc07e8ebacec86a3779b289e235dd7731aa208e)
---
 .../pulsar/proxy/server/LookupProxyHandler.java    | 96 ++++++++++------------
 1 file changed, 45 insertions(+), 51 deletions(-)

diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
index a8525e4da98..8c18d456cbb 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
@@ -18,14 +18,13 @@
  */
 package org.apache.pulsar.proxy.server;
 
-import static org.apache.commons.lang3.StringUtils.isBlank;
 
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Optional;
-
+import java.util.concurrent.Semaphore;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.common.protocol.Commands;
@@ -47,8 +46,8 @@ import io.prometheus.client.Counter;
 
 public class LookupProxyHandler {
     private final String throttlingErrorMessage = "Too many concurrent lookup and partitionsMetadata requests";
-    private final ProxyService service;
     private final ProxyConnection proxyConnection;
+    private final BrokerDiscoveryProvider discoveryProvider;
     private final boolean connectWithTLS;
 
     private SocketAddress clientAddress;
@@ -83,9 +82,11 @@ public class LookupProxyHandler {
             .build("pulsar_proxy_rejected_get_topics_of_namespace_requests",
                     "Counter of getTopicsOfNamespace requests rejected due to throttling")
             .create().register();
+    private final Semaphore lookupRequestSemaphore;
 
     public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) {
-        this.service = proxy;
+        this.discoveryProvider = proxy.getDiscoveryProvider();
+        this.lookupRequestSemaphore = proxy.getLookupRequestSemaphore();
         this.proxyConnection = proxyConnection;
         this.clientAddress = proxyConnection.clientAddress();
         this.connectWithTLS = proxy.getConfiguration().isTlsEnabledWithBroker();
@@ -98,28 +99,16 @@ public class LookupProxyHandler {
             log.debug("Received Lookup from {}", clientAddress);
         }
         long clientRequestId = lookup.getRequestId();
-        if (this.service.getLookupRequestSemaphore().tryAcquire()) {
-            lookupRequests.inc();
-            String topic = lookup.getTopic();
-            String serviceUrl;
-            if (isBlank(brokerServiceURL)) {
-                ServiceLookupData availableBroker = null;
-                try {
-                    availableBroker = service.getDiscoveryProvider().nextBroker();
-                } catch (Exception e) {
-                    log.warn("[{}] Failed to get next active broker {}", clientAddress, e.getMessage(), e);
-                    proxyConnection.ctx().writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady,
-                            e.getMessage(), clientRequestId));
-                    return;
+        if (lookupRequestSemaphore.tryAcquire()) {
+            try {
+                lookupRequests.inc();
+                String serviceUrl = getBrokerServiceUrl(clientRequestId);
+                if (serviceUrl != null) {
+                    performLookup(clientRequestId, lookup.getTopic(), serviceUrl, false, 10);
                 }
-                serviceUrl = this.connectWithTLS ? availableBroker.getPulsarServiceUrlTls()
-                        : availableBroker.getPulsarServiceUrl();
-            } else {
-                serviceUrl = this.connectWithTLS ? service.getConfiguration().getBrokerServiceURLTLS()
-                        : service.getConfiguration().getBrokerServiceURL();
+            } finally {
+                lookupRequestSemaphore.release();
             }
-            performLookup(clientRequestId, topic, serviceUrl, false, 10);
-            this.service.getLookupRequestSemaphore().release();
         } else {
             rejectedLookupRequests.inc();
             if (log.isDebugEnabled()) {
@@ -203,9 +192,12 @@ public class LookupProxyHandler {
             log.debug("[{}] Received PartitionMetadataLookup", clientAddress);
         }
         final long clientRequestId = partitionMetadata.getRequestId();
-        if (this.service.getLookupRequestSemaphore().tryAcquire()) {
-            handlePartitionMetadataResponse(partitionMetadata, clientRequestId);
-            this.service.getLookupRequestSemaphore().release();
+        if (lookupRequestSemaphore.tryAcquire()) {
+            try {
+                handlePartitionMetadataResponse(partitionMetadata, clientRequestId);
+            } finally {
+                lookupRequestSemaphore.release();
+            }
         } else {
             rejectedPartitionsMetadataRequests.inc();
             if (log.isDebugEnabled()) {
@@ -226,7 +218,7 @@ public class LookupProxyHandler {
             long clientRequestId) {
         TopicName topicName = TopicName.get(partitionMetadata.getTopic());
 
-        String serviceUrl = getServiceUrl(clientRequestId);
+        String serviceUrl = getBrokerServiceUrl(clientRequestId);
         if (serviceUrl == null) {
             log.warn("No available broker for {} to lookup partition metadata", topicName);
             return;
@@ -273,9 +265,12 @@ public class LookupProxyHandler {
 
         final long requestId = commandGetTopicsOfNamespace.getRequestId();
 
-        if (this.service.getLookupRequestSemaphore().tryAcquire()) {
-            handleGetTopicsOfNamespace(commandGetTopicsOfNamespace, requestId);
-            this.service.getLookupRequestSemaphore().release();
+        if (lookupRequestSemaphore.tryAcquire()) {
+            try {
+                handleGetTopicsOfNamespace(commandGetTopicsOfNamespace, requestId);
+            } finally {
+                lookupRequestSemaphore.release();
+            }
         } else {
             rejectedGetTopicsOfNamespaceRequests.inc();
             if (log.isDebugEnabled()) {
@@ -290,7 +285,7 @@ public class LookupProxyHandler {
 
     private void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace,
                                             long clientRequestId) {
-        String serviceUrl = getServiceUrl(clientRequestId);
+        String serviceUrl = getBrokerServiceUrl(clientRequestId);
 
         if(!StringUtils.isNotBlank(serviceUrl)) {
             return;
@@ -352,7 +347,7 @@ public class LookupProxyHandler {
         }
 
         final long clientRequestId = commandGetSchema.getRequestId();
-        String serviceUrl = getServiceUrl(clientRequestId);
+        String serviceUrl = getBrokerServiceUrl(clientRequestId);
         String topic = commandGetSchema.getTopic();
         Optional<SchemaVersion> schemaVersion;
         if (commandGetSchema.hasSchemaVersion()) {
@@ -402,25 +397,24 @@ public class LookupProxyHandler {
 
     }
 
-    private String getServiceUrl(long clientRequestId) {
-        if (isBlank(brokerServiceURL)) {
-            ServiceLookupData availableBroker;
-            try {
-                availableBroker = service.getDiscoveryProvider().nextBroker();
-            } catch (Exception e) {
-                log.warn("[{}] Failed to get next active broker {}", clientAddress, e.getMessage(), e);
-                proxyConnection.ctx().writeAndFlush(Commands.newError(
-                        clientRequestId, ServerError.ServiceNotReady, e.getMessage()
-                ));
-                return null;
-            }
-            return this.connectWithTLS ?
-                    availableBroker.getPulsarServiceUrlTls() : availableBroker.getPulsarServiceUrl();
-        } else {
-            return this.connectWithTLS ?
-                    service.getConfiguration().getBrokerServiceURLTLS() : service.getConfiguration().getBrokerServiceURL();
+    /**
+     *  Get default broker service url or discovery an available broker.
+     **/
+    private String getBrokerServiceUrl(long clientRequestId) {
+        if (StringUtils.isNotBlank(brokerServiceURL)) {
+            return brokerServiceURL;
         }
-
+        ServiceLookupData availableBroker;
+        try {
+            availableBroker = discoveryProvider.nextBroker();
+        } catch (Exception e) {
+            log.warn("[{}] Failed to get next active broker {}", clientAddress, e.getMessage(), e);
+            proxyConnection.ctx().writeAndFlush(Commands.newError(
+                    clientRequestId, ServerError.ServiceNotReady, e.getMessage()
+            ));
+            return null;
+        }
+        return this.connectWithTLS ? availableBroker.getPulsarServiceUrlTls() : availableBroker.getPulsarServiceUrl();
     }
 
     private InetSocketAddress getAddr(String brokerServiceUrl, long clientRequestId) {


[pulsar] 10/10: Configure DLog Bookie, Pulsar, and Admin clients via pass through config (#15818)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e5d035ae413cbe67e0efa8afb7d62d979cc58659
Author: Michael Marshall <mm...@apache.org>
AuthorDate: Wed Jun 1 02:14:49 2022 -0500

    Configure DLog Bookie, Pulsar, and Admin clients via pass through config (#15818)
    
    (cherry picked from commit aa673498f88d0ed4f9d5788a5036355834ea5119)
---
 conf/broker.conf                                   |  14 ++-
 conf/functions_worker.yml                          |  20 ++++
 conf/proxy.conf                                    |   4 +
 conf/websocket.conf                                |   4 +
 .../pulsar/broker/BookKeeperClientFactoryImpl.java |  16 ++--
 .../org/apache/pulsar/broker/PulsarService.java    |  32 ++++---
 .../pulsar/broker/namespace/NamespaceService.java  |  10 +-
 .../pulsar/broker/service/BrokerService.java       |  21 ++++-
 .../apache/pulsar/compaction/CompactorTool.java    |   6 ++
 ...kerInternalClientConfigurationOverrideTest.java | 103 +++++++++++++++++++++
 .../PulsarClientConfigurationOverrideTest.java     |  56 +++++++++++
 .../websocket/proxy/ProxyConfigurationTest.java    |   8 +-
 .../pulsar/client/admin/PulsarAdminBuilder.java    |  23 +++++
 .../admin/internal/PulsarAdminBuilderImpl.java     |   9 +-
 .../pulsar/client/internal/PropertiesUtils.java    |  64 +++++++++++++
 .../src/test/resources/test_worker_config.yml      |   3 +
 .../pulsar/functions/worker/WorkerService.java     |   7 +-
 .../pulsar/functions/worker/WorkerUtils.java       |  45 ++++++++-
 .../pulsar/functions/worker/WorkerUtilsTest.java   |  44 +++++++++
 .../pulsar/proxy/server/ProxyConnection.java       |  16 +++-
 .../apache/pulsar/websocket/WebSocketService.java  |   7 +-
 site2/docs/reference-configuration.md              |  22 +++++
 22 files changed, 489 insertions(+), 45 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index 9534ed73c63..0c554ada913 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -570,6 +570,9 @@ brokerClientTlsCiphers=
 # used by the internal client to authenticate with Pulsar brokers
 brokerClientTlsProtocols=
 
+# You can add extra configuration options for the Pulsar Client and the Pulsar Admin Client
+# by prefixing them with "brokerClient_". These configurations are applied after hard coded configuration
+# and before the above brokerClient configurations named above.
 
 ### --- Authentication --- ###
 
@@ -798,8 +801,11 @@ managedLedgerDefaultWriteQuorum=2
 # Number of guaranteed copies (acks to wait before write is complete)
 managedLedgerDefaultAckQuorum=2
 
-# you can add other configuration options for the BookKeeper client
-# by prefixing them with bookkeeper_
+# You can add other configuration options for the BookKeeper client
+# by prefixing them with "bookkeeper_". These configurations are applied
+# to all bookkeeper clients started by the broker (including the managed ledger bookkeeper clients as well as
+# the BookkeeperPackagesStorage bookkeeper client), except the distributed log bookkeeper client.
+# The dlog bookkeeper client is configured in the functions worker configuration file.
 
 # How frequently to flush the cursor positions that were accumulated due to rate limiting. (seconds).
 # Default is 60 seconds
@@ -1175,6 +1181,10 @@ fileSystemURI=
 
 ### --- Deprecated config variables --- ###
 
+# When using BookKeeperPackagesStorageProvider, you can configure the
+# bookkeeper client by prefixing configurations with "bookkeeper_".
+# This config applies to managed ledger bookkeeper clients, as well.
+
 # Deprecated. Use configurationStoreServers
 globalZookeeperServers=
 
diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml
index 4dfcf586307..2376edbd2a3 100644
--- a/conf/functions_worker.yml
+++ b/conf/functions_worker.yml
@@ -243,3 +243,23 @@ functionsDirectory: ./functions
 
 # Should connector config be validated during during submission
 validateConnectorConfig: false
+###########################
+# Arbitrary Configuration
+###########################
+# When a configuration parameter is not explicitly named in the WorkerConfig class, it is only accessible from the
+# properties map. This map can be configured by supplying values to the properties map in this config file.
+
+# Configure the DLog bookkeeper client by prefixing configurations with "bookkeeper_". Because these are arbitrary, they
+# must be added to the properties map to get correctly applied. This configuration applies to the Dlog bookkeeper client
+# in both the standalone function workers and function workers initialized in the broker.
+
+# You can add extra configuration options for the Pulsar Client and the Pulsar Admin Client
+# by prefixing them with "brokerClient_". These configurations are applied after hard coded configuration
+# and before the above brokerClient configurations named above.
+
+## For example, when using the token authentication provider (AuthenticationProviderToken), you must configure several
+## custom configurations. Here is a sample for configuring one of the necessary configs:
+#properties:
+#    tokenPublicKey: "file:///path/to/my/key"
+#    tokenPublicAlg: "RSA256"
+
diff --git a/conf/proxy.conf b/conf/proxy.conf
index 5ac664ef48b..f0fa60483af 100644
--- a/conf/proxy.conf
+++ b/conf/proxy.conf
@@ -126,6 +126,10 @@ tlsEnabledWithBroker=false
 # Tls cert refresh duration in seconds (set 0 to check on every new connection)
 tlsCertRefreshCheckDurationSec=300
 
+# You can add extra configuration options for the Pulsar Client
+# by prefixing them with "brokerClient_". These configurations are applied after hard coded configuration
+# and before the above brokerClient configurations named above.
+
 ##### --- Rate Limiting --- #####
 
 # Max concurrent inbound connections. The proxy will reject requests beyond that.
diff --git a/conf/websocket.conf b/conf/websocket.conf
index 8e4a59be945..85005bab46c 100644
--- a/conf/websocket.conf
+++ b/conf/websocket.conf
@@ -92,6 +92,10 @@ brokerClientAuthenticationPlugin=
 brokerClientAuthenticationParameters=
 brokerClientTrustCertsFilePath=
 
+# You can add extra configuration options for the Pulsar Client
+# by prefixing them with "brokerClient_". These configurations are applied after hard coded configuration
+# and before the above brokerClient configurations named above.
+
 # When this parameter is not empty, unauthenticated users perform as anonymousUserRole
 anonymousUserRole=
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java
index f00c9050d5f..e2bf0d91729 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java
@@ -27,7 +27,6 @@ import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.RE
 import java.io.IOException;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import com.google.common.annotations.VisibleForTesting;
@@ -41,6 +40,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping;
 import org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy;
@@ -141,15 +141,11 @@ public class BookKeeperClientFactoryImpl implements BookKeeperClientFactory {
                 conf.getBookkeeperClientGetBookieInfoIntervalSeconds(), TimeUnit.SECONDS);
         bkConf.setGetBookieInfoRetryIntervalSeconds(
                 conf.getBookkeeperClientGetBookieInfoRetryIntervalSeconds(), TimeUnit.SECONDS);
-        Properties allProps = conf.getProperties();
-        allProps.forEach((key, value) -> {
-            String sKey = key.toString();
-            if (sKey.startsWith("bookkeeper_") && value != null) {
-                String bkExtraConfigKey = sKey.substring(11);
-                log.info("Extra BookKeeper client configuration {}, setting {}={}", sKey, bkExtraConfigKey, value);
-                bkConf.setProperty(bkExtraConfigKey, value);
-            }
-        });
+        PropertiesUtils.filterAndMapProperties(conf.getProperties(), "bookkeeper_")
+                .forEach((key, value) -> {
+                    log.info("Applying BookKeeper client configuration setting {}={}", key, value);
+                    bkConf.setProperty(key, value);
+                });
         return bkConf;
     }
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
index 3cfdaf73366..4492c1d1e0b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java
@@ -22,21 +22,17 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.commons.lang3.StringUtils.isNotBlank;
 import static org.apache.pulsar.broker.admin.impl.NamespacesBase.getBundles;
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
-
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.DefaultThreadFactory;
-
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.URI;
-
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
@@ -55,7 +51,6 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
 import java.util.function.Supplier;
-
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.Setter;
@@ -68,12 +63,13 @@ import org.apache.bookkeeper.mledger.LedgerOffloaderFactory;
 import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
 import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader;
 import org.apache.bookkeeper.mledger.offload.Offloaders;
-import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.mledger.offload.OffloadersCache;
+import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
 import org.apache.pulsar.PulsarVersion;
+import org.apache.pulsar.ZookeeperSessionExpiredHandlers;
 import org.apache.pulsar.broker.admin.AdminResource;
 import org.apache.pulsar.broker.authentication.AuthenticationService;
 import org.apache.pulsar.broker.authorization.AuthorizationService;
@@ -108,6 +104,7 @@ import org.apache.pulsar.client.api.ClientBuilder;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.transaction.TransactionBufferClient;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.conf.InternalConfigurationData;
 import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
 import org.apache.pulsar.common.configuration.VipStatus;
@@ -141,9 +138,8 @@ import org.apache.pulsar.zookeeper.LocalZooKeeperConnectionService;
 import org.apache.pulsar.zookeeper.ZooKeeperCache;
 import org.apache.pulsar.zookeeper.ZooKeeperCacheListener;
 import org.apache.pulsar.zookeeper.ZooKeeperClientFactory;
-import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl;
 import org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher.ShutdownService;
-import org.apache.pulsar.ZookeeperSessionExpiredHandlers;
+import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl;
 import org.apache.pulsar.zookeeper.ZookeeperSessionExpiredHandler;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -1060,6 +1056,12 @@ public class PulsarService implements AutoCloseable {
                     .enableTls(this.getConfiguration().isTlsEnabled())
                     .allowTlsInsecureConnection(this.getConfiguration().isTlsAllowInsecureConnection())
                     .tlsTrustCertsFilePath(this.getConfiguration().getTlsCertificateFilePath());
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                Map<String, Object> overrides = PropertiesUtils
+                        .filterAndMapProperties(this.getConfiguration().getProperties(), "brokerClient_");
+                builder.loadConf(overrides);
 
                 if (this.getConfiguration().isBrokerClientTlsEnabled()) {
                     if (this.getConfiguration().isBrokerClientTlsEnabledWithKeyStore()) {
@@ -1098,10 +1100,16 @@ public class PulsarService implements AutoCloseable {
                             + ", webServiceAddressTls: " + webServiceAddressTls
                             + ", webServiceAddress: " + webServiceAddress);
                 }
-                PulsarAdminBuilder builder = PulsarAdmin.builder().serviceHttpUrl(adminApiUrl) //
-                        .authentication( //
-                                conf.getBrokerClientAuthenticationPlugin(), //
-                                conf.getBrokerClientAuthenticationParameters());
+                PulsarAdminBuilder builder = PulsarAdmin.builder().serviceHttpUrl(adminApiUrl);
+
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                builder.loadConf(PropertiesUtils.filterAndMapProperties(config.getProperties(), "brokerClient_"));
+
+                builder.authentication(
+                        conf.getBrokerClientAuthenticationPlugin(),
+                        conf.getBrokerClientAuthenticationParameters());
 
                 if (conf.isBrokerClientTlsEnabled()) {
                     if (conf.isBrokerClientTlsEnabledWithKeyStore()) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
index 52cce387d55..72f233836c5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java
@@ -29,13 +29,10 @@ import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_PO
 import static org.apache.pulsar.broker.web.PulsarWebResource.joinPath;
 import static org.apache.pulsar.common.naming.NamespaceBundleFactory.getBundlesData;
 import static org.apache.pulsar.common.util.Codec.decode;
-
 import com.google.common.collect.Lists;
 import com.google.common.hash.Hashing;
-
 import io.netty.channel.EventLoopGroup;
 import io.prometheus.client.Counter;
-
 import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
@@ -53,7 +50,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -77,6 +73,7 @@ import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.impl.ClientBuilderImpl;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode;
 import org.apache.pulsar.common.lookup.data.LookupData;
 import org.apache.pulsar.common.naming.NamespaceBundle;
@@ -1256,6 +1253,11 @@ public class NamespaceService {
                     .enableTcpNoDelay(false)
                     .statsInterval(0, TimeUnit.SECONDS);
 
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                clientBuilder.loadConf(PropertiesUtils.filterAndMapProperties(config.getProperties(), "brokerClient_"));
+
                 if (pulsar.getConfiguration().isAuthenticationEnabled()) {
                     clientBuilder.authentication(pulsar.getConfiguration().getBrokerClientAuthenticationPlugin(),
                         pulsar.getConfiguration().getBrokerClientAuthenticationParameters());
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 47ecd6e7cbf..29d1001e4bd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -126,6 +126,7 @@ import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.impl.ClientBuilderImpl;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.apache.pulsar.common.configuration.FieldContext;
 import org.apache.pulsar.common.events.EventsTopicNames;
@@ -943,6 +944,12 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                         .enableTcpNoDelay(false)
                         .connectionsPerBroker(pulsar.getConfiguration().getReplicationConnectionsPerBroker())
                         .statsInterval(0, TimeUnit.SECONDS);
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                clientBuilder.loadConf(PropertiesUtils.filterAndMapProperties(pulsar.getConfiguration().getProperties(),
+                    "brokerClient_"));
+
                 if (pulsar.getConfiguration().isAuthenticationEnabled()) {
                     clientBuilder.authentication(pulsar.getConfiguration().getBrokerClientAuthenticationPlugin(),
                             pulsar.getConfiguration().getBrokerClientAuthenticationParameters());
@@ -996,10 +1003,16 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
 
                 boolean isTlsUrl = conf.isBrokerClientTlsEnabled() && isNotBlank(data.getServiceUrlTls());
                 String adminApiUrl = isTlsUrl ? data.getServiceUrlTls() : data.getServiceUrl();
-                PulsarAdminBuilder builder = PulsarAdmin.builder().serviceHttpUrl(adminApiUrl)
-                        .authentication(
-                                conf.getBrokerClientAuthenticationPlugin(),
-                                conf.getBrokerClientAuthenticationParameters());
+                PulsarAdminBuilder builder = PulsarAdmin.builder().serviceHttpUrl(adminApiUrl);
+
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                builder.loadConf(PropertiesUtils.filterAndMapProperties(conf.getProperties(), "brokerClient_"));
+
+                builder.authentication(
+                        conf.getBrokerClientAuthenticationPlugin(),
+                        conf.getBrokerClientAuthenticationParameters());
 
                 if (isTlsUrl) {
                     builder.allowTlsInsecureConnection(conf.isTlsAllowInsecureConnection());
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java
index 4e3b76e57de..88609157cd5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java
@@ -36,6 +36,7 @@ import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.ServiceConfigurationUtils;
 import org.apache.pulsar.client.api.ClientBuilder;
 import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
 import org.apache.pulsar.zookeeper.ZooKeeperClientFactory;
 import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl;
@@ -80,6 +81,11 @@ public class CompactorTool {
 
         ClientBuilder clientBuilder = PulsarClient.builder();
 
+        // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+        // @Secret on the ClientConfigurationData object because of the way they are serialized.
+        // See https://github.com/apache/pulsar/issues/8509 for more information.
+        clientBuilder.loadConf(PropertiesUtils.filterAndMapProperties(brokerConfig.getProperties(), "brokerClient_"));
+
         if (isNotBlank(brokerConfig.getBrokerClientAuthenticationPlugin())) {
             clientBuilder.authentication(brokerConfig.getBrokerClientAuthenticationPlugin(),
                     brokerConfig.getBrokerClientAuthenticationParameters());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java
new file mode 100644
index 00000000000..407e9ea05fd
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java
@@ -0,0 +1,103 @@
+/**
+ * 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.broker.service;
+
+import java.util.Properties;
+import org.apache.pulsar.broker.PulsarServerException;
+import org.apache.pulsar.client.impl.PulsarClientImpl;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class BrokerInternalClientConfigurationOverrideTest extends BrokerTestBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        super.baseSetup();
+    }
+
+    @AfterClass(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testPulsarServiceAdminClientConfiguration() throws PulsarServerException {
+        Properties config = pulsar.getConfiguration().getProperties();
+        config.setProperty("brokerClient_operationTimeoutMs", "60000");
+        config.setProperty("brokerClient_statsIntervalSeconds", "10");
+        ClientConfigurationData clientConf = pulsar.getAdminClient().getClientConfigData();
+        Assert.assertEquals(clientConf.getOperationTimeoutMs(), 60000);
+        Assert.assertEquals(clientConf.getStatsIntervalSeconds(), 10);
+    }
+
+    @Test
+    public void testPulsarServicePulsarClientConfiguration() throws PulsarServerException {
+        Properties config = pulsar.getConfiguration().getProperties();
+        config.setProperty("brokerClient_operationTimeoutMs", "60000");
+        config.setProperty("brokerClient_statsIntervalSeconds", "10");
+        pulsar.getConfiguration().setBrokerClientAuthenticationParameters("sensitive");
+        ClientConfigurationData clientConf = ((PulsarClientImpl) pulsar.getClient()).getConfiguration();
+        Assert.assertEquals(clientConf.getOperationTimeoutMs(), 60000);
+        // Config should override internal default, which is 0.
+        Assert.assertEquals(clientConf.getStatsIntervalSeconds(), 10);
+        Assert.assertEquals(clientConf.getAuthParams(), "sensitive");
+    }
+
+    @Test
+    public void testBrokerServicePulsarClientConfiguration() {
+        // This data only needs to have the service url for this test.
+        ClusterData data = new ClusterData("http://localhost:8080");
+
+        // Set the configs and set some configs that won't apply
+        Properties config = pulsar.getConfiguration().getProperties();
+        config.setProperty("brokerClient_operationTimeoutMs", "60000");
+        config.setProperty("brokerClient_statsIntervalSeconds", "10");
+
+        PulsarClientImpl client = (PulsarClientImpl) pulsar.getBrokerService()
+                .getReplicationClient("test");
+        ClientConfigurationData clientConf = client.getConfiguration();
+        Assert.assertEquals(clientConf.getOperationTimeoutMs(), 60000);
+        // Config should override internal default, which is 0.
+        Assert.assertEquals(clientConf.getStatsIntervalSeconds(), 10);
+    }
+
+    @Test
+    public void testNamespaceServicePulsarClientConfiguration() {
+        // This data only needs to have the service url for this test.
+        ClusterData data = new ClusterData("http://localhost:8080");
+
+        // Set the configs and set some configs that won't apply
+        Properties config = pulsar.getConfiguration().getProperties();
+        config.setProperty("brokerClient_operationTimeoutMs", "60000");
+        config.setProperty("brokerClient_statsIntervalSeconds", "10");
+
+        PulsarClientImpl client = pulsar.getNamespaceService().getNamespaceClient(data);
+        ClientConfigurationData clientConf = client.getConfiguration();
+        Assert.assertEquals(clientConf.getOperationTimeoutMs(), 60000);
+        // Config should override internal default, which is 0.
+        Assert.assertEquals(clientConf.getStatsIntervalSeconds(), 10);
+    }
+
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarClientConfigurationOverrideTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarClientConfigurationOverrideTest.java
new file mode 100644
index 00000000000..4f885ecc46b
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarClientConfigurationOverrideTest.java
@@ -0,0 +1,56 @@
+/**
+ * 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.impl;
+
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.internal.PropertiesUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Map;
+
+public class PulsarClientConfigurationOverrideTest {
+    @Test
+    public void testFilterAndMapProperties() {
+        // Create a default config
+        ServiceConfiguration conf = new ServiceConfiguration();
+        conf.getProperties().setProperty("keepAliveIntervalSeconds", "15");
+        conf.getProperties().setProperty("brokerClient_keepAliveIntervalSeconds", "25");
+
+        // Apply the filtering and mapping logic
+        Map<String, Object> result = PropertiesUtils.filterAndMapProperties(conf.getProperties(), "brokerClient_");
+
+        // Ensure the results match expectations
+        Assert.assertEquals(result.size(), 1, "The filtered map should have one entry.");
+        Assert.assertNull(result.get("brokerClient_keepAliveIntervalSeconds"),
+                "The mapped prop should not be in the result.");
+        Assert.assertEquals(result.get("keepAliveIntervalSeconds"), "25", "The original value is overridden.");
+
+        // Create sample ClientBuilder
+        ClientBuilder builder = PulsarClient.builder();
+        Assert.assertEquals(
+                ((ClientBuilderImpl) builder).getClientConfigurationData().getKeepAliveIntervalSeconds(), 30);
+        // Note: this test would fail if any @Secret fields were set before the loadConf and the accessed afterwards.
+        builder.loadConf(result);
+        Assert.assertEquals(
+                ((ClientBuilderImpl) builder).getClientConfigurationData().getKeepAliveIntervalSeconds(), 25);
+    }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyConfigurationTest.java
index afe91a77e3e..af2bd957c8d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyConfigurationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyConfigurationTest.java
@@ -21,9 +21,7 @@ package org.apache.pulsar.websocket.proxy;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.spy;
 import static org.testng.Assert.assertEquals;
-
 import java.util.Optional;
-
 import org.apache.pulsar.client.api.ProducerConsumerBase;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.websocket.WebSocketService;
@@ -61,6 +59,9 @@ public class ProxyConfigurationTest extends ProducerConsumerBase {
     public void configTest(int numIoThreads, int connectionsPerBroker) throws Exception {
         config.setWebSocketNumIoThreads(numIoThreads);
         config.setWebSocketConnectionsPerBroker(connectionsPerBroker);
+        config.getProperties().setProperty("brokerClient_serviceUrl", "https://broker.com:8080");
+        config.setServiceUrl("http://localhost:8080");
+        config.getProperties().setProperty("brokerClient_requestTimeoutMs", "100");
         WebSocketService service = spy(new WebSocketService(config));
         doReturn(mockZooKeeperClientFactory).when(service).getZooKeeperClientFactory();
         service.start();
@@ -68,6 +69,9 @@ public class ProxyConfigurationTest extends ProducerConsumerBase {
         PulsarClientImpl client = (PulsarClientImpl) service.getPulsarClient();
         assertEquals(client.getConfiguration().getNumIoThreads(), numIoThreads);
         assertEquals(client.getConfiguration().getConnectionsPerBroker(), connectionsPerBroker);
+        assertEquals(client.getConfiguration().getServiceUrl(), "http://localhost:8080",
+                "brokerClient_ configs take precedence");
+        assertEquals(client.getConfiguration().getRequestTimeoutMs(), 100);
 
         service.close();
     }
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
index 9de2d39a7ca..c1c9b800db2 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
@@ -37,6 +37,29 @@ public interface PulsarAdminBuilder {
      */
     PulsarAdmin build() throws PulsarClientException;
 
+    /**
+     * Load the configuration from provided <tt>config</tt> map.
+     *
+     * <p>Example:
+     *
+     * <pre>
+     * {@code
+     * Map<String, Object> config = new HashMap<>();
+     * config.put("serviceHttpUrl", "http://localhost:6650");
+     *
+     * PulsarAdminBuilder builder = ...;
+     * builder = builder.loadConf(config);
+     *
+     * PulsarAdmin client = builder.build();
+     * }
+     * </pre>
+     *
+     * @param config
+     *            configuration to load
+     * @return the client builder instance
+     */
+    PulsarAdminBuilder loadConf(Map<String, Object> config);
+
     /**
      * Create a copy of the current client builder.
      * <p/>
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
index 146faaf29d7..7e552adfa49 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
@@ -29,10 +29,11 @@ import org.apache.pulsar.client.api.AuthenticationFactory;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils;
 
 public class PulsarAdminBuilderImpl implements PulsarAdminBuilder {
 
-    protected final ClientConfigurationData conf;
+    protected ClientConfigurationData conf;
     private int connectTimeout = PulsarAdmin.DEFAULT_CONNECT_TIMEOUT_SECONDS;
     private int readTimeout = PulsarAdmin.DEFAULT_READ_TIMEOUT_SECONDS;
     private int requestTimeout = PulsarAdmin.DEFAULT_REQUEST_TIMEOUT_SECONDS;
@@ -61,6 +62,12 @@ public class PulsarAdminBuilderImpl implements PulsarAdminBuilder {
         return new PulsarAdminBuilderImpl(conf.clone());
     }
 
+    @Override
+    public PulsarAdminBuilder loadConf(Map<String, Object> config) {
+        conf = ConfigurationDataUtils.loadData(config, conf, ClientConfigurationData.class);
+        return this;
+    }
+
     @Override
     public PulsarAdminBuilder serviceHttpUrl(String serviceHttpUrl) {
         conf.setServiceUrl(serviceHttpUrl);
diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PropertiesUtils.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PropertiesUtils.java
new file mode 100644
index 00000000000..4a418b1d515
--- /dev/null
+++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PropertiesUtils.java
@@ -0,0 +1,64 @@
+/**
+ * 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.internal;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Internal utility methods for filtering and mapping {@link Properties} objects.
+ */
+public class PropertiesUtils {
+
+    /**
+     * Filters the {@link Properties} object so that only properties with the configured prefix are retained,
+     * and then removes that prefix and puts the key value pairs into the result map.
+     * @param props - the properties object to filter
+     * @param prefix - the prefix to filter against and then remove for keys in the resulting map
+     * @return a map of properties
+     */
+    public static Map<String, Object> filterAndMapProperties(Properties props, String prefix) {
+        return filterAndMapProperties(props, prefix, "");
+    }
+
+    /**
+     * Filters the {@link Properties} object so that only properties with the configured prefix are retained,
+     * and then replaces the srcPrefix with the targetPrefix when putting the key value pairs in the resulting map.
+     * @param props - the properties object to filter
+     * @param srcPrefix - the prefix to filter against and then remove for keys in the resulting map
+     * @param targetPrefix - the prefix to add to keys in the result map
+     * @return a map of properties
+     */
+    public static Map<String, Object> filterAndMapProperties(Properties props, String srcPrefix, String targetPrefix) {
+        Map<String, Object> result = new HashMap<>();
+        int prefixLength = srcPrefix.length();
+        props.forEach((keyObject, value) -> {
+            if (!(keyObject instanceof String)) {
+                return;
+            }
+            String key = (String) keyObject;
+            if (key.startsWith(srcPrefix) && value != null) {
+                String truncatedKey = key.substring(prefixLength);
+                result.put(targetPrefix + truncatedKey, value);
+            }
+        });
+        return result;
+    }
+}
diff --git a/pulsar-functions/src/test/resources/test_worker_config.yml b/pulsar-functions/src/test/resources/test_worker_config.yml
index f2645f61e8d..36c762af45c 100644
--- a/pulsar-functions/src/test/resources/test_worker_config.yml
+++ b/pulsar-functions/src/test/resources/test_worker_config.yml
@@ -22,4 +22,7 @@ workerPort: 7654
 pulsarServiceUrl: pulsar://localhost:6650
 functionMetadataTopicName: test-function-metadata-topic
 numFunctionPackageReplicas: 3
+properties:
+  # Fake Bookkeeper Client config to be applied to the DLog Bookkeeper Client
+  bookkeeper_testKey: "fakeValue"
 
diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java
index d0218b45790..b660ff64823 100644
--- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java
+++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java
@@ -148,18 +148,19 @@ public class WorkerService {
                 this.brokerAdmin = WorkerUtils.getPulsarAdminClient(workerConfig.getPulsarWebServiceUrl(),
                     workerConfig.getBrokerClientAuthenticationPlugin(), workerConfig.getBrokerClientAuthenticationParameters(),
                     pulsarClientTlsTrustCertsFilePath, workerConfig.isTlsAllowInsecureConnection(),
-                    workerConfig.isTlsEnableHostnameVerification());
+                    workerConfig.isTlsEnableHostnameVerification(), workerConfig);
 
                 this.functionAdmin = WorkerUtils.getPulsarAdminClient(functionWebServiceUrl,
                     workerConfig.getBrokerClientAuthenticationPlugin(), workerConfig.getBrokerClientAuthenticationParameters(),
                     workerConfig.getTlsTrustCertsFilePath(), workerConfig.isTlsAllowInsecureConnection(),
-                    workerConfig.isTlsEnableHostnameVerification());
+                    workerConfig.isTlsEnableHostnameVerification(), workerConfig);
 
                 this.client = WorkerUtils.getPulsarClient(workerConfig.getPulsarServiceUrl(),
                         workerConfig.getBrokerClientAuthenticationPlugin(),
                         workerConfig.getBrokerClientAuthenticationParameters(),
                         workerConfig.isUseTls(), pulsarClientTlsTrustCertsFilePath,
-                        workerConfig.isTlsAllowInsecureConnection(), workerConfig.isTlsEnableHostnameVerification());
+                        workerConfig.isTlsAllowInsecureConnection(), workerConfig.isTlsEnableHostnameVerification(),
+                        workerConfig);
             } else {
                 this.brokerAdmin = WorkerUtils.getPulsarAdminClient(workerConfig.getPulsarWebServiceUrl());
 
diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java
index 81659ad8095..05eb809337f 100644
--- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java
+++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java
@@ -38,6 +38,7 @@ import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.Reader;
 import org.apache.pulsar.client.api.ReaderBuilder;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.conf.InternalConfigurationData;
 import org.apache.pulsar.common.policies.data.FunctionStats;
 import org.apache.pulsar.functions.proto.Function;
@@ -152,6 +153,13 @@ public final class WorkerUtils {
                         workerConfig.getBookkeeperClientAuthenticationParameters());
             }
         }
+        // Map arbitrary bookkeeper client configuration into DLog Config. Note that this only configures the
+        // bookie client.
+        PropertiesUtils.filterAndMapProperties(workerConfig.getProperties(), "bookkeeper_", "bkc.")
+                .forEach((key, value) -> {
+                    log.info("Applying DLog BookKeeper client configuration setting {}={}", key, value);
+                    conf.setProperty(key, value);
+                });
         return conf;
     }
 
@@ -184,12 +192,20 @@ public final class WorkerUtils {
     }
 
     public static PulsarAdmin getPulsarAdminClient(String pulsarWebServiceUrl) {
-        return getPulsarAdminClient(pulsarWebServiceUrl, null, null, null, null, null);
+        return getPulsarAdminClient(pulsarWebServiceUrl, null, null, null, null, null, null);
     }
 
     public static PulsarAdmin getPulsarAdminClient(String pulsarWebServiceUrl, String authPlugin, String authParams,
                                                    String tlsTrustCertsFilePath, Boolean allowTlsInsecureConnection,
                                                    Boolean enableTlsHostnameVerificationEnable) {
+        return getPulsarAdminClient(pulsarWebServiceUrl, authPlugin, authParams, tlsTrustCertsFilePath,
+                allowTlsInsecureConnection, enableTlsHostnameVerificationEnable, null);
+    }
+
+    public static PulsarAdmin getPulsarAdminClient(String pulsarWebServiceUrl, String authPlugin, String authParams,
+                                                   String tlsTrustCertsFilePath, Boolean allowTlsInsecureConnection,
+                                                   Boolean enableTlsHostnameVerificationEnable,
+                                                   WorkerConfig workerConfig) {
         log.info("Create Pulsar Admin to service url {}: "
                 + "authPlugin = {}, authParams = {}, "
                 + "tlsTrustCerts = {}, allowTlsInsecureConnector = {}, enableTlsHostnameVerification = {}",
@@ -197,6 +213,13 @@ public final class WorkerUtils {
             tlsTrustCertsFilePath, allowTlsInsecureConnection, enableTlsHostnameVerificationEnable);
         try {
             PulsarAdminBuilder adminBuilder = PulsarAdmin.builder().serviceHttpUrl(pulsarWebServiceUrl);
+            if (workerConfig != null) {
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                adminBuilder.loadConf(
+                        PropertiesUtils.filterAndMapProperties(workerConfig.getProperties(), "brokerClient_"));
+            }
             if (isNotBlank(authPlugin) && isNotBlank(authParams)) {
                 adminBuilder.authentication(authPlugin, authParams);
             }
@@ -209,6 +232,7 @@ public final class WorkerUtils {
             if (enableTlsHostnameVerificationEnable != null) {
                 adminBuilder.enableTlsHostnameVerification(enableTlsHostnameVerificationEnable);
             }
+
             return adminBuilder.build();
         } catch (PulsarClientException e) {
             log.error("Error creating pulsar admin client", e);
@@ -218,17 +242,33 @@ public final class WorkerUtils {
 
     public static PulsarClient getPulsarClient(String pulsarServiceUrl) {
         return getPulsarClient(pulsarServiceUrl, null, null, null,
-                null, null, null);
+                null, null, null, null);
     }
 
     public static PulsarClient getPulsarClient(String pulsarServiceUrl, String authPlugin, String authParams,
                                                Boolean useTls, String tlsTrustCertsFilePath,
                                                Boolean allowTlsInsecureConnection,
                                                Boolean enableTlsHostnameVerificationEnable) {
+        return getPulsarClient(pulsarServiceUrl, authPlugin, authParams, useTls, tlsTrustCertsFilePath,
+                allowTlsInsecureConnection, enableTlsHostnameVerificationEnable, null);
+    }
+
+    public static PulsarClient getPulsarClient(String pulsarServiceUrl, String authPlugin, String authParams,
+                                               Boolean useTls, String tlsTrustCertsFilePath,
+                                               Boolean allowTlsInsecureConnection,
+                                               Boolean enableTlsHostnameVerificationEnable,
+                                               WorkerConfig workerConfig) {
 
         try {
             ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(pulsarServiceUrl);
 
+            if (workerConfig != null) {
+                // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+                // @Secret on the ClientConfigurationData object because of the way they are serialized.
+                // See https://github.com/apache/pulsar/issues/8509 for more information.
+                clientBuilder.loadConf(
+                        PropertiesUtils.filterAndMapProperties(workerConfig.getProperties(), "brokerClient_"));
+            }
             if (isNotBlank(authPlugin)
                     && isNotBlank(authParams)) {
                 clientBuilder.authentication(authPlugin, authParams);
@@ -245,7 +285,6 @@ public final class WorkerUtils {
             if (enableTlsHostnameVerificationEnable != null) {
                 clientBuilder.enableTlsHostnameVerification(enableTlsHostnameVerificationEnable);
             }
-
             return clientBuilder.build();
         } catch (PulsarClientException e) {
             log.error("Error creating pulsar client", e);
diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/WorkerUtilsTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/WorkerUtilsTest.java
new file mode 100644
index 00000000000..042b505472d
--- /dev/null
+++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/WorkerUtilsTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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.functions.worker;
+
+import static org.testng.Assert.assertEquals;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.testng.annotations.Test;
+
+public class WorkerUtilsTest {
+
+
+    @Test
+    public void testDLogConfiguration() throws URISyntaxException, IOException {
+        // The config yml is seeded with a fake bookie config.
+        URL yamlUrl = getClass().getClassLoader().getResource("test_worker_config.yml");
+        WorkerConfig config = WorkerConfig.load(yamlUrl.toURI().getPath());
+
+        // Map the config.
+        DistributedLogConfiguration dlogConf = WorkerUtils.getDlogConf(config);
+
+        // Verify the outcome.
+        assertEquals(dlogConf.getString("bkc.testKey"), "fakeValue",
+                "The bookkeeper client config mapping should apply.");
+    }
+}
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index bdfb11a9f0e..3c73284e7ed 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -27,6 +27,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
@@ -47,6 +48,8 @@ import org.apache.pulsar.client.impl.ClientCnx;
 import org.apache.pulsar.client.impl.ConnectionPool;
 import org.apache.pulsar.client.impl.PulsarChannelInitializer;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.api.AuthData;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.common.protocol.PulsarHandler;
@@ -524,10 +527,17 @@ public class ProxyConnection extends PulsarHandler {
     }
 
     ClientConfigurationData createClientConfiguration()
-            throws PulsarClientException.UnsupportedAuthenticationException {
-        ClientConfigurationData clientConf = new ClientConfigurationData();
-        clientConf.setServiceUrl(service.getServiceUrl());
+        throws PulsarClientException.UnsupportedAuthenticationException {
+        ClientConfigurationData initialConf = new ClientConfigurationData();
+        initialConf.setServiceUrl(service.getServiceUrl());
         ProxyConfiguration proxyConfig = service.getConfiguration();
+        // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+        // @Secret on the ClientConfigurationData object because of the way they are serialized.
+        // See https://github.com/apache/pulsar/issues/8509 for more information.
+        Map<String, Object> overrides = PropertiesUtils
+                .filterAndMapProperties(proxyConfig.getProperties(), "brokerClient_");
+        ClientConfigurationData clientConf = ConfigurationDataUtils
+                .loadData(overrides, initialConf, ClientConfigurationData.class);
         if (proxyConfig.getBrokerClientAuthenticationPlugin() != null) {
             clientConf.setAuthentication(AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(),
                     proxyConfig.getBrokerClientAuthenticationParameters()));
diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java
index 7b7ba26f44e..c36ad98740a 100644
--- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java
+++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java
@@ -40,6 +40,7 @@ import org.apache.pulsar.broker.cache.ConfigurationCacheService;
 import org.apache.pulsar.client.api.ClientBuilder;
 import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.internal.PropertiesUtils;
 import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
@@ -186,6 +187,11 @@ public class WebSocketService implements Closeable {
                 .ioThreads(config.getWebSocketNumIoThreads()) //
                 .connectionsPerBroker(config.getWebSocketConnectionsPerBroker());
 
+        // Apply all arbitrary configuration. This must be called before setting any fields annotated as
+        // @Secret on the ClientConfigurationData object because of the way they are serialized.
+        // See https://github.com/apache/pulsar/issues/8509 for more information.
+        clientBuilder.loadConf(PropertiesUtils.filterAndMapProperties(config.getProperties(), "brokerClient_"));
+
         if (isNotBlank(config.getBrokerClientAuthenticationPlugin())
                 && isNotBlank(config.getBrokerClientAuthenticationParameters())) {
             clientBuilder.authentication(config.getBrokerClientAuthenticationPlugin(),
@@ -203,7 +209,6 @@ public class WebSocketService implements Closeable {
         } else {
             clientBuilder.serviceUrl(clusterData.getServiceUrl());
         }
-
         return clientBuilder.build();
     }
 
diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md
index d03cb2eae9d..7c55b744e03 100644
--- a/site2/docs/reference-configuration.md
+++ b/site2/docs/reference-configuration.md
@@ -345,6 +345,15 @@ subscriptionExpirationTimeMinutes | How long to delete inactive subscriptions fr
 | preciseTopicPublishRateLimiterEnable | Enable precise topic publish rate limiting. | false |
 | lazyCursorRecovery | Whether to recover cursors lazily when trying to recover a managed ledger backing a persistent topic. It can improve write availability of topics. The caveat is now when recovered ledger is ready to write we're not sure if all old consumers' last mark delete position(ack position) can be recovered or not. So user can make the trade off or have custom logic in application to checkpoint consumer state.| false |  
 
+#### Configuration Override For Clients Internal to Broker
+
+It's possible to configure some clients by using the appropriate prefix.
+
+|Prefix|Description|
+|brokerClient_| Configure **all** the broker's Pulsar Clients and Pulsar Admin Clients. These configurations are applied after hard coded configuration and before the above brokerClient configurations named above.|
+|bookkeeper_| Configure the broker's bookkeeper clients used by managed ledgers and the BookkeeperPackagesStorage bookkeeper client. Takes precedence over most other configuration values.|
+
+Note: when running the function worker within the broker, these prefixed configurations do not apply to any of those clients. You must instead configure those clients using the `functions_worker.yml`.
 
 
 ## Client
@@ -682,6 +691,12 @@ The value of 0 disables message-byte dispatch-throttling.|0|
 |tlsKeyFilePath |||
 |tlsTrustCertsFilePath|||
 
+#### Configuration Override For Clients Internal to WebSocket
+
+It's possible to configure some clients by using the appropriate prefix.
+
+|Prefix|Description|
+|brokerClient_| Configure **all** the broker's Pulsar Clients. These configurations are applied after hard coded configuration and before the above brokerClient configurations named above.|
 
 ## Pulsar proxy
 
@@ -746,6 +761,13 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config
 | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| |
 | proxyLogLevel | Set the Pulsar Proxy log level. <li> If the value is set to 0, no TCP channel information is logged. <li> If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged. <li> If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 |
 
+#### Configuration Override For Clients Internal to Proxy
+
+It's possible to configure some clients by using the appropriate prefix.
+
+|Prefix|Description|
+|brokerClient_| Configure **all** the proxy's Pulsar Clients. These configurations are applied after hard coded configuration and before the above brokerClient configurations named above.|
+
 ## ZooKeeper
 
 ZooKeeper handles a broad range of essential configuration- and coordination-related tasks for Pulsar. The default configuration file for ZooKeeper is in the `conf/zookeeper.conf` file in your Pulsar installation. The following parameters are available:


[pulsar] 09/10: Switch to rely on Netty for Hostname Verification (#15824)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0348502a20694b58e6e79a467ca3a2142a90800f
Author: Michael Marshall <mm...@apache.org>
AuthorDate: Wed Jun 1 00:00:01 2022 -0500

    Switch to rely on Netty for Hostname Verification (#15824)
    
    * Switch to relying on Netty for Hostname Verification
    
    - Add "subjectAltName = DNS:localhost, IP:127.0.0.1" to unit test certs
    
    Co-authored-by: Lari Hotari <lh...@apache.org>
    (cherry picked from commit aa7700dbf45303fab8c874bd9e5fcf95745d2777)
---
 .../resources/authentication/tls/broker-cert.pem   |  74 +++++++-------
 .../test/resources/authentication/tls/cacert.pem   | 109 ++++++++++++---------
 .../resources/authentication/tls/client-cert.pem   |  87 ++++++++--------
 build/regenerate_certs_for_tests.sh                |  70 +++++++++++++
 .../AuthenticationTlsHostnameVerificationTest.java |  58 +++++------
 .../admin/internal/http/AsyncHttpConnector.java    |   9 +-
 .../org/apache/pulsar/client/impl/ClientCnx.java   |  48 ---------
 .../org/apache/pulsar/client/impl/HttpClient.java  |   1 +
 .../client/impl/PulsarChannelInitializer.java      |   7 ++
 .../util/NettyClientSslContextRefresher.java       |   3 +-
 .../apache/pulsar/common/util/SecurityUtility.java |  10 ++
 .../pulsar/proxy/server/AdminProxyHandler.java     |   7 +-
 .../pulsar/proxy/server/DirectProxyHandler.java    |  93 ++++++++++++------
 .../pulsar/proxy/server/ProxyConnection.java       |   9 +-
 .../proxy/server/ServiceChannelInitializer.java    |  58 +----------
 .../proxy/server/ProxyWithAuthorizationTest.java   |  70 +++++++------
 .../ProxyWithAuthorizationTest/broker-cacert.pem   | 109 ++++++++++++---------
 .../tls/ProxyWithAuthorizationTest/broker-cert.pem |  86 ++++++++--------
 .../ProxyWithAuthorizationTest/client-cacert.pem   | 109 ++++++++++++---------
 .../tls/ProxyWithAuthorizationTest/client-cert.pem |  86 ++++++++--------
 .../ProxyWithAuthorizationTest/proxy-cacert.pem    | 109 ++++++++++++---------
 .../tls/ProxyWithAuthorizationTest/proxy-cert.pem  |  86 ++++++++--------
 .../test/resources/authentication/tls/cacert.pem   | 109 ++++++++++++---------
 .../resources/authentication/tls/client-cert.pem   |  87 ++++++++--------
 .../resources/authentication/tls/server-cert.pem   |  87 ++++++++--------
 25 files changed, 837 insertions(+), 744 deletions(-)

diff --git a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-cert.pem b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-cert.pem
index 69ad71cd0cb..0aac0e9fec6 100644
--- a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-cert.pem
+++ b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-cert.pem
@@ -2,12 +2,12 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:97
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:05
         Signature Algorithm: sha1WithRSAEncryption
         Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
         Validity
-            Not Before: Feb 17 02:06:21 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
         Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
@@ -33,41 +33,41 @@ Certificate:
                     a7:35
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
             Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                D3:F3:19:AE:74:B1:AF:E7:AF:08:7B:16:72:78:29:87:79:ED:30:8C
-            X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-
-    Signature Algorithm: sha1WithRSAEncryption
-        0f:04:f3:91:f2:87:19:fe:9d:f8:34:5a:24:4a:00:d1:58:bf:
-        1e:b2:77:67:07:bc:78:b5:4b:9a:4b:fd:a1:e5:dc:0e:09:84:
-        9e:59:c4:dd:cf:f7:2e:bf:da:f3:31:36:6b:81:6e:a2:88:76:
-        e4:2e:0b:36:44:82:36:8f:80:93:f4:9e:fc:ed:85:d0:97:da:
-        0f:fb:c9:b9:8b:da:ae:07:3d:4f:82:b7:0c:25:22:63:12:6b:
-        0a:e9:c4:12:a4:5c:ed:11:12:cc:fe:b0:2e:d4:c1:ec:79:01:
-        60:ea:cc:cc:e5:66:cc:57:f6:55:a9:09:4c:63:01:e9:b4:2e:
-        73:a5
+         88:1d:a7:42:a1:1c:87:45:4a:e6:5e:aa:9c:7b:71:2e:5c:9e:
+         11:85:0f:a3:c5:b4:ea:73:9e:b7:61:9d:4a:e9:cd:1a:c5:2e:
+         03:be:a3:2b:b6:12:6a:15:03:04:3f:fb:4a:09:0d:84:0e:dd:
+         c0:63:2b:0f:13:fb:1f:98:64:49:48:e7:96:d5:41:c4:ca:94:
+         bf:ab:c5:ea:80:2c:ee:1f:ab:12:54:74:f1:f1:56:ea:03:c0:
+         1c:0d:8d:b9:6e:b0:d0:5f:21:c1:d3:e3:45:df:cf:64:69:13:
+         6c:54:79:06:7d:53:46:77:3c:21:cc:c4:6a:5f:f9:9a:07:0f:
+         a5:95:20:f0:0e:93:07:48:96:a9:2c:28:50:21:d7:f8:13:4f:
+         b8:ca:aa:1f:a6:41:7c:71:1f:ad:11:3f:3d:1e:e9:81:3c:86:
+         c1:af:2d:39:a0:13:9f:99:ec:9a:47:44:df:28:02:a7:1d:6a:
+         8d:c0:1e:24:e8:19:fc:1d:dc:67:29:04:be:0a:d6:c5:81:59:
+         27:2c:f5:e5:df:ba:0b:c6:50:e5:b3:bd:73:12:3e:2c:ef:a6:
+         8a:ed:eb:86:9a:45:45:52:a3:44:78:12:60:17:e2:3a:32:92:
+         03:6e:89:89:16:c5:e0:bc:be:a7:cb:93:4b:d8:56:33:a0:a0:
+         53:b2:0d:a5
 -----BEGIN CERTIFICATE-----
-MIIDLjCCApegAwIBAgIJAIgImLMT2ACXMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDIw
-NjIxWhcNMzAxMTE2MDAwMDAwWjBXMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEWMBQGA1UECxMNQXBhY2hlIFB1bHNhcjESMBAGA1UE
-AxMJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAr7+3
-LZitnfbaoxPUYg+YvhyiiSK6b9X9H2fjkQOYgIEO7dj2cH8sNmg9U+pYOqbViWZL
-vR5XcRNtSxHlQKV2hCSSQFiAlskfLMRV66N5c3BcN5qJ7S+6a+OCfGlKAlSLgV48
-v0yKy+osXoPntxAIX4JYo4nR2pK6KijuMCg/W64QcZbH4RLFsBqtRG9EOhFKmjwP
-jQaAezTvP2z0XsVEVB7I3ceAhYDZaObGUwN34f4YYQd3BUztWbxdQThq712hsmCY
-1EgolQKKDv3PexvSEcwQDFBz18w4bIPdeSaqkMibhIa8WeliafSYG8SAeH6gGoGd
-0uFm3cTM/GMErOynNQIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQf
-Fh1PcGVuU1NMIEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQU0/MZrnSx
-r+evCHsWcngph3ntMIwwHwYDVR0jBBgwFoAU1HrND0QbFiklFO2i7xMPp0YJePYw
-DQYJKoZIhvcNAQEFBQADgYEADwTzkfKHGf6d+DRaJEoA0Vi/HrJ3Zwe8eLVLmkv9
-oeXcDgmEnlnE3c/3Lr/a8zE2a4Fuooh25C4LNkSCNo+Ak/Se/O2F0JfaD/vJuYva
-rgc9T4K3DCUiYxJrCunEEqRc7RESzP6wLtTB7HkBYOrMzOVmzFf2VakJTGMB6bQu
+MIIDFDCCAfygAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgUwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVzELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYDVQQKEwZBcGFj
+aGUxFjAUBgNVBAsTDUFwYWNoZSBQdWxzYXIxEjAQBgNVBAMTCWxvY2FsaG9zdDCC
+ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAK+/ty2YrZ322qMT1GIPmL4c
+ookium/V/R9n45EDmICBDu3Y9nB/LDZoPVPqWDqm1YlmS70eV3ETbUsR5UCldoQk
+kkBYgJbJHyzEVeujeXNwXDeaie0vumvjgnxpSgJUi4FePL9MisvqLF6D57cQCF+C
+WKOJ0dqSuioo7jAoP1uuEHGWx+ESxbAarURvRDoRSpo8D40GgHs07z9s9F7FRFQe
+yN3HgIWA2WjmxlMDd+H+GGEHdwVM7Vm8XUE4au9dobJgmNRIKJUCig79z3sb0hHM
+EAxQc9fMOGyD3XkmqpDIm4SGvFnpYmn0mBvEgHh+oBqBndLhZt3EzPxjBKzspzUC
+AwEAAaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEB
+CwUAA4IBAQCIHadCoRyHRUrmXqqce3EuXJ4RhQ+jxbTqc563YZ1K6c0axS4DvqMr
+thJqFQMEP/tKCQ2EDt3AYysPE/sfmGRJSOeW1UHEypS/q8XqgCzuH6sSVHTx8Vbq
+A8AcDY25brDQXyHB0+NF389kaRNsVHkGfVNGdzwhzMRqX/maBw+llSDwDpMHSJap
+LChQIdf4E0+4yqofpkF8cR+tET89HumBPIbBry05oBOfmeyaR0TfKAKnHWqNwB4k
+6Bn8HdxnKQS+CtbFgVknLPXl37oLxlDls71zEj4s76aK7euGmkVFUqNEeBJgF+I6
+MpIDbomJFsXgvL6ny5NL2FYzoKBTsg2l
 c6U=
 -----END CERTIFICATE-----
diff --git a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/cacert.pem b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/cacert.pem
index 55e9067458a..21bbaba213f 100644
--- a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/cacert.pem
+++ b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/cacert.pem
@@ -2,61 +2,76 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:94
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            70:4c:6b:e0:aa:cc:01:77:f2:1f:04:8c:d4:72:03:a5:32:5f:c7:be
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 17 01:37:33 2018 GMT
-            Not After : Feb 16 01:37:33 2021 GMT
-        Subject: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: CN = CARoot
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (1024 bit)
-                Modulus (1024 bit):
-                    00:ea:16:8d:a5:b1:19:61:34:54:07:02:60:4e:6d:
-                    54:92:08:fd:fb:23:79:9c:05:bf:14:f7:bc:aa:db:
-                    2b:42:a4:35:74:86:e3:00:ad:8b:18:79:73:7d:f2:
-                    d1:74:dd:74:bc:b8:a2:4c:80:c9:f3:80:ce:bf:f8:
-                    6d:97:f5:05:4f:f4:b2:99:50:e8:d8:b0:c4:57:a0:
-                    e7:dc:82:57:75:2a:a2:02:21:76:f7:37:c2:dc:7c:
-                    4c:36:a6:73:6f:dc:75:48:72:ad:fa:98:02:70:b2:
-                    5e:a2:83:cc:c3:8d:20:a7:1e:bc:d7:1e:c1:d1:7e:
-                    39:35:4b:f5:be:6b:c1:0f:f9
+                RSA Public-Key: (2048 bit)
+                Modulus:
+                    00:dc:9c:01:30:5f:c5:42:48:10:78:30:5d:66:20:
+                    0e:74:61:f6:82:74:9f:6f:b2:ed:00:9e:6c:21:b6:
+                    83:21:6b:54:34:e8:a9:dc:81:83:7a:0e:9f:cc:3d:
+                    eb:97:ee:cf:ca:0e:5f:96:81:dc:e7:75:88:91:2f:
+                    d5:65:74:c2:d8:67:58:d8:41:6a:5f:a9:79:dc:29:
+                    36:4a:b8:39:20:d2:f8:a8:59:9f:e3:be:f9:61:80:
+                    1b:ce:63:bb:12:56:06:b9:77:4e:6a:40:65:9b:bf:
+                    5b:f8:27:88:f5:ff:40:ee:47:bc:2d:8e:c3:a6:62:
+                    0d:18:76:d1:f5:af:1a:6b:25:4e:d4:55:15:f0:e3:
+                    97:1b:68:eb:75:b8:80:ea:64:ef:7e:e2:f0:5c:da:
+                    6d:d6:16:7b:0f:5e:ae:72:47:5a:df:0b:8a:e0:74:
+                    c1:b7:82:0d:97:41:d7:84:16:51:40:37:15:a1:eb:
+                    70:0c:f1:5a:26:39:11:1e:97:b9:36:32:ce:16:b9:
+                    42:ad:31:5b:1e:89:f5:3e:07:0e:d6:fc:9a:46:8e:
+                    87:89:90:5c:f3:00:e4:9b:ce:7b:93:fe:9a:d8:65:
+                    ec:49:5c:e8:eb:41:3d:53:bc:ce:e8:6d:44:ec:76:
+                    3f:e6:9b:13:e4:f8:d0:1c:00:e6:4f:73:e1:b0:27:
+                    6f:99
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
             X509v3 Subject Key Identifier: 
-                D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
+                8B:30:D2:81:7C:BE:AB:4D:76:37:19:2B:69:5E:DB:F7:81:95:73:F5
             X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-                DirName:/C=US/ST=CA/O=Apache/OU=Pulsar Incubator/CN=localhost
-                serial:88:08:98:B3:13:D8:00:94
+                keyid:8B:30:D2:81:7C:BE:AB:4D:76:37:19:2B:69:5E:DB:F7:81:95:73:F5
 
-            X509v3 Basic Constraints: 
+            X509v3 Basic Constraints: critical
                 CA:TRUE
-    Signature Algorithm: sha1WithRSAEncryption
-        5e:30:c5:7b:30:3e:1e:16:cd:ba:66:f1:2a:19:13:8a:1a:00:
-        08:f4:1e:8c:e4:3d:57:13:65:96:bf:07:58:55:52:37:3e:aa:
-        2c:19:de:ee:c3:92:6e:79:f3:06:0e:9a:7b:e0:02:50:c3:ef:
-        3b:84:ea:8f:e0:f0:16:a6:a6:67:8b:be:73:0e:5d:f7:88:39:
-        d3:d4:df:85:ad:7c:c1:4f:fa:55:55:6f:c2:48:4e:8e:82:fa:
-        72:3b:8e:9d:dc:f7:2e:9d:47:8e:e5:c9:a2:ee:b1:76:94:15:
-        7c:7a:62:bc:06:45:fa:61:2e:33:8c:18:3e:e9:d5:90:a5:a6:
-        80:5a
+    Signature Algorithm: sha256WithRSAEncryption
+         02:4c:80:4f:a4:b5:f4:70:be:82:cf:3a:ed:40:f9:97:17:22:
+         07:5d:e0:9b:4e:54:f8:4b:64:99:f5:07:7f:87:5b:9c:60:ec:
+         9f:69:e6:00:97:5a:cd:14:59:31:45:be:b7:bd:c4:ce:57:82:
+         1a:4a:62:ce:8e:c8:59:d5:62:43:8b:94:c0:ab:c2:cc:3a:a0:
+         69:d3:65:15:82:35:de:85:64:e6:7b:d9:3a:22:12:77:f7:71:
+         82:86:d7:6c:e5:69:d5:3a:f2:a7:25:f7:dc:f3:6f:cb:eb:85:
+         48:44:63:e2:6d:3c:82:eb:3a:c0:e1:bd:9d:3a:12:11:66:1f:
+         05:8f:49:65:31:d6:cf:26:06:46:ba:73:c7:ad:61:fc:14:5f:
+         68:d1:ee:02:5f:4b:98:b6:5b:0c:98:4e:61:7b:cb:35:ee:44:
+         a1:ce:e1:00:a2:56:f0:0d:72:3b:58:66:e8:9a:dc:62:d5:95:
+         3e:5a:48:21:a8:7c:f8:1f:5a:13:db:53:33:11:3e:e6:14:39:
+         cd:2b:3f:77:5b:ee:f7:0c:59:69:2f:46:9a:34:56:89:05:8e:
+         40:94:94:3f:95:f6:fa:f9:1a:e8:1a:80:7b:1d:f7:0c:a1:be:
+         e2:38:98:fd:0f:e7:68:4d:7d:fe:ae:5f:e3:32:c6:5d:37:77:
+         7a:28:ce:cc
 -----BEGIN CERTIFICATE-----
-MIIC8jCCAlugAwIBAgIJAIgImLMT2ACUMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDEz
-NzMzWhcNMjEwMjE2MDEzNzMzWjBaMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEZMBcGA1UECxMQUHVsc2FyIEluY3ViYXRvcjESMBAG
-A1UEAxMJbG9jYWxob3N0MIGfMA0GCSqGSIb3DQEBAQUAA4GNADCBiQKBgQDqFo2l
-sRlhNFQHAmBObVSSCP37I3mcBb8U97yq2ytCpDV0huMArYsYeXN98tF03XS8uKJM
-gMnzgM6/+G2X9QVP9LKZUOjYsMRXoOfcgld1KqICIXb3N8LcfEw2pnNv3HVIcq36
-mAJwsl6ig8zDjSCnHrzXHsHRfjk1S/W+a8EP+QIDAQABo4G/MIG8MB0GA1UdDgQW
-BBTUes0PRBsWKSUU7aLvEw+nRgl49jCBjAYDVR0jBIGEMIGBgBTUes0PRBsWKSUU
-7aLvEw+nRgl49qFepFwwWjELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYD
-VQQKEwZBcGFjaGUxGTAXBgNVBAsTEFB1bHNhciBJbmN1YmF0b3IxEjAQBgNVBAMT
-CWxvY2FsaG9zdIIJAIgImLMT2ACUMAwGA1UdEwQFMAMBAf8wDQYJKoZIhvcNAQEF
-BQADgYEAXjDFezA+HhbNumbxKhkTihoACPQejOQ9VxNllr8HWFVSNz6qLBne7sOS
-bnnzBg6ae+ACUMPvO4Tqj+DwFqamZ4u+cw5d94g509Tfha18wU/6VVVvwkhOjoL6
-cjuOndz3Lp1HjuXJou6xdpQVfHpivAZF+mEuM4wYPunVkKWmgFo=
+MIIDAzCCAeugAwIBAgIUcExr4KrMAXfyHwSM1HIDpTJfx74wDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowETEPMA0GA1UEAwwGQ0FSb290MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A
+MIIBCgKCAQEA3JwBMF/FQkgQeDBdZiAOdGH2gnSfb7LtAJ5sIbaDIWtUNOip3IGD
+eg6fzD3rl+7Pyg5floHc53WIkS/VZXTC2GdY2EFqX6l53Ck2Srg5INL4qFmf4775
+YYAbzmO7ElYGuXdOakBlm79b+CeI9f9A7ke8LY7DpmINGHbR9a8aayVO1FUV8OOX
+G2jrdbiA6mTvfuLwXNpt1hZ7D16uckda3wuK4HTBt4INl0HXhBZRQDcVoetwDPFa
+JjkRHpe5NjLOFrlCrTFbHon1PgcO1vyaRo6HiZBc8wDkm857k/6a2GXsSVzo60E9
+U7zO6G1E7HY/5psT5PjQHADmT3PhsCdvmQIDAQABo1MwUTAdBgNVHQ4EFgQUizDS
+gXy+q012NxkraV7b94GVc/UwHwYDVR0jBBgwFoAUizDSgXy+q012NxkraV7b94GV
+c/UwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAAkyAT6S19HC+
+gs867UD5lxciB13gm05U+EtkmfUHf4dbnGDsn2nmAJdazRRZMUW+t73EzleCGkpi
+zo7IWdViQ4uUwKvCzDqgadNlFYI13oVk5nvZOiISd/dxgobXbOVp1TrypyX33PNv
+y+uFSERj4m08gus6wOG9nToSEWYfBY9JZTHWzyYGRrpzx61h/BRfaNHuAl9LmLZb
+DJhOYXvLNe5Eoc7hAKJW8A1yO1hm6JrcYtWVPlpIIah8+B9aE9tTMxE+5hQ5zSs/
+d1vu9wxZaS9GmjRWiQWOQJSUP5X2+vka6BqAex33DKG+4jiY/Q/naE19/q5f4zLG
+XTd3eijOzA==
 -----END CERTIFICATE-----
diff --git a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-cert.pem b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-cert.pem
index 61847f2e612..e5d9e6e74b2 100644
--- a/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-cert.pem
+++ b/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:99
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:06
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 17 02:50:05 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=superUser
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache, OU = Apache Pulsar, CN = superUser
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:cd:43:7d:98:40:f9:b0:5b:bc:ae:db:c0:0b:ad:
                     26:90:96:e0:62:38:ed:68:b1:70:46:3b:de:44:f9:
                     14:51:86:10:eb:ca:90:e7:88:e8:f9:91:85:e0:dd:
@@ -33,41 +33,40 @@ Certificate:
                     e1:0b
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                53:7C:D5:D1:52:97:9A:D6:D5:EA:EC:B6:0C:9B:43:39:19:73:F6:2C
-            X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-
-    Signature Algorithm: sha1WithRSAEncryption
-        e4:03:82:ff:be:df:7c:73:2a:c5:8f:7d:87:ab:95:b1:2b:e5:
-        f7:41:22:4f:28:54:84:7a:cc:fe:70:89:0f:48:e5:8a:17:e1:
-        44:ad:12:e9:a1:3a:c7:84:55:f0:7c:29:52:0a:a1:ab:cc:5b:
-        31:e5:b2:37:73:3a:8d:f2:f1:fb:e8:f6:a2:b9:ef:11:10:f8:
-        31:43:8f:af:ce:09:f4:cb:96:0e:d4:58:42:6e:86:ab:b9:03:
-        19:8b:4a:6e:ef:50:c0:7e:c9:0b:1d:2b:42:bf:eb:d0:06:05:
-        84:ea:5a:8a:22:5c:56:fa:da:2a:9f:8a:b2:90:66:8c:5e:01:
-        87:45
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         90:62:ba:7b:6f:45:95:7a:71:2f:e7:88:0c:64:b8:6c:05:86:
+         7f:47:08:ce:d6:e2:5a:32:13:0c:82:ad:a7:af:f0:a2:f7:86:
+         79:87:1a:89:78:95:b1:9f:be:c5:8b:39:fd:12:94:b6:e1:69:
+         ff:fa:1e:c3:82:d8:6c:03:80:45:ac:1c:06:70:bb:77:c3:41:
+         5f:b6:9d:fe:36:6f:ae:23:6c:bf:43:79:8e:74:85:8e:96:89:
+         a9:c4:6d:d9:fa:05:ba:a8:11:7c:82:45:94:3d:9f:b6:7c:2f:
+         4e:6d:37:c3:fb:79:7e:0c:d2:15:fa:0e:ea:2d:c9:24:f3:34:
+         13:6f:db:d7:55:e1:0c:2f:7e:fe:4c:3b:fa:7e:03:26:0f:6a:
+         95:d2:22:ce:27:71:6a:97:ac:36:0a:20:ec:19:a0:78:23:0c:
+         54:f3:b1:dd:33:36:7c:b7:61:23:70:8f:7f:c8:5f:e8:9e:b5:
+         02:31:4d:b3:40:b0:7b:b2:ee:14:a7:69:22:8b:38:85:5d:04:
+         6e:d5:44:41:31:a7:4b:71:86:fb:81:cd:3d:db:96:23:0b:bc:
+         e1:67:46:0e:87:86:91:4e:1a:35:37:af:a4:ac:9a:de:e3:4f:
+         82:47:f1:c4:16:58:11:8f:76:d2:4d:df:a1:c6:a2:8f:33:6d:
+         72:15:28:76
 -----BEGIN CERTIFICATE-----
-MIIDLjCCApegAwIBAgIJAIgImLMT2ACZMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDI1
-MDA1WhcNMzAxMTE2MDAwMDAwWjBXMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEWMBQGA1UECxMNQXBhY2hlIFB1bHNhcjESMBAGA1UE
-AxMJc3VwZXJVc2VyMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAzUN9
-mED5sFu8rtvAC60mkJbgYjjtaLFwRjveRPkUUYYQ68qQ54jo+ZGF4N21tBS5eOOG
-1VRtaOwUkrT4IlsFPe0xJWUIBYTK5gwhElgyxxpgo0/SSp4oGXxFhACMidzeiuVP
-iJHMpPGBRUx9wv/iwYnGEnOV4ja9266LWmhqkFHeK4hfqmf0qONj3L4Zgsydf+aN
-+4K+IgE9VhM7WwS06MUY5i4N+rpKjejGWqFRmkpi16/dtPzi1c2umWxcYVYL1wwa
-d1z1OmpUtZ4zrKl1KJp2r9B6VwAbkRMx/UKIIUcFEAEvWbvHOtnhWEwbbHG2mO/d
-A4JYozLckKG2ph7hCwIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQf
-Fh1PcGVuU1NMIEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQUU3zV0VKX
-mtbV6uy2DJtDORlz9iwwHwYDVR0jBBgwFoAU1HrND0QbFiklFO2i7xMPp0YJePYw
-DQYJKoZIhvcNAQEFBQADgYEA5AOC/77ffHMqxY99h6uVsSvl90EiTyhUhHrM/nCJ
-D0jlihfhRK0S6aE6x4RV8HwpUgqhq8xbMeWyN3M6jfLx++j2ornvERD4MUOPr84J
-9MuWDtRYQm6Gq7kDGYtKbu9QwH7JCx0rQr/r0AYFhOpaiiJcVvraKp+KspBmjF4B
-h0U=
+MIIDFDCCAfygAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgYwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVzELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYDVQQKEwZBcGFj
+aGUxFjAUBgNVBAsTDUFwYWNoZSBQdWxzYXIxEjAQBgNVBAMTCXN1cGVyVXNlcjCC
+ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAM1DfZhA+bBbvK7bwAutJpCW
+4GI47WixcEY73kT5FFGGEOvKkOeI6PmRheDdtbQUuXjjhtVUbWjsFJK0+CJbBT3t
+MSVlCAWEyuYMIRJYMscaYKNP0kqeKBl8RYQAjInc3orlT4iRzKTxgUVMfcL/4sGJ
+xhJzleI2vduui1poapBR3iuIX6pn9KjjY9y+GYLMnX/mjfuCviIBPVYTO1sEtOjF
+GOYuDfq6So3oxlqhUZpKYtev3bT84tXNrplsXGFWC9cMGndc9TpqVLWeM6ypdSia
+dq/QelcAG5ETMf1CiCFHBRABL1m7xzrZ4VhMG2xxtpjv3QOCWKMy3JChtqYe4QsC
+AwEAAaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEB
+CwUAA4IBAQCQYrp7b0WVenEv54gMZLhsBYZ/RwjO1uJaMhMMgq2nr/Ci94Z5hxqJ
+eJWxn77Fizn9EpS24Wn/+h7DgthsA4BFrBwGcLt3w0Fftp3+Nm+uI2y/Q3mOdIWO
+lompxG3Z+gW6qBF8gkWUPZ+2fC9ObTfD+3l+DNIV+g7qLckk8zQTb9vXVeEML37+
+TDv6fgMmD2qV0iLOJ3Fql6w2CiDsGaB4IwxU87HdMzZ8t2EjcI9/yF/onrUCMU2z
+QLB7su4Up2kiiziFXQRu1URBMadLcYb7gc0925YjC7zhZ0YOh4aRTho1N6+krJre
+40+CR/HEFlgRj3bSTd+hxqKPM21yFSh2
 -----END CERTIFICATE-----
diff --git a/build/regenerate_certs_for_tests.sh b/build/regenerate_certs_for_tests.sh
new file mode 100755
index 00000000000..fb0274cc193
--- /dev/null
+++ b/build/regenerate_certs_for_tests.sh
@@ -0,0 +1,70 @@
+#!/bin/bash -xe
+#
+# 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.
+#
+
+ROOT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")"/.. >/dev/null 2>&1 && pwd)"
+
+cd /tmp
+mkdir keygendir$$
+cd keygendir$$
+
+# create CA key and cert
+function generate_ca() {
+  openssl req -x509 -nodes -newkey rsa:2048 -keyout ca-key -outform pem -text -out ca-cert.pem -days 3650 -sha256 \
+    -subj "/CN=CARoot" -extensions v3_ca
+}
+
+function reissue_certificate() {
+  keyfile=$1
+  certfile=$2
+  openssl x509 -x509toreq -in $certfile -signkey $keyfile -out ${certfile}.csr
+  openssl x509 -req -CA ca-cert.pem -CAkey ca-key -in ${certfile}.csr -text -outform pem -out $certfile -days 3650 -CAcreateserial -extfile <(printf "subjectAltName = DNS:localhost, IP:127.0.0.1")
+}
+
+generate_ca
+cp ca-cert.pem $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/cacert.pem
+reissue_certificate $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/client-key.pem \
+  $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/client-cert.pem
+reissue_certificate $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/server-key.pem \
+  $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/server-cert.pem
+
+# use same CA key and cert for ProxyWithAuthorizationTest/client-cacert.pem
+cp ca-cert.pem $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cacert.pem
+reissue_certificate $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-key.pem \
+  $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cert.pem
+
+# use same CA key and cert for ProxyWithAuthorizationTest/proxy-cacert.pem
+cp ca-cert.pem $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem
+reissue_certificate $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-key.pem \
+  $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem
+
+generate_ca
+cp ca-cert.pem $ROOT_DIR/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/cacert.pem
+reissue_certificate $ROOT_DIR/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-key.pem \
+  $ROOT_DIR/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/broker-cert.pem
+reissue_certificate $ROOT_DIR/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-key.pem \
+  $ROOT_DIR/bouncy-castle/bcfips-include-test/src/test/resources/authentication/tls/client-cert.pem
+
+generate_ca
+cp ca-cert.pem $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cacert.pem
+reissue_certificate $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-key.pem \
+  $ROOT_DIR/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cert.pem
+
+cd $ROOT_DIR
+rm -rf /tmp/keygendir$$
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java
index c2c4256e558..d6af356401c 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java
@@ -18,8 +18,7 @@
  */
 package org.apache.pulsar.client.api;
 
-import static org.mockito.Mockito.spy;
-
+import com.google.common.collect.Sets;
 import java.lang.reflect.Method;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -27,15 +26,11 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.pulsar.broker.authentication.AuthenticationProviderBasic;
 import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
-import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.impl.auth.AuthenticationTls;
 import org.apache.pulsar.client.impl.tls.PublicSuffixMatcher;
 import org.apache.pulsar.client.impl.tls.TlsHostnameVerifier;
-import org.apache.pulsar.common.policies.data.ClusterData;
-import org.apache.pulsar.common.policies.data.TenantInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -43,8 +38,7 @@ import org.testng.annotations.AfterMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
-import com.google.common.collect.Sets;
-
+@Test(groups = "broker-api")
 public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerBase {
     private static final Logger log = LoggerFactory.getLogger(AuthenticationTlsHostnameVerificationTest.class);
 
@@ -63,8 +57,15 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
     private final String BASIC_CONF_FILE_PATH = "./src/test/resources/authentication/basic/.htpasswd";
 
     private boolean hostnameVerificationEnabled = true;
+    private String clientTrustCertFilePath = TLS_TRUST_CERT_FILE_PATH;
 
     protected void setup() throws Exception {
+        conf.setAuthenticationEnabled(false);
+        conf.setAuthorizationEnabled(false);
+        super.internalSetup();
+        super.producerBaseSetup();
+        super.stopBroker();
+
         if (methodName.equals("testAnonymousSyncProducerAndConsumer")) {
             conf.setAnonymousUserRole("anonymousUser");
         }
@@ -72,7 +73,7 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
         conf.setAuthenticationEnabled(true);
         conf.setAuthorizationEnabled(true);
 
-        conf.setTlsAllowInsecureConnection(true);
+        conf.setTlsAllowInsecureConnection(false);
 
         Set<String> superUserRoles = new HashSet<>();
         superUserRoles.add("localhost");
@@ -83,7 +84,7 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
 
         conf.setBrokerClientAuthenticationPlugin(AuthenticationTls.class.getName());
         conf.setBrokerClientAuthenticationParameters(
-                "tlsCertFile:" + TLS_CLIENT_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_SERVER_KEY_FILE_PATH);
+            "tlsCertFile:" + TLS_CLIENT_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_SERVER_KEY_FILE_PATH);
 
         Set<String> providers = new HashSet<>();
         providers.add(AuthenticationProviderTls.class.getName());
@@ -94,7 +95,7 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
         conf.setClusterName("test");
         conf.setNumExecutorThreadPoolSize(5);
 
-        super.init();
+        startBroker();
 
         setupClient();
     }
@@ -107,21 +108,13 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
         Authentication authTls = new AuthenticationTls();
         authTls.configure(authParams);
 
-        admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString())
-                .tlsTrustCertsFilePath(TLS_MIM_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(true)
-                .authentication(authTls).build());
+        pulsarClient.close();
         pulsarClient = PulsarClient.builder()
-                .serviceUrl(pulsar.getBrokerServiceUrlTls())
-                .statsInterval(0, TimeUnit.SECONDS)
-                .tlsTrustCertsFilePath(TLS_MIM_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(true)
-                .authentication(authTls).enableTls(true).enableTlsHostnameVerification(hostnameVerificationEnabled)
-                .build();
-
-        admin.clusters().createCluster("test", new ClusterData(brokerUrl.toString()));
-
-        admin.tenants().createTenant("my-property",
-                new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test")));
-        admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test"));
+            .serviceUrl(pulsar.getBrokerServiceUrlTls())
+            .statsInterval(0, TimeUnit.SECONDS)
+            .tlsTrustCertsFilePath(clientTrustCertFilePath)
+            .authentication(authTls).enableTls(true).enableTlsHostnameVerification(hostnameVerificationEnabled)
+            .build();
     }
 
     @AfterMethod(alwaysRun = true)
@@ -150,24 +143,25 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
      */
     @Test(dataProvider = "hostnameVerification")
     public void testTlsSyncProducerAndConsumerWithInvalidBrokerHost(boolean hostnameVerificationEnabled)
-            throws Exception {
+        throws Exception {
         log.info("-- Starting {} test --", methodName);
 
         this.hostnameVerificationEnabled = hostnameVerificationEnabled;
+        clientTrustCertFilePath = TLS_MIM_TRUST_CERT_FILE_PATH;
         // setup broker cert which has CN = "pulsar" different than broker's hostname="localhost"
         conf.setBrokerServicePortTls(Optional.of(0));
         conf.setWebServicePortTls(Optional.of(0));
-        conf.setTlsTrustCertsFilePath(TLS_MIM_TRUST_CERT_FILE_PATH);
+        conf.setTlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH);
         conf.setTlsCertificateFilePath(TLS_MIM_SERVER_CERT_FILE_PATH);
         conf.setTlsKeyFilePath(TLS_MIM_SERVER_KEY_FILE_PATH);
         conf.setBrokerClientAuthenticationParameters(
-                "tlsCertFile:" + TLS_CLIENT_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_MIM_SERVER_KEY_FILE_PATH);
+            "tlsCertFile:" + TLS_CLIENT_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_MIM_SERVER_KEY_FILE_PATH);
 
         setup();
 
         try {
             pulsarClient.newConsumer().topic("persistent://my-property/my-ns/my-topic")
-                    .subscriptionName("my-subscriber-name").subscribe();
+                .subscriptionName("my-subscriber-name").subscribe();
             if (hostnameVerificationEnabled) {
                 Assert.fail("Connection should be failed due to hostnameVerification enabled");
             }
@@ -205,10 +199,10 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
         setup();
 
         Consumer<byte[]> consumer = pulsarClient.newConsumer().topic("persistent://my-property/my-ns/my-topic")
-                .subscriptionName("my-subscriber-name").subscribe();
+            .subscriptionName("my-subscriber-name").subscribe();
 
         Producer<byte[]> producer = pulsarClient.newProducer().topic("persistent://my-property/my-ns/my-topic")
-                .create();
+            .create();
         for (int i = 0; i < 10; i++) {
             String message = "my-message-" + i;
             producer.send(message.getBytes());
@@ -239,7 +233,7 @@ public class AuthenticationTlsHostnameVerificationTest extends ProducerConsumerB
     public void testDefaultHostVerifier() throws Exception {
         log.info("-- Starting {} test --", methodName);
         Method matchIdentityStrict = TlsHostnameVerifier.class.getDeclaredMethod("matchIdentityStrict",
-                String.class, String.class, PublicSuffixMatcher.class);
+            String.class, String.class, PublicSuffixMatcher.class);
         matchIdentityStrict.setAccessible(true);
         Assert.assertTrue((boolean) matchIdentityStrict.invoke(null, "pulsar", "pulsar", null));
         Assert.assertFalse((boolean) matchIdentityStrict.invoke(null, "pulsar.com", "pulsar", null));
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java
index 97b79376be4..6bf03415c6a 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java
@@ -127,7 +127,7 @@ public class AsyncHttpConnector implements Connector {
                             params != null ? params.getKeyStoreType() : null,
                             params != null ? params.getKeyStorePath() : null,
                             params != null ? params.getKeyStorePassword() : null,
-                            conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                            conf.isTlsAllowInsecureConnection(),
                             conf.getTlsTrustStoreType(),
                             conf.getTlsTrustStorePath(),
                             conf.getTlsTrustStorePassword(),
@@ -141,21 +141,22 @@ public class AsyncHttpConnector implements Connector {
                     if (authData.hasDataForTls()) {
                         sslCtx = authData.getTlsTrustStoreStream() == null
                                 ? SecurityUtility.createNettySslContextForClient(
-                                        conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                                        conf.isTlsAllowInsecureConnection(),
                                         conf.getTlsTrustCertsFilePath(), authData.getTlsCertificates(),
                                         authData.getTlsPrivateKey())
                                 : SecurityUtility.createNettySslContextForClient(
-                                        conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                                        conf.isTlsAllowInsecureConnection(),
                                         authData.getTlsTrustStoreStream(), authData.getTlsCertificates(),
                                         authData.getTlsPrivateKey());
                     } else {
                         sslCtx = SecurityUtility.createNettySslContextForClient(
-                                conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                                conf.isTlsAllowInsecureConnection(),
                                 conf.getTlsTrustCertsFilePath());
                     }
                     confBuilder.setSslContext(sslCtx);
                 }
             }
+            confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable());
         }
         httpClient = new DefaultAsyncHttpClient(confBuilder.build());
         this.readTimeout = readTimeoutMs;
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
index 8c1f8cc045b..796f626209b 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java
@@ -26,12 +26,10 @@ import com.google.common.collect.Queues;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.unix.Errors.NativeIoException;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.ssl.SslHandler;
 import io.netty.util.concurrent.Promise;
 
 import java.net.InetSocketAddress;
@@ -48,7 +46,6 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
-import javax.net.ssl.SSLSession;
 
 import lombok.Getter;
 import org.apache.commons.lang3.exception.ExceptionUtils;
@@ -132,9 +129,6 @@ public class ClientCnx extends PulsarHandler {
     protected String proxyToTargetBrokerAddress = null;
     // Remote hostName with which client is connected
     protected String remoteHostName = null;
-    private boolean isTlsHostnameVerificationEnable;
-
-    private static final TlsHostnameVerifier HOSTNAME_VERIFIER = new TlsHostnameVerifier();
 
     private ScheduledFuture<?> timeoutTask;
 
@@ -191,7 +185,6 @@ public class ClientCnx extends PulsarHandler {
         this.maxNumberOfRejectedRequestPerConnection = conf.getMaxNumberOfRejectedRequestPerConnection();
         this.operationTimeoutMs = conf.getOperationTimeoutMs();
         this.state = State.None;
-        this.isTlsHostnameVerificationEnable = conf.isTlsHostnameVerificationEnable();
         this.protocolVersion = protocolVersion;
     }
 
@@ -288,14 +281,6 @@ public class ClientCnx extends PulsarHandler {
 
     @Override
     protected void handleConnected(CommandConnected connected) {
-
-        if (isTlsHostnameVerificationEnable && 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;
-        }
-
         checkArgument(state == State.SentConnectFrame || state == State.Connecting);
         if (connected.hasMaxMessageSize()) {
             if (log.isDebugEnabled()) {
@@ -949,39 +934,6 @@ public class ClientCnx extends PulsarHandler {
         }
     }
 
-    /**
-     * verifies host name provided in x509 Certificate in tls session
-     *
-     * it matches hostname with below scenarios
-     *
-     * <pre>
-     *  1. Supports IPV4 and IPV6 host matching
-     *  2. Supports wild card matching for DNS-name
-     *  eg:
-     *     HostName                     CN           Result
-     * 1.  localhost                    localhost    PASS
-     * 2.  localhost                    local*       PASS
-     * 3.  pulsar1-broker.com           pulsar*.com  PASS
-     * </pre>
-     *
-     * @param ctx
-     * @return true if hostname is verified else return false
-     */
-    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();
-            if (log.isDebugEnabled()) {
-                log.debug("Verifying HostName for {}, Cipher {}, Protocols {}", hostname, sslSession.getCipherSuite(),
-                        sslSession.getProtocol());
-            }
-            return HOSTNAME_VERIFIER.verify(hostname, sslSession);
-        }
-        return false;
-    }
-
     void registerConsumer(final long consumerId, final ConsumerImpl<?> consumer) {
         consumers.put(consumerId, consumer);
     }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java
index 1d3839c3bce..cf2b3bd8b3a 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java
@@ -130,6 +130,7 @@ public class HttpClient implements Closeable {
                 }
 
                 confBuilder.setUseInsecureTrustManager(conf.isTlsAllowInsecureConnection());
+                confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable());
             } catch (Exception e) {
                 throw new PulsarClientException.InvalidConfigurationException(e);
             }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
index e9a8bcd8e5d..10a9e9c74e9 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
@@ -49,6 +49,7 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
     private final Supplier<ClientCnx> clientCnxSupplier;
     @Getter
     private final boolean tlsEnabled;
+    private final boolean tlsHostnameVerificationEnabled;
     private final boolean tlsEnabledWithKeyStore;
 
     private final Supplier<SslContext> sslContextSupplier;
@@ -61,6 +62,7 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
         super();
         this.clientCnxSupplier = clientCnxSupplier;
         this.tlsEnabled = conf.isUseTls();
+        this.tlsHostnameVerificationEnabled = conf.isTlsHostnameVerificationEnable();
         this.tlsEnabledWithKeyStore = conf.isUseKeyStoreTls();
 
         if (tlsEnabled) {
@@ -138,6 +140,11 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
                         ? new SslHandler(nettySSLContextAutoRefreshBuilder.get()
                                 .createSSLEngine(sniHost.getHostString(), sniHost.getPort()))
                         : sslContextSupplier.get().newHandler(ch.alloc(), sniHost.getHostString(), sniHost.getPort());
+
+                if (tlsHostnameVerificationEnabled) {
+                    SecurityUtility.configureSSLHandler(handler);
+                }
+
                 ch.pipeline().addFirst(TLS_HANDLER, handler);
                 initTlsFuture.complete(ch);
             } catch (Throwable t) {
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
index 35919c9d29f..e40d8a73c6e 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
@@ -42,8 +42,7 @@ public class NettyClientSslContextRefresher extends SslContextAutoRefreshBuilder
     public NettyClientSslContextRefresher(boolean allowInsecure,
                                           String trustCertsFilePath,
                                           AuthenticationDataProvider authData,
-                                          long delayInSeconds)
-            throws IOException, GeneralSecurityException {
+                                          long delayInSeconds) {
         super(delayInSeconds);
         this.tlsAllowInsecureConnection = allowInsecure;
         this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
index aba2f929f1c..f7ad9a480ec 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
@@ -21,6 +21,7 @@ package org.apache.pulsar.common.util;
 import io.netty.handler.ssl.ClientAuth;
 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 java.io.BufferedReader;
 import java.io.File;
@@ -52,7 +53,9 @@ import java.util.Set;
 import javax.net.ssl.KeyManager;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
 import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLParameters;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 import lombok.extern.slf4j.Slf4j;
@@ -374,6 +377,13 @@ public class SecurityUtility {
         }
     }
 
+    public static void configureSSLHandler(SslHandler handler) {
+        SSLEngine sslEngine = handler.engine();
+        SSLParameters sslParameters = sslEngine.getSSLParameters();
+        sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
+        sslEngine.setSSLParameters(sslParameters);
+    }
+
     public static SslContextFactory createSslContextFactory(boolean tlsAllowInsecureConnection,
             String tlsTrustCertsFilePath, String tlsCertificateFilePath, String tlsKeyFilePath,
             boolean tlsRequireTrustedClientCertOnConnect, boolean autoRefresh, long certRefreshInSec)
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
index 28a314019ec..b660ae7dbd3 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java
@@ -254,10 +254,11 @@ class AdminProxyHandler extends ProxyServlet {
                         );
                     }
 
-
-                    SslContextFactory contextFactory = new SslContextFactory.Client(true);
+                    SslContextFactory contextFactory = new SslContextFactory.Client();
                     contextFactory.setSslContext(sslCtx);
-
+                    if (!config.isTlsHostnameVerificationEnabled()) {
+                        contextFactory.setEndpointIdentificationAlgorithm(null);
+                    }
                     return new JettyHttpClient(contextFactory);
                 } catch (Exception e) {
                     try {
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 64ce8c68b27..e80d6d61249 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
@@ -21,13 +21,13 @@ package org.apache.pulsar.proxy.server;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
+import static org.apache.commons.lang3.StringUtils.isEmpty;
 import io.netty.bootstrap.Bootstrap;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
@@ -37,20 +37,19 @@ import io.netty.handler.codec.haproxy.HAProxyCommand;
 import io.netty.handler.codec.haproxy.HAProxyMessage;
 import io.netty.handler.codec.haproxy.HAProxyProtocolVersion;
 import io.netty.handler.codec.haproxy.HAProxyProxiedProtocol;
+import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.handler.timeout.ReadTimeoutHandler;
 import io.netty.util.CharsetUtil;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-import javax.net.ssl.SSLSession;
 import lombok.Getter;
 import org.apache.pulsar.PulsarVersion;
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.AuthenticationFactory;
 import org.apache.pulsar.client.api.PulsarClientException;
-import org.apache.pulsar.client.impl.tls.TlsHostnameVerifier;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.apache.pulsar.common.api.AuthData;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge;
@@ -58,6 +57,10 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.common.protocol.PulsarDecoder;
 import org.apache.pulsar.common.stats.Rate;
+import org.apache.pulsar.common.util.NettyClientSslContextRefresher;
+import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,6 +82,11 @@ public class DirectProxyHandler {
     private AuthenticationDataProvider authenticationDataProvider;
     private final ProxyService service;
     private final Runnable onHandshakeCompleteAction;
+    private final boolean tlsHostnameVerificationEnabled;
+    private final boolean tlsEnabledWithKeyStore;
+    private final boolean tlsEnabledWithBroker;
+    private final SslContextAutoRefreshBuilder<SslContext> clientSslCtxRefresher;
+    private final NettySSLContextAutoRefreshBuilder clientSSLContextAutoRefreshBuilder;
 
     public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) {
         this.service = service;
@@ -89,11 +97,52 @@ public class DirectProxyHandler {
         this.originalPrincipal = proxyConnection.clientAuthRole;
         this.clientAuthData = proxyConnection.clientAuthData;
         this.clientAuthMethod = proxyConnection.clientAuthMethod;
+        this.tlsEnabledWithBroker = service.getConfiguration().isTlsEnabledWithBroker();
+        this.tlsHostnameVerificationEnabled = service.getConfiguration().isTlsHostnameVerificationEnabled();
+        this.tlsEnabledWithKeyStore = service.getConfiguration().isTlsEnabledWithKeyStore();
         this.onHandshakeCompleteAction = proxyConnection::cancelKeepAliveTask;
+        ProxyConfiguration config = service.getConfiguration();
+
+        if (tlsEnabledWithBroker) {
+            AuthenticationDataProvider authData = null;
+
+            if (!isEmpty(config.getBrokerClientAuthenticationPlugin())) {
+                try {
+                    authData = AuthenticationFactory.create(config.getBrokerClientAuthenticationPlugin(),
+                            config.getBrokerClientAuthenticationParameters()).getAuthData();
+                } catch (PulsarClientException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+
+            if (tlsEnabledWithKeyStore) {
+                clientSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                        config.getBrokerClientSslProvider(),
+                        config.isTlsAllowInsecureConnection(),
+                        config.getBrokerClientTlsTrustStoreType(),
+                        config.getBrokerClientTlsTrustStore(),
+                        config.getBrokerClientTlsTrustStorePassword(),
+                        config.getBrokerClientTlsCiphers(),
+                        config.getBrokerClientTlsProtocols(),
+                        config.getTlsCertRefreshCheckDurationSec(),
+                        authData);
+                clientSslCtxRefresher = null;
+            } else {
+                clientSslCtxRefresher = new NettyClientSslContextRefresher(
+                        config.isTlsAllowInsecureConnection(),
+                        config.getBrokerClientTrustCertsFilePath(),
+                        authData,
+                        config.getTlsCertRefreshCheckDurationSec()
+                );
+                clientSSLContextAutoRefreshBuilder = null;
+            }
+        } else {
+            clientSSLContextAutoRefreshBuilder = null;
+            clientSslCtxRefresher = null;
+        }
     }
 
-    public void connect(String brokerHostAndPort, InetSocketAddress targetBrokerAddress,
-                           int protocolVersion, Supplier<SslHandler> sslHandlerSupplier) {
+    public void connect(String brokerHostAndPort, InetSocketAddress targetBrokerAddress, int protocolVersion) {
         ProxyConfiguration config = service.getConfiguration();
 
         // Start the connection attempt.
@@ -121,8 +170,16 @@ public class DirectProxyHandler {
         b.handler(new ChannelInitializer<SocketChannel>() {
             @Override
             protected void initChannel(SocketChannel ch) {
-                if (sslHandlerSupplier != null) {
-                    ch.pipeline().addLast(TLS_HANDLER, sslHandlerSupplier.get());
+                if (tlsEnabledWithBroker) {
+                    String host = targetBrokerAddress.getHostString();
+                    int port = targetBrokerAddress.getPort();
+                    SslHandler handler = tlsEnabledWithKeyStore
+                            ? new SslHandler(clientSSLContextAutoRefreshBuilder.get().createSSLEngine(host, port))
+                            : clientSslCtxRefresher.get().newHandler(ch.alloc(), host, port);
+                    if (tlsHostnameVerificationEnabled) {
+                        SecurityUtility.configureSSLHandler(handler);
+                    }
+                    ch.pipeline().addLast(TLS_HANDLER, handler);
                 }
                 int brokerProxyReadTimeoutMs = service.getConfiguration().getBrokerProxyReadTimeoutMs();
                 if (brokerProxyReadTimeoutMs > 0) {
@@ -338,15 +395,6 @@ public class DirectProxyHandler {
                 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;
 
             onHandshakeCompleteAction.run();
@@ -409,17 +457,6 @@ public class DirectProxyHandler {
             log.warn("[{}] [{}] Caught exception: {}", inboundChannel, outboundChannel, cause.getMessage(), cause);
             ctx.close();
         }
-
-        private boolean verifyTlsHostName(String hostname, ChannelHandlerContext ctx) {
-            ChannelHandler sslHandler = ctx.channel().pipeline().get("tls");
-
-            SSLSession sslSession;
-            if (sslHandler != null) {
-                sslSession = ((SslHandler) sslHandler).engine().getSession();
-                return (new TlsHostnameVerifier()).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/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index b7b6e359a11..bdfb11a9f0e 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -86,7 +86,6 @@ public class ProxyConnection extends PulsarHandler {
     private Authentication clientAuthentication;
     AuthenticationDataSource authenticationData;
     private State state;
-    private final Supplier<SslHandler> sslHandlerSupplier;
 
     private LookupProxyHandler lookupProxyHandler = null;
     @Getter
@@ -133,13 +132,11 @@ public class ProxyConnection extends PulsarHandler {
         return connectionPool;
     }
 
-    public ProxyConnection(ProxyService proxyService, Supplier<SslHandler> sslHandlerSupplier,
-                           DnsAddressResolverGroup dnsAddressResolverGroup) {
+    public ProxyConnection(ProxyService proxyService, DnsAddressResolverGroup dnsAddressResolverGroup) {
         super(30, TimeUnit.SECONDS);
         this.service = proxyService;
         this.dnsAddressResolverGroup = dnsAddressResolverGroup;
         this.state = State.Init;
-        this.sslHandlerSupplier = sslHandlerSupplier;
         this.brokerProxyValidator = service.getBrokerProxyValidator();
     }
 
@@ -362,8 +359,7 @@ public class ProxyConnection extends PulsarHandler {
     private void connectToBroker(InetSocketAddress brokerAddress) {
         checkState(ctx.executor().inEventLoop(), "This method should be called in the event loop");
         DirectProxyHandler directProxyHandler = new DirectProxyHandler(service, this);
-        directProxyHandler.connect(proxyToBrokerUrl, brokerAddress,
-                protocolVersionToAdvertise, sslHandlerSupplier);
+        directProxyHandler.connect(proxyToBrokerUrl, brokerAddress, protocolVersionToAdvertise);
     }
 
     public void brokerConnected(DirectProxyHandler directProxyHandler, CommandConnected connected) {
@@ -538,6 +534,7 @@ public class ProxyConnection extends PulsarHandler {
         }
         if (proxyConfig.isTlsEnabledWithBroker()) {
             clientConf.setUseTls(true);
+            clientConf.setTlsHostnameVerificationEnable(proxyConfig.isTlsHostnameVerificationEnabled());
             if (proxyConfig.isBrokerClientTlsEnabledWithKeyStore()) {
                 clientConf.setUseKeyStoreTls(true);
                 clientConf.setTlsTrustStoreType(proxyConfig.getBrokerClientTlsTrustStoreType());
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
index a8f2287ffea..63506033be8 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
@@ -18,17 +18,12 @@
  */
 package org.apache.pulsar.proxy.server;
 
-import static org.apache.commons.lang3.StringUtils.isEmpty;
 
 import io.netty.handler.ssl.SslHandler;
 import io.netty.handler.timeout.ReadTimeoutHandler;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-import org.apache.pulsar.client.api.AuthenticationDataProvider;
-import org.apache.pulsar.client.api.AuthenticationFactory;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.common.protocol.OptionalProxyProtocolDecoder;
-import org.apache.pulsar.common.util.NettyClientSslContextRefresher;
 import org.apache.pulsar.common.util.NettyServerSslContextBuilder;
 
 import io.netty.channel.ChannelInitializer;
@@ -51,9 +46,7 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
     private final int brokerProxyReadTimeoutMs;
 
     private SslContextAutoRefreshBuilder<SslContext> serverSslCtxRefresher;
-    private SslContextAutoRefreshBuilder<SslContext> clientSslCtxRefresher;
     private NettySSLContextAutoRefreshBuilder serverSSLContextAutoRefreshBuilder;
-    private NettySSLContextAutoRefreshBuilder clientSSLContextAutoRefreshBuilder;
 
     public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration serviceConfig, boolean enableTls)
             throws Exception {
@@ -88,36 +81,6 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
         } else {
             this.serverSslCtxRefresher = null;
         }
-
-        if (serviceConfig.isTlsEnabledWithBroker()) {
-            AuthenticationDataProvider authData = null;
-
-            if (!isEmpty(serviceConfig.getBrokerClientAuthenticationPlugin())) {
-                authData = AuthenticationFactory.create(serviceConfig.getBrokerClientAuthenticationPlugin(),
-                        serviceConfig.getBrokerClientAuthenticationParameters()).getAuthData();
-            }
-
-            if (tlsEnabledWithKeyStore) {
-                clientSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
-                        serviceConfig.getBrokerClientSslProvider(),
-                        serviceConfig.isTlsAllowInsecureConnection(),
-                        serviceConfig.getBrokerClientTlsTrustStoreType(),
-                        serviceConfig.getBrokerClientTlsTrustStore(),
-                        serviceConfig.getBrokerClientTlsTrustStorePassword(),
-                        serviceConfig.getBrokerClientTlsCiphers(),
-                        serviceConfig.getBrokerClientTlsProtocols(),
-                        serviceConfig.getTlsCertRefreshCheckDurationSec(),
-                        authData);
-            } else {
-                clientSslCtxRefresher = new NettyClientSslContextRefresher(
-                        serviceConfig.isTlsAllowInsecureConnection(),
-                        serviceConfig.getBrokerClientTrustCertsFilePath(),
-                        authData,
-                        serviceConfig.getTlsCertRefreshCheckDurationSec());
-            }
-        } else {
-            this.clientSslCtxRefresher = null;
-        }
     }
 
     @Override
@@ -141,25 +104,6 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
         ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                 Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
 
-        Supplier<SslHandler> sslHandlerSupplier = null;
-        if (clientSslCtxRefresher != null) {
-            sslHandlerSupplier = new Supplier<SslHandler>() {
-                @Override
-                public SslHandler get() {
-                    return clientSslCtxRefresher.get().newHandler(ch.alloc());
-                }
-            };
-        } else if (clientSSLContextAutoRefreshBuilder != null) {
-            sslHandlerSupplier = new Supplier<SslHandler>() {
-                @Override
-                public SslHandler get() {
-                    return new SslHandler(clientSSLContextAutoRefreshBuilder.get().createSSLEngine());
-                }
-            };
-        }
-
-        ch.pipeline().addLast("handler",
-                new ProxyConnection(proxyService, sslHandlerSupplier, proxyService.getDnsAddressResolverGroup()));
-
+        ch.pipeline().addLast("handler", new ProxyConnection(proxyService, proxyService.getDnsAddressResolverGroup()));
     }
 }
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 23a82270377..c2b08e964be 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
@@ -19,15 +19,13 @@
 package org.apache.pulsar.proxy.server;
 
 import static org.mockito.Mockito.spy;
-
 import com.google.common.collect.Sets;
-
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
 import org.apache.pulsar.broker.authentication.AuthenticationService;
 import org.apache.pulsar.client.admin.PulsarAdmin;
@@ -144,20 +142,24 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         };
     }
 
-    @BeforeMethod
     @Override
-    protected void setup() throws Exception {
-
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
         // enable tls and auth&auth at broker
         conf.setAuthenticationEnabled(true);
         conf.setAuthorizationEnabled(true);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
 
         conf.setBrokerServicePortTls(Optional.of(0));
+        conf.setBrokerServicePort(Optional.empty());
         conf.setWebServicePortTls(Optional.of(0));
+        conf.setWebServicePort(Optional.empty());
         conf.setTlsTrustCertsFilePath(TLS_PROXY_TRUST_CERT_FILE_PATH);
         conf.setTlsCertificateFilePath(TLS_BROKER_CERT_FILE_PATH);
         conf.setTlsKeyFilePath(TLS_BROKER_KEY_FILE_PATH);
-        conf.setTlsAllowInsecureConnection(true);
+        conf.setTlsAllowInsecureConnection(false);
 
         Set<String> superUserRoles = new HashSet<>();
         superUserRoles.add("superUser");
@@ -167,20 +169,24 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         conf.setBrokerClientAuthenticationParameters(
                 "tlsCertFile:" + TLS_BROKER_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_BROKER_KEY_FILE_PATH);
         conf.setBrokerClientTrustCertsFilePath(TLS_BROKER_TRUST_CERT_FILE_PATH);
-        Set<String> providers = new HashSet<>();
-        providers.add(AuthenticationProviderTls.class.getName());
-        conf.setAuthenticationProviders(providers);
+        conf.setAuthenticationProviders(Collections.singleton(AuthenticationProviderTls.class.getName()));
 
         conf.setClusterName("proxy-authorization");
         conf.setNumExecutorThreadPoolSize(5);
+    }
 
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
         super.init();
 
         // start proxy service
         proxyConfig.setAuthenticationEnabled(true);
         proxyConfig.setAuthorizationEnabled(false);
+        proxyConfig.setForwardAuthorizationCredentials(true);
         proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl());
         proxyConfig.setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls());
+        proxyConfig.setAdvertisedAddress(null);
 
         proxyConfig.setServicePort(Optional.of(0));
         proxyConfig.setBrokerProxyAllowedTargetPorts("*");
@@ -197,7 +203,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationTls.class.getName());
         proxyConfig.setBrokerClientAuthenticationParameters(
                 "tlsCertFile:" + TLS_PROXY_CERT_FILE_PATH + "," + "tlsKeyFile:" + TLS_PROXY_KEY_FILE_PATH);
-        proxyConfig.setAuthenticationProviders(providers);
+        proxyConfig.setAuthenticationProviders(Collections.singleton(AuthenticationProviderTls.class.getName()));
 
         proxyService = Mockito.spy(new ProxyService(proxyConfig,
                                            new AuthenticationService(
@@ -238,11 +244,11 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         // create a client which connects to proxy over tls and pass authData
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(), PulsarClient.builder());
 
-        String namespaceName = "my-property/proxy-authorization/my-ns";
+        String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", new ClusterData(brokerUrl.toString()));
+        admin.clusters().createCluster("proxy-authorization", new ClusterData(null, brokerUrlTls.toString()));
 
-        admin.tenants().createTenant("my-property",
+        admin.tenants().createTenant("my-tenant",
                 new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
         admin.namespaces().createNamespace(namespaceName);
 
@@ -252,11 +258,11 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
                 Sets.newHashSet(AuthAction.consume, AuthAction.produce));
 
         Consumer<byte[]> consumer = proxyClient.newConsumer()
-                .topic("persistent://my-property/proxy-authorization/my-ns/my-topic1")
+                .topic("persistent://my-tenant/my-ns/my-topic1")
                 .subscriptionName("my-subscriber-name").subscribe();
 
         Producer<byte[]> producer = proxyClient.newProducer(Schema.BYTES)
-                .topic("persistent://my-property/proxy-authorization/my-ns/my-topic1").create();
+                .topic("persistent://my-tenant/my-ns/my-topic1").create();
         final int msgs = 10;
         for (int i = 0; i < msgs; i++) {
             String message = "my-message-" + i;
@@ -291,11 +297,11 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(),
                 PulsarClient.builder().enableTlsHostnameVerification(hostnameVerificationEnabled));
 
-        String namespaceName = "my-property/proxy-authorization/my-ns";
+        String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", new ClusterData(brokerUrl.toString()));
+        admin.clusters().createCluster("proxy-authorization", new ClusterData(null, brokerUrlTls.toString()));
 
-        admin.tenants().createTenant("my-property",
+        admin.tenants().createTenant("my-tenant",
                 new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
         admin.namespaces().createNamespace(namespaceName);
 
@@ -305,7 +311,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
                 Sets.newHashSet(AuthAction.consume, AuthAction.produce));
 
         try {
-            proxyClient.newConsumer().topic("persistent://my-property/proxy-authorization/my-ns/my-topic1")
+            proxyClient.newConsumer().topic("persistent://my-tenant/my-ns/my-topic1")
                     .subscriptionName("my-subscriber-name").subscribe();
             if (hostnameVerificationEnabled) {
                 Assert.fail("Connection should be failed due to hostnameVerification enabled");
@@ -340,13 +346,13 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         createAdminClient();
         // create a client which connects to proxy over tls and pass authData
         PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrlTls(),
-                PulsarClient.builder().operationTimeout(1, TimeUnit.SECONDS));
+                PulsarClient.builder().operationTimeout(15, TimeUnit.SECONDS));
 
-        String namespaceName = "my-property/proxy-authorization/my-ns";
+        String namespaceName = "my-tenant/my-ns";
 
-        admin.clusters().createCluster("proxy-authorization", new ClusterData(brokerUrl.toString()));
+        admin.clusters().createCluster("proxy-authorization", new ClusterData(null, brokerUrlTls.toString()));
 
-        admin.tenants().createTenant("my-property",
+        admin.tenants().createTenant("my-tenant",
                 new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
         admin.namespaces().createNamespace(namespaceName);
 
@@ -356,7 +362,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
                 Sets.newHashSet(AuthAction.consume, AuthAction.produce));
 
         try {
-            proxyClient.newConsumer().topic("persistent://my-property/proxy-authorization/my-ns/my-topic1")
+            proxyClient.newConsumer().topic("persistent://my-tenant/my-ns/my-topic1")
                     .subscriptionName("my-subscriber-name").subscribe();
             if (hostnameVerificationEnabled) {
                 Assert.fail("Connection should be failed due to hostnameVerification enabled");
@@ -378,12 +384,12 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
     public void tlsCiphersAndProtocols(Set<String> tlsCiphers, Set<String> tlsProtocols, boolean expectFailure)
             throws Exception {
         log.info("-- Starting {} test --", methodName);
-        String namespaceName = "my-property/proxy-authorization/my-ns";
+        String namespaceName = "my-tenant/my-ns";
         createAdminClient();
 
-        admin.clusters().createCluster("proxy-authorization", new ClusterData(brokerUrl.toString()));
+        admin.clusters().createCluster("proxy-authorization", new ClusterData(null, brokerUrlTls.toString()));
 
-        admin.tenants().createTenant("my-property",
+        admin.tenants().createTenant("my-tenant",
                 new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
         admin.namespaces().createNamespace(namespaceName);
 
@@ -395,8 +401,10 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         ProxyConfiguration proxyConfig = new ProxyConfiguration();
         proxyConfig.setAuthenticationEnabled(true);
         proxyConfig.setAuthorizationEnabled(false);
+        proxyConfig.setForwardAuthorizationCredentials(true);
         proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl());
         proxyConfig.setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls());
+        proxyConfig.setAdvertisedAddress(null);
 
         proxyConfig.setServicePort(Optional.of(0));
         proxyConfig.setBrokerProxyAllowedTargetPorts("*");
@@ -443,7 +451,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
 
             PulsarClient proxyClient = createPulsarClient("pulsar://localhost:" + proxyService.getListenPortTls().get(), PulsarClient.builder());
             Consumer<byte[]> consumer = proxyClient.newConsumer()
-                    .topic("persistent://my-property/proxy-authorization/my-ns/my-topic1")
+                    .topic("persistent://my-tenant/my-ns/my-topic1")
                     .subscriptionName("my-subscriber-name").subscribe();
 
             if (expectFailure) {
@@ -466,7 +474,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         authParams.put("tlsKeyFile", TLS_SUPERUSER_CLIENT_KEY_FILE_PATH);
 
         admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString())
-                .tlsTrustCertsFilePath(TLS_PROXY_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(true)
+                .tlsTrustCertsFilePath(TLS_BROKER_TRUST_CERT_FILE_PATH)
                 .authentication(AuthenticationTls.class.getName(), authParams).build());
     }
 
@@ -480,7 +488,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase {
         authTls.configure(authParams);
 
         return clientBuilder.serviceUrl(proxyServiceUrl).statsInterval(0, TimeUnit.SECONDS)
-                .tlsTrustCertsFilePath(TLS_PROXY_TRUST_CERT_FILE_PATH).allowTlsInsecureConnection(true)
+                .tlsTrustCertsFilePath(TLS_PROXY_TRUST_CERT_FILE_PATH)
                 .authentication(authTls).enableTls(true)
                 .operationTimeout(1000, TimeUnit.MILLISECONDS).build();
     }
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cacert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cacert.pem
index 08cfc67dd21..7d2d58d8d7a 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cacert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cacert.pem
@@ -2,61 +2,76 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            c1:32:3f:61:ff:0d:77:64
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Broker, CN=Broker
+            40:cd:a5:a5:35:76:ee:02:57:8b:30:8f:2a:12:34:03:45:c5:96:8c
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 03:51:25 2018 GMT
-            Not After : Feb 17 03:51:25 2021 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Broker, CN=Broker
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: CN = CARoot
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (1024 bit)
-                Modulus (1024 bit):
-                    00:ba:01:81:08:33:0c:38:03:e1:3b:7d:9e:0c:c5:
-                    9f:1e:c6:18:31:21:2d:67:1a:69:52:e0:76:52:c8:
-                    7b:c3:83:83:31:e1:5b:3f:4f:ad:7c:75:59:a1:39:
-                    df:a3:7b:a2:e6:e7:10:02:8f:2f:ad:13:9c:8a:f6:
-                    13:b1:43:6e:54:cd:a5:fe:35:57:ef:e1:a8:f3:48:
-                    09:ad:a7:1b:6d:ae:db:73:52:1c:0b:95:eb:da:e2:
-                    fa:4e:4b:d8:78:77:a1:61:8d:a3:e0:f9:9a:49:87:
-                    42:45:71:2e:a8:7a:d1:1e:c3:1d:ea:40:3f:3a:7c:
-                    a6:e3:34:ec:db:53:e7:d3:a9
+                RSA Public-Key: (2048 bit)
+                Modulus:
+                    00:d8:d5:00:e0:6b:4f:4e:8a:67:08:e9:e3:3f:23:
+                    ef:15:1d:82:10:85:f3:3b:77:9c:96:c1:aa:eb:90:
+                    41:0b:5b:ae:77:d9:a3:f1:cf:2a:32:40:78:33:6a:
+                    81:b9:c2:cd:91:36:98:df:41:84:c0:62:8a:a1:03:
+                    89:8d:2b:b8:91:49:a9:e8:a2:90:ad:b9:cd:23:84:
+                    bc:60:1f:6f:b5:81:9f:9c:cf:d5:26:a8:a5:b6:4d:
+                    59:5f:5c:7f:da:e8:1d:3d:04:f3:b8:ef:f8:d5:73:
+                    c6:fd:6a:b1:91:ae:16:b7:45:21:9a:1a:1a:76:74:
+                    01:40:ee:fc:3c:67:be:6a:7f:f4:a3:82:37:ee:43:
+                    41:f5:67:d5:d5:64:9c:d8:53:75:34:4d:23:80:b5:
+                    59:13:c2:27:47:8e:20:32:6f:f6:b3:70:bf:5e:15:
+                    08:7e:d1:bf:aa:4d:06:6b:0d:17:21:eb:95:47:52:
+                    fa:d7:97:ef:1a:5d:63:26:17:36:01:20:ac:57:50:
+                    34:f0:57:49:38:3d:9c:68:6a:87:91:38:b6:76:9d:
+                    bc:e9:4e:c2:58:54:8d:8a:32:05:9e:ba:cb:f0:d0:
+                    ec:91:67:1d:77:bf:d5:02:77:d4:22:78:94:f4:9a:
+                    49:fa:ef:b2:9b:30:1a:8a:f0:a7:9a:2b:e5:e9:c7:
+                    36:c5
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
             X509v3 Subject Key Identifier: 
-                54:D1:B0:95:A0:92:D5:5A:C0:35:8F:6C:EE:D5:6C:4E:90:48:2E:10
+                DD:AC:A0:40:6E:E9:2B:49:F2:35:DB:B4:E9:98:AD:58:7B:37:6B:55
             X509v3 Authority Key Identifier: 
-                keyid:54:D1:B0:95:A0:92:D5:5A:C0:35:8F:6C:EE:D5:6C:4E:90:48:2E:10
-                DirName:/C=US/ST=CA/O=Apache Pulsar/OU=Broker/CN=Broker
-                serial:C1:32:3F:61:FF:0D:77:64
+                keyid:DD:AC:A0:40:6E:E9:2B:49:F2:35:DB:B4:E9:98:AD:58:7B:37:6B:55
 
-            X509v3 Basic Constraints: 
+            X509v3 Basic Constraints: critical
                 CA:TRUE
-    Signature Algorithm: sha1WithRSAEncryption
-        81:81:2e:55:77:02:81:a6:dc:31:ce:ee:50:1e:c4:79:6f:14:
-        b0:5e:b3:85:99:0e:29:ba:ab:5e:b5:0b:f7:aa:71:bb:20:ae:
-        7a:08:1e:f3:5a:7a:a1:7d:b9:a6:89:9e:89:d4:a3:c5:68:22:
-        04:99:99:b0:e7:a8:c1:ac:17:76:1e:3d:e9:07:62:99:da:38:
-        ec:0e:7c:d8:3e:bc:0c:cb:71:31:9f:d1:6a:5c:d3:b1:1b:82:
-        11:8e:69:b7:f9:1c:a7:19:b8:6d:a4:2d:6a:85:8f:5f:f5:e3:
-        32:47:8b:85:47:ba:ef:66:c1:ad:f7:1f:b6:f2:9b:9a:65:3f:
-        2f:42
+    Signature Algorithm: sha256WithRSAEncryption
+         07:0c:90:05:fa:2c:c9:4e:05:ec:6b:7d:99:9c:52:2a:20:34:
+         46:ac:8d:24:81:f9:a7:f3:1d:03:32:45:82:9a:61:af:1f:63:
+         25:6b:97:ca:93:78:e5:d7:87:81:b6:29:22:d4:0d:8d:ed:0e:
+         bd:85:80:6c:38:e9:86:3c:bd:ee:ff:26:78:0a:f0:a7:54:0b:
+         af:27:9e:8b:83:b7:10:e9:44:0d:4a:7e:a8:e2:aa:1c:06:f8:
+         18:f1:c4:c9:e4:bb:17:41:59:94:b4:dc:78:53:fb:1b:43:57:
+         82:59:de:6c:03:52:9a:28:cb:e4:9e:ea:c5:00:93:e0:27:b4:
+         4b:e6:b3:c5:88:2d:14:33:10:ff:b0:23:4e:5d:ea:17:97:7d:
+         f4:e2:c8:fe:c3:4a:77:83:64:ef:c9:b6:3e:77:64:32:07:91:
+         bd:e1:58:9a:e1:38:ab:eb:d2:e3:cb:05:7c:c7:f3:2b:47:bf:
+         36:64:7e:32:5a:62:44:07:c8:8e:9d:55:1a:99:c4:14:5a:66:
+         ed:5f:8b:ab:dd:eb:36:28:cd:77:47:84:00:ae:a7:34:0e:0d:
+         77:df:67:72:08:94:75:52:1b:4a:71:4d:31:5d:aa:1b:aa:b6:
+         e0:d6:86:52:7c:26:ae:1f:96:ab:06:32:cb:7a:f3:bb:76:3e:
+         08:53:9f:64
 -----BEGIN CERTIFICATE-----
-MIIC3jCCAkegAwIBAgIJAMEyP2H/DXdkMA0GCSqGSIb3DQEBBQUAMFQxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEPMA0G
-A1UECxMGQnJva2VyMQ8wDQYDVQQDEwZCcm9rZXIwHhcNMTgwMjE4MDM1MTI1WhcN
-MjEwMjE3MDM1MTI1WjBUMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0ExFjAUBgNV
-BAoTDUFwYWNoZSBQdWxzYXIxDzANBgNVBAsTBkJyb2tlcjEPMA0GA1UEAxMGQnJv
-a2VyMIGfMA0GCSqGSIb3DQEBAQUAA4GNADCBiQKBgQC6AYEIMww4A+E7fZ4MxZ8e
-xhgxIS1nGmlS4HZSyHvDg4Mx4Vs/T618dVmhOd+je6Lm5xACjy+tE5yK9hOxQ25U
-zaX+NVfv4ajzSAmtpxttrttzUhwLleva4vpOS9h4d6FhjaPg+ZpJh0JFcS6oetEe
-wx3qQD86fKbjNOzbU+fTqQIDAQABo4G3MIG0MB0GA1UdDgQWBBRU0bCVoJLVWsA1
-j2zu1WxOkEguEDCBhAYDVR0jBH0we4AUVNGwlaCS1VrANY9s7tVsTpBILhChWKRW
-MFQxCzAJBgNVBAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1
-bHNhcjEPMA0GA1UECxMGQnJva2VyMQ8wDQYDVQQDEwZCcm9rZXKCCQDBMj9h/w13
-ZDAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBBQUAA4GBAIGBLlV3AoGm3DHO7lAe
-xHlvFLBes4WZDim6q161C/eqcbsgrnoIHvNaeqF9uaaJnonUo8VoIgSZmbDnqMGs
-F3YePekHYpnaOOwOfNg+vAzLcTGf0Wpc07EbghGOabf5HKcZuG2kLWqFj1/14zJH
-i4VHuu9mwa33H7bym5plPy9C
+MIIDAzCCAeugAwIBAgIUQM2lpTV27gJXizCPKhI0A0XFlowwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowETEPMA0GA1UEAwwGQ0FSb290MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A
+MIIBCgKCAQEA2NUA4GtPTopnCOnjPyPvFR2CEIXzO3eclsGq65BBC1uud9mj8c8q
+MkB4M2qBucLNkTaY30GEwGKKoQOJjSu4kUmp6KKQrbnNI4S8YB9vtYGfnM/VJqil
+tk1ZX1x/2ugdPQTzuO/41XPG/Wqxka4Wt0UhmhoadnQBQO78PGe+an/0o4I37kNB
+9WfV1WSc2FN1NE0jgLVZE8InR44gMm/2s3C/XhUIftG/qk0Gaw0XIeuVR1L615fv
+Gl1jJhc2ASCsV1A08FdJOD2caGqHkTi2dp286U7CWFSNijIFnrrL8NDskWcdd7/V
+AnfUIniU9JpJ+u+ymzAaivCnmivl6cc2xQIDAQABo1MwUTAdBgNVHQ4EFgQU3ayg
+QG7pK0nyNdu06ZitWHs3a1UwHwYDVR0jBBgwFoAU3aygQG7pK0nyNdu06ZitWHs3
+a1UwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEABwyQBfosyU4F
+7Gt9mZxSKiA0RqyNJIH5p/MdAzJFgpphrx9jJWuXypN45deHgbYpItQNje0OvYWA
+bDjphjy97v8meArwp1QLryeei4O3EOlEDUp+qOKqHAb4GPHEyeS7F0FZlLTceFP7
+G0NXglnebANSmijL5J7qxQCT4Ce0S+azxYgtFDMQ/7AjTl3qF5d99OLI/sNKd4Nk
+78m2PndkMgeRveFYmuE4q+vS48sFfMfzK0e/NmR+MlpiRAfIjp1VGpnEFFpm7V+L
+q93rNijNd0eEAK6nNA4Nd99ncgiUdVIbSnFNMV2qG6q24NaGUnwmrh+WqwYyy3rz
+u3Y+CFOfZA==
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cert.pem
index 5ce3ce59464..31743d06846 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/broker-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            c1:32:3f:61:ff:0d:77:65
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Broker, CN=Broker
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:07
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 03:53:39 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Broker, CN=Broker
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache Pulsar, OU = Broker, CN = Broker
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:ca:77:dc:2a:13:25:24:cb:29:62:06:12:5f:a8:
                     92:c9:53:d6:3f:07:ca:aa:0a:5f:72:92:cd:b7:ea:
                     45:47:71:f0:63:4f:58:1a:3d:fa:ce:a6:73:90:c0:
@@ -33,40 +33,40 @@ Certificate:
                     06:0b
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                71:34:A9:AE:A7:29:C0:93:85:07:94:FE:63:AE:61:91:1D:7B:57:7D
-            X509v3 Authority Key Identifier: 
-                keyid:54:D1:B0:95:A0:92:D5:5A:C0:35:8F:6C:EE:D5:6C:4E:90:48:2E:10
-
-    Signature Algorithm: sha1WithRSAEncryption
-        24:ce:79:65:1d:bd:1a:4b:0f:7b:c2:91:e5:0b:43:4b:c7:28:
-        c0:b7:77:9b:57:ca:c7:05:37:46:2d:f9:cd:1f:f9:f7:95:44:
-        39:e9:69:64:c1:33:6e:0f:dd:56:dc:e7:f4:18:aa:e6:92:8a:
-        f1:73:ff:90:72:a1:2c:46:e5:14:9a:d7:25:fe:ac:aa:3c:bc:
-        81:50:d0:09:1a:e8:2e:3b:bc:77:ac:e1:f7:ef:eb:7d:76:44:
-        5f:29:a9:2f:4a:92:33:2d:60:0f:d5:6d:12:c4:e3:a4:4a:eb:
-        95:8c:d8:06:06:59:c1:3e:31:12:de:23:ac:af:75:0e:9c:b0:
-        9a:a5
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         8d:1d:69:d2:44:1f:af:68:30:80:c1:91:b2:2f:9a:7e:ca:ff:
+         38:46:8e:28:59:02:2d:e7:74:c4:3c:b3:ac:b3:22:53:e9:54:
+         3a:e2:4d:4d:65:63:47:dd:38:86:ec:d1:7d:4f:fe:5d:c6:c8:
+         c8:10:b8:33:5a:4d:9e:83:e3:92:97:c5:f1:d8:e3:97:6d:01:
+         50:03:de:25:d8:e4:de:62:70:b8:c4:55:5b:9f:8c:61:b8:d7:
+         f0:8f:6c:2d:80:cc:b8:7b:8b:b4:54:9a:d6:e1:f9:7f:52:99:
+         7b:ef:23:88:61:e5:7c:85:5c:57:98:cc:a6:98:4b:71:84:5c:
+         ab:5e:82:48:5a:da:5f:d6:84:b5:52:43:df:3c:0f:95:06:29:
+         00:94:f8:98:94:6d:1c:c8:76:21:7a:2f:61:34:ab:bd:27:59:
+         d1:41:99:91:69:68:f7:b6:65:21:e8:9a:b1:9b:ac:72:12:17:
+         54:0b:56:08:bd:9d:6b:0e:35:4a:f8:97:b6:83:00:55:96:0c:
+         66:13:06:c9:27:5f:cc:d0:81:4b:3e:6e:d2:85:cd:79:7a:8c:
+         a0:1e:d8:9b:e4:da:e9:ba:51:f1:29:0f:69:00:df:24:a0:55:
+         5e:cd:d0:84:c9:4a:a8:b4:12:33:29:6f:8a:8c:d7:a1:b4:8b:
+         4a:7d:a2:30
 -----BEGIN CERTIFICATE-----
-MIIDJTCCAo6gAwIBAgIJAMEyP2H/DXdlMA0GCSqGSIb3DQEBBQUAMFQxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEPMA0G
-A1UECxMGQnJva2VyMQ8wDQYDVQQDEwZCcm9rZXIwHhcNMTgwMjE4MDM1MzM5WhcN
-MzAxMTE2MDAwMDAwWjBUMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0ExFjAUBgNV
-BAoTDUFwYWNoZSBQdWxzYXIxDzANBgNVBAsTBkJyb2tlcjEPMA0GA1UEAxMGQnJv
-a2VyMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAynfcKhMlJMspYgYS
-X6iSyVPWPwfKqgpfcpLNt+pFR3HwY09YGj36zqZzkMCp9yXwdnXtsgMXvtiKVvNP
-akx+A2WV5UXrjUfoYF6eOHRQVGWg7NhcZWA0G5aDfXHUXX/jYlln6PDWJH3AbjcD
-VEw9DDM5mzPhUkTFQ9rq7izzHBYuRkx8n11Nbv6MI573fp85wXEGUvQmmiLUz8Ul
-OanS5CTG2EpIou52Jcs88L/NEHf/gRFDIcw7zBB6B4T8zAKiRd6RLWvR7Rca0Eb0
-rn2zifgxd5XlRrGpMdbY40cAsoGB24oc2fHN40019jiRDeoH8LAGTyxMdcI3/zUN
-sUIGCwIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQfFh1PcGVuU1NM
-IEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQUcTSprqcpwJOFB5T+Y65h
-kR17V30wHwYDVR0jBBgwFoAUVNGwlaCS1VrANY9s7tVsTpBILhAwDQYJKoZIhvcN
-AQEFBQADgYEAJM55ZR29GksPe8KR5QtDS8cowLd3m1fKxwU3Ri35zR/595VEOelp
-ZMEzbg/dVtzn9Biq5pKK8XP/kHKhLEblFJrXJf6sqjy8gVDQCRroLju8d6zh9+/r
-fXZEXympL0qSMy1gD9VtEsTjpErrlYzYBgZZwT4xEt4jrK91DpywmqU=
+MIIDETCCAfmgAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgcwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVDELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQKEw1BcGFj
+aGUgUHVsc2FyMQ8wDQYDVQQLEwZCcm9rZXIxDzANBgNVBAMTBkJyb2tlcjCCASIw
+DQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAMp33CoTJSTLKWIGEl+okslT1j8H
+yqoKX3KSzbfqRUdx8GNPWBo9+s6mc5DAqfcl8HZ17bIDF77YilbzT2pMfgNlleVF
+641H6GBenjh0UFRloOzYXGVgNBuWg31x1F1/42JZZ+jw1iR9wG43A1RMPQwzOZsz
+4VJExUPa6u4s8xwWLkZMfJ9dTW7+jCOe936fOcFxBlL0Jpoi1M/FJTmp0uQkxthK
+SKLudiXLPPC/zRB3/4ERQyHMO8wQegeE/MwCokXekS1r0e0XGtBG9K59s4n4MXeV
+5UaxqTHW2ONHALKBgduKHNnxzeNNNfY4kQ3qB/CwBk8sTHXCN/81DbFCBgsCAwEA
+AaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEBCwUA
+A4IBAQCNHWnSRB+vaDCAwZGyL5p+yv84Ro4oWQIt53TEPLOssyJT6VQ64k1NZWNH
+3TiG7NF9T/5dxsjIELgzWk2eg+OSl8Xx2OOXbQFQA94l2OTeYnC4xFVbn4xhuNfw
+j2wtgMy4e4u0VJrW4fl/Upl77yOIYeV8hVxXmMymmEtxhFyrXoJIWtpf1oS1UkPf
+PA+VBikAlPiYlG0cyHYhei9hNKu9J1nRQZmRaWj3tmUh6Jqxm6xyEhdUC1YIvZ1r
+DjVK+Je2gwBVlgxmEwbJJ1/M0IFLPm7Shc15eoygHtib5NrpulHxKQ9pAN8koFVe
+zdCEyUqotBIzKW+KjNehtItKfaIw
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cacert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cacert.pem
index 2940c4c5b24..127f56dd777 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cacert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cacert.pem
@@ -2,61 +2,76 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            f8:db:4d:4a:12:e2:bf:0a
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Client, CN=Client
+            77:4f:f6:cf:99:ca:77:e8:a7:6e:1e:fd:e2:cf:ac:a9:da:68:d2:42
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 03:56:51 2018 GMT
-            Not After : Feb 17 03:56:51 2021 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Client, CN=Client
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: CN = CARoot
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (1024 bit)
-                Modulus (1024 bit):
-                    00:c9:b4:bc:fe:63:eb:34:97:fb:c2:bd:84:d4:47:
-                    ea:5e:21:3f:ce:7e:0b:38:b9:a7:5c:9b:02:93:34:
-                    06:68:1c:2c:7e:5a:d9:a9:c6:db:39:d5:5a:40:52:
-                    e8:63:bb:db:76:78:8a:8c:a7:cb:dc:23:9e:b2:56:
-                    6a:c9:4f:5e:8d:f0:50:1c:2f:68:ef:0e:03:d7:e9:
-                    30:0e:6e:45:eb:a6:39:0d:67:9c:b2:f7:10:e7:a5:
-                    a4:f3:4a:6e:0d:d3:86:6f:16:66:15:04:fb:4f:95:
-                    f1:bd:c2:36:3c:5d:b3:c3:7b:a9:36:c5:f1:1a:64:
-                    c6:b5:f7:ff:c2:be:09:c0:35
+                RSA Public-Key: (2048 bit)
+                Modulus:
+                    00:b8:5e:c2:60:ed:c4:ee:3c:5b:ab:fc:64:52:f3:
+                    30:41:fc:10:5a:ac:a6:9b:0a:93:d0:d0:c9:bf:96:
+                    14:a7:cf:5c:3e:23:91:7e:54:ec:fe:2d:9f:c9:34:
+                    d1:4e:95:2f:85:9c:cc:be:90:a3:a4:cb:4d:a4:72:
+                    d2:84:e0:c7:42:c4:bf:70:b6:fa:d2:45:8b:83:66:
+                    1e:a4:e9:0e:06:a3:46:ea:a7:18:cd:33:b9:f1:ff:
+                    76:91:72:8f:cd:f9:93:43:c3:6e:17:1f:2d:86:df:
+                    b6:fb:2d:d6:be:2d:98:ad:de:00:c7:de:f9:68:b5:
+                    40:40:56:49:ae:23:e5:a1:3b:5f:15:5a:44:50:da:
+                    fb:02:d3:42:c6:87:0d:c0:8d:3a:e6:e2:aa:73:31:
+                    ab:79:58:51:cd:03:80:f3:12:ce:2f:35:04:8b:39:
+                    5f:b0:cc:b8:41:99:47:c1:17:96:8b:c2:44:84:b5:
+                    21:8a:15:52:fe:1a:5a:f9:88:cc:11:17:ee:48:dd:
+                    ba:bf:ed:67:6e:27:35:42:cf:07:5e:b1:8b:81:55:
+                    92:01:8e:61:fd:8e:82:74:b1:70:7a:3d:52:1f:16:
+                    78:12:bb:b5:09:62:ce:6d:18:4a:e9:f5:27:19:bc:
+                    93:4e:ed:dd:53:a8:c1:bb:48:b7:18:20:7b:79:48:
+                    48:9d
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
             X509v3 Subject Key Identifier: 
-                4F:E4:CE:4A:8E:79:B6:43:C0:A4:9F:8B:78:A9:6F:BD:60:81:46:54
+                0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
             X509v3 Authority Key Identifier: 
-                keyid:4F:E4:CE:4A:8E:79:B6:43:C0:A4:9F:8B:78:A9:6F:BD:60:81:46:54
-                DirName:/C=US/ST=CA/O=Apache Pulsar/OU=Client/CN=Client
-                serial:F8:DB:4D:4A:12:E2:BF:0A
+                keyid:0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
 
-            X509v3 Basic Constraints: 
+            X509v3 Basic Constraints: critical
                 CA:TRUE
-    Signature Algorithm: sha1WithRSAEncryption
-        85:04:19:99:c8:27:4f:f2:60:71:6b:f4:25:d0:b2:d0:eb:6a:
-        d8:1a:1d:5f:c5:a5:c5:af:1b:41:16:30:a2:42:f2:53:85:5e:
-        42:03:9d:e8:75:35:14:46:91:18:b3:12:ad:b8:db:7f:12:0f:
-        32:8b:02:ff:51:0c:ce:d9:15:01:98:11:81:61:e0:f2:52:d3:
-        36:2b:9f:b5:93:67:80:70:57:b8:cb:a3:5d:94:14:93:cd:f7:
-        a4:b0:d0:43:a6:f7:5e:c1:bc:b1:95:1e:dc:2d:b4:67:65:24:
-        6b:9d:eb:fc:ef:6f:ea:ea:c6:59:4c:fe:05:3f:48:89:47:a1:
-        f2:b1
+    Signature Algorithm: sha256WithRSAEncryption
+         91:e8:d8:c4:32:2e:80:5c:d4:cb:24:7a:81:43:a9:c7:95:90:
+         1a:2e:7a:d3:0c:5d:b6:21:05:67:4d:98:5a:0d:71:ea:80:01:
+         95:42:fe:fa:f1:7c:dc:bd:76:ff:05:26:3b:f0:94:b3:09:2c:
+         34:dd:43:56:46:2b:15:35:99:d9:94:54:22:cf:a6:68:b0:d1:
+         79:e2:f0:9f:0b:02:7c:cf:1f:bd:d0:f6:49:c6:82:28:a5:c6:
+         ae:94:65:cf:fd:ad:a8:6c:c2:17:da:db:f3:be:30:1a:1b:b4:
+         2c:fa:08:71:9d:64:09:45:02:92:02:ad:eb:15:47:14:43:5b:
+         a8:2d:1a:ec:14:93:dc:ff:bb:51:33:a3:d5:4d:e2:77:ca:e1:
+         a5:98:5c:7a:b6:10:19:d3:d7:f5:14:a5:d5:08:f1:97:18:3d:
+         5f:a6:4e:a2:4a:0d:4b:d4:bb:56:6b:a8:44:35:62:c5:d8:c6:
+         67:11:93:1c:22:64:3e:aa:15:08:dc:87:39:dd:f6:e0:a0:d5:
+         00:db:27:79:3d:f4:35:7c:46:a9:fa:0c:fa:fc:74:f5:bf:f4:
+         fe:71:40:45:33:22:35:83:f7:1a:96:2a:fc:b2:33:e0:1a:e8:
+         24:48:91:5d:90:5c:4c:93:33:4c:40:de:26:bb:24:ac:48:9b:
+         ae:fe:19:34
 -----BEGIN CERTIFICATE-----
-MIIC3jCCAkegAwIBAgIJAPjbTUoS4r8KMA0GCSqGSIb3DQEBBQUAMFQxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEPMA0G
-A1UECxMGQ2xpZW50MQ8wDQYDVQQDEwZDbGllbnQwHhcNMTgwMjE4MDM1NjUxWhcN
-MjEwMjE3MDM1NjUxWjBUMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0ExFjAUBgNV
-BAoTDUFwYWNoZSBQdWxzYXIxDzANBgNVBAsTBkNsaWVudDEPMA0GA1UEAxMGQ2xp
-ZW50MIGfMA0GCSqGSIb3DQEBAQUAA4GNADCBiQKBgQDJtLz+Y+s0l/vCvYTUR+pe
-IT/Ofgs4uadcmwKTNAZoHCx+Wtmpxts51VpAUuhju9t2eIqMp8vcI56yVmrJT16N
-8FAcL2jvDgPX6TAObkXrpjkNZ5yy9xDnpaTzSm4N04ZvFmYVBPtPlfG9wjY8XbPD
-e6k2xfEaZMa19//CvgnANQIDAQABo4G3MIG0MB0GA1UdDgQWBBRP5M5Kjnm2Q8Ck
-n4t4qW+9YIFGVDCBhAYDVR0jBH0we4AUT+TOSo55tkPApJ+LeKlvvWCBRlShWKRW
-MFQxCzAJBgNVBAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1
-bHNhcjEPMA0GA1UECxMGQ2xpZW50MQ8wDQYDVQQDEwZDbGllbnSCCQD4201KEuK/
-CjAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBBQUAA4GBAIUEGZnIJ0/yYHFr9CXQ
-stDratgaHV/FpcWvG0EWMKJC8lOFXkIDneh1NRRGkRizEq24238SDzKLAv9RDM7Z
-FQGYEYFh4PJS0zYrn7WTZ4BwV7jLo12UFJPN96Sw0EOm917BvLGVHtwttGdlJGud
-6/zvb+rqxllM/gU/SIlHofKx
+MIIDAzCCAeugAwIBAgIUd0/2z5nKd+inbh794s+sqdpo0kIwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowETEPMA0GA1UEAwwGQ0FSb290MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A
+MIIBCgKCAQEAuF7CYO3E7jxbq/xkUvMwQfwQWqymmwqT0NDJv5YUp89cPiORflTs
+/i2fyTTRTpUvhZzMvpCjpMtNpHLShODHQsS/cLb60kWLg2YepOkOBqNG6qcYzTO5
+8f92kXKPzfmTQ8NuFx8tht+2+y3Wvi2Yrd4Ax975aLVAQFZJriPloTtfFVpEUNr7
+AtNCxocNwI065uKqczGreVhRzQOA8xLOLzUEizlfsMy4QZlHwReWi8JEhLUhihVS
+/hpa+YjMERfuSN26v+1nbic1Qs8HXrGLgVWSAY5h/Y6CdLFwej1SHxZ4Eru1CWLO
+bRhK6fUnGbyTTu3dU6jBu0i3GCB7eUhInQIDAQABo1MwUTAdBgNVHQ4EFgQUD0Zh
+Pm9xIuYfMjd8soGmzNud9XwwHwYDVR0jBBgwFoAUD0ZhPm9xIuYfMjd8soGmzNud
+9XwwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAkejYxDIugFzU
+yyR6gUOpx5WQGi560wxdtiEFZ02YWg1x6oABlUL++vF83L12/wUmO/CUswksNN1D
+VkYrFTWZ2ZRUIs+maLDReeLwnwsCfM8fvdD2ScaCKKXGrpRlz/2tqGzCF9rb874w
+Ghu0LPoIcZ1kCUUCkgKt6xVHFENbqC0a7BST3P+7UTOj1U3id8rhpZhcerYQGdPX
+9RSl1Qjxlxg9X6ZOokoNS9S7VmuoRDVixdjGZxGTHCJkPqoVCNyHOd324KDVANsn
+eT30NXxGqfoM+vx09b/0/nFARTMiNYP3GpYq/LIz4BroJEiRXZBcTJMzTEDeJrsk
+rEibrv4ZNA==
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cert.pem
index 2412bc024d9..1a21d9d4138 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/client-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            f8:db:4d:4a:12:e2:bf:0b
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Client, CN=Client
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:03
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 03:58:13 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Client, CN=Client
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache Pulsar, OU = Client, CN = Client
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:de:1e:10:bd:64:13:c1:6c:7a:49:86:01:3b:ab:
                     ab:1d:ec:b2:93:41:6c:6c:21:f2:e6:15:1b:51:ce:
                     ad:67:fd:18:3e:7f:7a:64:a2:62:5f:2e:0b:59:b4:
@@ -33,40 +33,40 @@ Certificate:
                     9a:d1
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                E1:E9:57:60:A7:47:48:F0:1F:A8:C6:2F:95:BF:3A:42:DB:BC:7A:4D
-            X509v3 Authority Key Identifier: 
-                keyid:4F:E4:CE:4A:8E:79:B6:43:C0:A4:9F:8B:78:A9:6F:BD:60:81:46:54
-
-    Signature Algorithm: sha1WithRSAEncryption
-        a5:eb:02:90:4c:a3:33:e4:6c:c3:47:66:94:d8:3c:05:c0:ac:
-        f4:44:56:de:85:a8:41:4a:bb:28:0f:7e:aa:b9:58:40:a4:22:
-        b3:a3:46:94:42:0c:f2:93:0e:b5:c1:17:29:58:48:12:4a:3d:
-        83:40:e0:6b:07:11:54:ca:7b:58:a8:f3:7a:e4:3d:69:aa:04:
-        2e:3a:5e:d8:c1:ac:08:2f:41:17:b4:cb:35:89:00:65:f1:2b:
-        07:80:4c:c2:90:49:cd:2d:ca:43:8c:64:c1:eb:8a:b3:88:d1:
-        4b:50:95:14:41:4b:b7:76:b2:10:97:52:63:bf:17:c7:36:6f:
-        d8:bb
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         8b:88:90:00:1a:15:fa:11:f2:f0:35:6f:0f:f2:76:74:fc:8d:
+         bc:03:ee:a5:c5:21:17:c9:01:6b:58:93:fa:3e:7b:e0:0d:6d:
+         db:1f:2a:48:fa:15:34:66:b7:cb:be:82:c6:28:91:99:42:5a:
+         36:b6:0b:2f:bb:85:14:88:a9:ea:dd:0a:7a:be:c4:e7:b2:2d:
+         82:a9:37:bc:d9:5c:aa:03:2e:54:68:b1:b7:e8:d6:45:a5:8f:
+         48:45:2c:9c:7a:55:0a:4a:07:1b:30:8a:49:6d:f4:62:b1:9e:
+         92:0e:d9:34:44:6c:6d:e7:a3:18:bb:85:58:6d:da:20:83:d5:
+         ca:65:63:1e:3b:e6:df:7b:97:40:4f:b1:59:63:a9:b5:80:6f:
+         97:51:53:a1:d3:29:1f:1a:26:05:17:59:3e:16:4f:5f:38:36:
+         76:30:c6:bf:1e:3e:ed:39:83:91:31:58:01:13:59:5c:c5:e9:
+         d6:61:e0:f3:5f:c7:47:8a:5f:af:23:98:89:7b:b4:e6:f6:51:
+         98:a0:26:31:c8:67:91:6d:d5:68:75:3d:4d:48:44:5f:3b:9c:
+         df:a7:87:a0:11:02:d2:13:5f:c1:4c:3f:3e:09:59:2e:fc:cb:
+         c2:c5:f0:f8:91:df:c3:dd:ad:c8:fc:44:23:9b:78:0d:3b:f2:
+         82:f6:02:82
 -----BEGIN CERTIFICATE-----
-MIIDJTCCAo6gAwIBAgIJAPjbTUoS4r8LMA0GCSqGSIb3DQEBBQUAMFQxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEPMA0G
-A1UECxMGQ2xpZW50MQ8wDQYDVQQDEwZDbGllbnQwHhcNMTgwMjE4MDM1ODEzWhcN
-MzAxMTE2MDAwMDAwWjBUMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0ExFjAUBgNV
-BAoTDUFwYWNoZSBQdWxzYXIxDzANBgNVBAsTBkNsaWVudDEPMA0GA1UEAxMGQ2xp
-ZW50MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEA3h4QvWQTwWx6SYYB
-O6urHeyyk0FsbCHy5hUbUc6tZ/0YPn96ZKJiXy4LWbTt2RcOt7xQZkG348RxyXNz
-PdhtNIDy47mYjytUFJWzURvWkYXNtzSiULbxhm4HMPquVaBd+XwckVBifbsUhpIK
-rCk+KBuZyjBj3KlfBfg4PjAQAp/MlNdH4Br0HGiWPRJeWCFBLOyWrZ4IVoN6kl9L
-5r0BFnAor6onHcT+sgm/pbRH2VhL/kGBDqJGV8E5fI3ksacl5rTd854kyefAjBq0
-q925M78Ry767Ivf8rcRAQdfvNwgalUUf2xRfC/hI/0Eky1yOGEhMXxnpsHsi07xC
-MkWa0QIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQfFh1PcGVuU1NM
-IEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQU4elXYKdHSPAfqMYvlb86
-Qtu8ek0wHwYDVR0jBBgwFoAUT+TOSo55tkPApJ+LeKlvvWCBRlQwDQYJKoZIhvcN
-AQEFBQADgYEApesCkEyjM+Rsw0dmlNg8BcCs9ERW3oWoQUq7KA9+qrlYQKQis6NG
-lEIM8pMOtcEXKVhIEko9g0DgawcRVMp7WKjzeuQ9aaoELjpe2MGsCC9BF7TLNYkA
-ZfErB4BMwpBJzS3KQ4xkweuKs4jRS1CVFEFLt3ayEJdSY78XxzZv2Ls=
+MIIDETCCAfmgAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgMwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVDELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQKEw1BcGFj
+aGUgUHVsc2FyMQ8wDQYDVQQLEwZDbGllbnQxDzANBgNVBAMTBkNsaWVudDCCASIw
+DQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAN4eEL1kE8FsekmGATurqx3sspNB
+bGwh8uYVG1HOrWf9GD5/emSiYl8uC1m07dkXDre8UGZBt+PEcclzcz3YbTSA8uO5
+mI8rVBSVs1Eb1pGFzbc0olC28YZuBzD6rlWgXfl8HJFQYn27FIaSCqwpPigbmcow
+Y9ypXwX4OD4wEAKfzJTXR+Aa9Bxolj0SXlghQSzslq2eCFaDepJfS+a9ARZwKK+q
+Jx3E/rIJv6W0R9lYS/5BgQ6iRlfBOXyN5LGnJea03fOeJMnnwIwatKvduTO/Ecu+
+uyL3/K3EQEHX7zcIGpVFH9sUXwv4SP9BJMtcjhhITF8Z6bB7ItO8QjJFmtECAwEA
+AaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEBCwUA
+A4IBAQCLiJAAGhX6EfLwNW8P8nZ0/I28A+6lxSEXyQFrWJP6PnvgDW3bHypI+hU0
+ZrfLvoLGKJGZQlo2tgsvu4UUiKnq3Qp6vsTnsi2CqTe82VyqAy5UaLG36NZFpY9I
+RSycelUKSgcbMIpJbfRisZ6SDtk0RGxt56MYu4VYbdogg9XKZWMeO+bfe5dAT7FZ
+Y6m1gG+XUVOh0ykfGiYFF1k+Fk9fODZ2MMa/Hj7tOYORMVgBE1lcxenWYeDzX8dH
+il+vI5iJe7Tm9lGYoCYxyGeRbdVodT1NSERfO5zfp4egEQLSE1/BTD8+CVku/MvC
+xfD4kd/D3a3I/EQjm3gNO/KC9gKC
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem
index 1f71b88bb45..127f56dd777 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cacert.pem
@@ -2,61 +2,76 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            a5:2d:2e:41:e9:fc:8a:91
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Proxy, CN=Proxy
+            77:4f:f6:cf:99:ca:77:e8:a7:6e:1e:fd:e2:cf:ac:a9:da:68:d2:42
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 04:00:32 2018 GMT
-            Not After : Feb 17 04:00:32 2021 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Proxy, CN=Proxy
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: CN = CARoot
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (1024 bit)
-                Modulus (1024 bit):
-                    00:aa:ce:ea:82:4f:ac:a8:97:7b:0c:33:cd:ef:7f:
-                    24:45:e5:81:a2:2c:7a:ab:65:34:27:27:39:ae:f4:
-                    b2:f3:0e:cc:08:3b:8e:1d:78:95:aa:95:01:0e:a3:
-                    df:db:4b:9a:ad:85:e6:af:96:16:41:35:dc:b2:23:
-                    03:ff:b9:d6:75:25:29:37:f5:3f:26:43:c3:36:a0:
-                    9c:0f:36:a5:91:dd:7d:18:5d:45:24:d3:f6:bf:86:
-                    91:91:10:b5:00:bf:12:6a:01:9f:28:38:01:08:5f:
-                    fd:a8:6d:98:33:cc:77:fb:a1:fe:06:59:92:6d:0b:
-                    14:bc:9b:59:fd:98:69:ec:6d
+                RSA Public-Key: (2048 bit)
+                Modulus:
+                    00:b8:5e:c2:60:ed:c4:ee:3c:5b:ab:fc:64:52:f3:
+                    30:41:fc:10:5a:ac:a6:9b:0a:93:d0:d0:c9:bf:96:
+                    14:a7:cf:5c:3e:23:91:7e:54:ec:fe:2d:9f:c9:34:
+                    d1:4e:95:2f:85:9c:cc:be:90:a3:a4:cb:4d:a4:72:
+                    d2:84:e0:c7:42:c4:bf:70:b6:fa:d2:45:8b:83:66:
+                    1e:a4:e9:0e:06:a3:46:ea:a7:18:cd:33:b9:f1:ff:
+                    76:91:72:8f:cd:f9:93:43:c3:6e:17:1f:2d:86:df:
+                    b6:fb:2d:d6:be:2d:98:ad:de:00:c7:de:f9:68:b5:
+                    40:40:56:49:ae:23:e5:a1:3b:5f:15:5a:44:50:da:
+                    fb:02:d3:42:c6:87:0d:c0:8d:3a:e6:e2:aa:73:31:
+                    ab:79:58:51:cd:03:80:f3:12:ce:2f:35:04:8b:39:
+                    5f:b0:cc:b8:41:99:47:c1:17:96:8b:c2:44:84:b5:
+                    21:8a:15:52:fe:1a:5a:f9:88:cc:11:17:ee:48:dd:
+                    ba:bf:ed:67:6e:27:35:42:cf:07:5e:b1:8b:81:55:
+                    92:01:8e:61:fd:8e:82:74:b1:70:7a:3d:52:1f:16:
+                    78:12:bb:b5:09:62:ce:6d:18:4a:e9:f5:27:19:bc:
+                    93:4e:ed:dd:53:a8:c1:bb:48:b7:18:20:7b:79:48:
+                    48:9d
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
             X509v3 Subject Key Identifier: 
-                4F:39:5A:C4:BF:78:EF:3D:FC:F1:68:5A:F6:B9:4B:D2:B7:03:C7:87
+                0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
             X509v3 Authority Key Identifier: 
-                keyid:4F:39:5A:C4:BF:78:EF:3D:FC:F1:68:5A:F6:B9:4B:D2:B7:03:C7:87
-                DirName:/C=US/ST=CA/O=Apache Pulsar/OU=Proxy/CN=Proxy
-                serial:A5:2D:2E:41:E9:FC:8A:91
+                keyid:0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
 
-            X509v3 Basic Constraints: 
+            X509v3 Basic Constraints: critical
                 CA:TRUE
-    Signature Algorithm: sha1WithRSAEncryption
-        84:e1:30:a5:a5:7e:39:9b:2a:1f:cb:1e:67:c6:00:75:f3:8f:
-        6a:d0:ef:d7:46:39:2c:b6:ba:1f:03:7d:eb:cf:22:ef:46:82:
-        bb:89:08:dd:3f:28:b3:6e:79:1a:14:26:ed:38:2f:f0:c9:fe:
-        7f:72:5c:8a:82:b8:05:fe:f7:45:6c:e9:6e:ff:f9:d3:a4:60:
-        1a:e9:7b:71:c8:a1:80:3d:0f:33:44:06:30:c7:c9:2f:8f:e4:
-        5d:68:25:cb:28:49:5a:5d:ac:10:f7:d2:90:cf:0c:1f:ff:7c:
-        7b:04:95:a7:b9:27:d9:66:ac:73:6e:92:84:de:68:fc:86:27:
-        e8:d3
+    Signature Algorithm: sha256WithRSAEncryption
+         91:e8:d8:c4:32:2e:80:5c:d4:cb:24:7a:81:43:a9:c7:95:90:
+         1a:2e:7a:d3:0c:5d:b6:21:05:67:4d:98:5a:0d:71:ea:80:01:
+         95:42:fe:fa:f1:7c:dc:bd:76:ff:05:26:3b:f0:94:b3:09:2c:
+         34:dd:43:56:46:2b:15:35:99:d9:94:54:22:cf:a6:68:b0:d1:
+         79:e2:f0:9f:0b:02:7c:cf:1f:bd:d0:f6:49:c6:82:28:a5:c6:
+         ae:94:65:cf:fd:ad:a8:6c:c2:17:da:db:f3:be:30:1a:1b:b4:
+         2c:fa:08:71:9d:64:09:45:02:92:02:ad:eb:15:47:14:43:5b:
+         a8:2d:1a:ec:14:93:dc:ff:bb:51:33:a3:d5:4d:e2:77:ca:e1:
+         a5:98:5c:7a:b6:10:19:d3:d7:f5:14:a5:d5:08:f1:97:18:3d:
+         5f:a6:4e:a2:4a:0d:4b:d4:bb:56:6b:a8:44:35:62:c5:d8:c6:
+         67:11:93:1c:22:64:3e:aa:15:08:dc:87:39:dd:f6:e0:a0:d5:
+         00:db:27:79:3d:f4:35:7c:46:a9:fa:0c:fa:fc:74:f5:bf:f4:
+         fe:71:40:45:33:22:35:83:f7:1a:96:2a:fc:b2:33:e0:1a:e8:
+         24:48:91:5d:90:5c:4c:93:33:4c:40:de:26:bb:24:ac:48:9b:
+         ae:fe:19:34
 -----BEGIN CERTIFICATE-----
-MIIC2DCCAkGgAwIBAgIJAKUtLkHp/IqRMA0GCSqGSIb3DQEBBQUAMFIxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEOMAwG
-A1UECxMFUHJveHkxDjAMBgNVBAMTBVByb3h5MB4XDTE4MDIxODA0MDAzMloXDTIx
-MDIxNzA0MDAzMlowUjELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQK
-Ew1BcGFjaGUgUHVsc2FyMQ4wDAYDVQQLEwVQcm94eTEOMAwGA1UEAxMFUHJveHkw
-gZ8wDQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBAKrO6oJPrKiXewwzze9/JEXlgaIs
-eqtlNCcnOa70svMOzAg7jh14laqVAQ6j39tLmq2F5q+WFkE13LIjA/+51nUlKTf1
-PyZDwzagnA82pZHdfRhdRSTT9r+GkZEQtQC/EmoBnyg4AQhf/ahtmDPMd/uh/gZZ
-km0LFLybWf2YaextAgMBAAGjgbUwgbIwHQYDVR0OBBYEFE85WsS/eO89/PFoWva5
-S9K3A8eHMIGCBgNVHSMEezB5gBRPOVrEv3jvPfzxaFr2uUvStwPHh6FWpFQwUjEL
-MAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQKEw1BcGFjaGUgUHVsc2Fy
-MQ4wDAYDVQQLEwVQcm94eTEOMAwGA1UEAxMFUHJveHmCCQClLS5B6fyKkTAMBgNV
-HRMEBTADAQH/MA0GCSqGSIb3DQEBBQUAA4GBAIThMKWlfjmbKh/LHmfGAHXzj2rQ
-79dGOSy2uh8DfevPIu9GgruJCN0/KLNueRoUJu04L/DJ/n9yXIqCuAX+90Vs6W7/
-+dOkYBrpe3HIoYA9DzNEBjDHyS+P5F1oJcsoSVpdrBD30pDPDB//fHsElae5J9lm
-rHNukoTeaPyGJ+jT
+MIIDAzCCAeugAwIBAgIUd0/2z5nKd+inbh794s+sqdpo0kIwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowETEPMA0GA1UEAwwGQ0FSb290MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A
+MIIBCgKCAQEAuF7CYO3E7jxbq/xkUvMwQfwQWqymmwqT0NDJv5YUp89cPiORflTs
+/i2fyTTRTpUvhZzMvpCjpMtNpHLShODHQsS/cLb60kWLg2YepOkOBqNG6qcYzTO5
+8f92kXKPzfmTQ8NuFx8tht+2+y3Wvi2Yrd4Ax975aLVAQFZJriPloTtfFVpEUNr7
+AtNCxocNwI065uKqczGreVhRzQOA8xLOLzUEizlfsMy4QZlHwReWi8JEhLUhihVS
+/hpa+YjMERfuSN26v+1nbic1Qs8HXrGLgVWSAY5h/Y6CdLFwej1SHxZ4Eru1CWLO
+bRhK6fUnGbyTTu3dU6jBu0i3GCB7eUhInQIDAQABo1MwUTAdBgNVHQ4EFgQUD0Zh
+Pm9xIuYfMjd8soGmzNud9XwwHwYDVR0jBBgwFoAUD0ZhPm9xIuYfMjd8soGmzNud
+9XwwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAkejYxDIugFzU
+yyR6gUOpx5WQGi560wxdtiEFZ02YWg1x6oABlUL++vF83L12/wUmO/CUswksNN1D
+VkYrFTWZ2ZRUIs+maLDReeLwnwsCfM8fvdD2ScaCKKXGrpRlz/2tqGzCF9rb874w
+Ghu0LPoIcZ1kCUUCkgKt6xVHFENbqC0a7BST3P+7UTOj1U3id8rhpZhcerYQGdPX
+9RSl1Qjxlxg9X6ZOokoNS9S7VmuoRDVixdjGZxGTHCJkPqoVCNyHOd324KDVANsn
+eT30NXxGqfoM+vx09b/0/nFARTMiNYP3GpYq/LIz4BroJEiRXZBcTJMzTEDeJrsk
+rEibrv4ZNA==
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem
index a3962c32ddc..e2c1e5a230c 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/ProxyWithAuthorizationTest/proxy-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            a5:2d:2e:41:e9:fc:8a:92
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache Pulsar, OU=Proxy, CN=Proxy
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:04
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 18 04:02:27 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache Pulsar, OU=Proxy, CN=Proxy
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache Pulsar, OU = Proxy, CN = Proxy
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:c3:5c:c5:ad:17:dc:f4:d4:c4:ea:1c:60:5a:24:
                     46:13:d9:cf:c0:cd:83:2e:2f:82:70:e5:e0:8d:33:
                     bd:95:b5:cf:c6:f0:54:d5:8d:bd:87:0d:62:6c:1d:
@@ -33,40 +33,40 @@ Certificate:
                     f3:8d
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                D5:A5:19:6A:3B:38:5F:19:C7:34:C6:BC:68:BE:16:A5:0B:43:57:2D
-            X509v3 Authority Key Identifier: 
-                keyid:4F:39:5A:C4:BF:78:EF:3D:FC:F1:68:5A:F6:B9:4B:D2:B7:03:C7:87
-
-    Signature Algorithm: sha1WithRSAEncryption
-        a0:f1:e6:d4:75:75:10:0e:27:18:28:93:9f:c5:15:2b:f3:52:
-        3c:f7:c7:6d:96:b3:7f:65:6c:78:be:26:f5:f2:41:36:f0:b2:
-        fb:64:67:73:d2:bf:d7:24:af:30:1e:6f:3a:9c:80:98:34:06:
-        11:ba:45:06:57:ec:d9:f0:77:1f:d6:e8:0c:13:9d:d1:15:c7:
-        d8:73:fb:aa:dc:0d:3c:4b:3a:bb:87:3c:21:6d:05:9d:fa:74:
-        db:61:4c:47:6a:e7:6b:79:2b:3f:62:a8:fc:e6:11:c8:0f:40:
-        48:51:71:a2:ad:77:d5:fe:ff:1d:73:82:0c:3c:98:ab:26:9b:
-        78:d5
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         8d:b6:2c:5f:87:13:06:a8:66:ce:11:2a:2c:20:1e:c7:ee:50:
+         75:a7:d1:7c:ad:c6:ec:d1:18:d0:fa:aa:00:fa:08:f9:0f:cc:
+         df:59:9a:6b:1c:18:07:15:84:d0:9a:24:8d:dd:46:79:9c:dc:
+         9e:3e:97:10:24:b2:9d:d4:f6:c5:79:58:87:7c:a6:af:cf:69:
+         23:fb:43:7a:0f:4d:26:e0:e9:66:c5:ad:fa:88:e2:c5:6e:6a:
+         ce:70:0c:8f:73:01:d6:fd:a9:1f:31:49:41:17:45:22:cc:a6:
+         71:e4:f4:0f:0f:2e:3e:49:0b:5f:04:94:36:49:fa:72:42:c9:
+         25:75:84:9a:dc:16:cb:69:44:44:e5:3a:ff:26:f6:44:42:4c:
+         6c:e2:56:d6:3e:bc:f2:8b:83:de:e2:91:70:65:b9:d0:dd:a3:
+         d1:de:53:27:77:13:2d:86:27:c3:40:2f:c1:a5:50:1c:5a:44:
+         51:b4:29:11:c3:30:9d:1a:96:25:7a:d6:05:70:ad:06:0d:f2:
+         9b:b1:b6:82:39:06:c7:7c:b2:49:04:19:e4:7e:87:b8:d8:42:
+         1d:ab:ed:d0:b0:7f:79:6b:89:75:2f:6a:26:67:3d:33:57:5f:
+         5a:49:52:98:3b:2a:e5:43:d7:f9:97:ca:75:cd:6f:e9:e4:66:
+         b6:d6:c2:c7
 -----BEGIN CERTIFICATE-----
-MIIDITCCAoqgAwIBAgIJAKUtLkHp/IqSMA0GCSqGSIb3DQEBBQUAMFIxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEWMBQGA1UEChMNQXBhY2hlIFB1bHNhcjEOMAwG
-A1UECxMFUHJveHkxDjAMBgNVBAMTBVByb3h5MB4XDTE4MDIxODA0MDIyN1oXDTMw
-MTExNjAwMDAwMFowUjELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQK
-Ew1BcGFjaGUgUHVsc2FyMQ4wDAYDVQQLEwVQcm94eTEOMAwGA1UEAxMFUHJveHkw
-ggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDDXMWtF9z01MTqHGBaJEYT
-2c/AzYMuL4Jw5eCNM72Vtc/G8FTVjb2HDWJsHT9SZnT/BjMcPNXtLmPZlsbxmILH
-lEq8ZPKbOlTsgZm8FIJDhwxr2gOMqgtB1/4nxPmIgTSx/yrgbdBH3cERpVSpUzLN
-j/Z1WI4F5NmxrGn+tlTDrTYEonf1U7Z0g9VqAeCWtaKvUI+1152nwr34MYYJX3wK
-sts04YAlF199b4vcjtX5z8/19o9q/j6WAMlWsNDjRt65popem45/6hnMolt1Ijwd
-Nkjk8hoBlWHB8HonnYOWdMypBEIIUzSYLrfjg/nyoynhI8TtoBz2Ku3cwN+XqfON
-AgMBAAGjezB5MAkGA1UdEwQCMAAwLAYJYIZIAYb4QgENBB8WHU9wZW5TU0wgR2Vu
-ZXJhdGVkIENlcnRpZmljYXRlMB0GA1UdDgQWBBTVpRlqOzhfGcc0xrxovhalC0NX
-LTAfBgNVHSMEGDAWgBRPOVrEv3jvPfzxaFr2uUvStwPHhzANBgkqhkiG9w0BAQUF
-AAOBgQCg8ebUdXUQDicYKJOfxRUr81I898dtlrN/ZWx4vib18kE28LL7ZGdz0r/X
-JK8wHm86nICYNAYRukUGV+zZ8Hcf1ugME53RFcfYc/uq3A08Szq7hzwhbQWd+nTb
-YUxHaudreSs/Yqj85hHID0BIUXGirXfV/v8dc4IMPJirJpt41Q==
+MIIDDzCCAfegAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgQwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowUjELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMRYwFAYDVQQKEw1BcGFj
+aGUgUHVsc2FyMQ4wDAYDVQQLEwVQcm94eTEOMAwGA1UEAxMFUHJveHkwggEiMA0G
+CSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDDXMWtF9z01MTqHGBaJEYT2c/AzYMu
+L4Jw5eCNM72Vtc/G8FTVjb2HDWJsHT9SZnT/BjMcPNXtLmPZlsbxmILHlEq8ZPKb
+OlTsgZm8FIJDhwxr2gOMqgtB1/4nxPmIgTSx/yrgbdBH3cERpVSpUzLNj/Z1WI4F
+5NmxrGn+tlTDrTYEonf1U7Z0g9VqAeCWtaKvUI+1152nwr34MYYJX3wKsts04YAl
+F199b4vcjtX5z8/19o9q/j6WAMlWsNDjRt65popem45/6hnMolt1IjwdNkjk8hoB
+lWHB8HonnYOWdMypBEIIUzSYLrfjg/nyoynhI8TtoBz2Ku3cwN+XqfONAgMBAAGj
+HjAcMBoGA1UdEQQTMBGCCWxvY2FsaG9zdIcEfwAAATANBgkqhkiG9w0BAQsFAAOC
+AQEAjbYsX4cTBqhmzhEqLCAex+5QdafRfK3G7NEY0PqqAPoI+Q/M31maaxwYBxWE
+0Jokjd1GeZzcnj6XECSyndT2xXlYh3ymr89pI/tDeg9NJuDpZsWt+ojixW5qznAM
+j3MB1v2pHzFJQRdFIsymceT0Dw8uPkkLXwSUNkn6ckLJJXWEmtwWy2lEROU6/yb2
+REJMbOJW1j688ouD3uKRcGW50N2j0d5TJ3cTLYYnw0AvwaVQHFpEUbQpEcMwnRqW
+JXrWBXCtBg3ym7G2gjkGx3yySQQZ5H6HuNhCHavt0LB/eWuJdS9qJmc9M1dfWklS
+mDsq5UPX+ZfKdc1v6eRmttbCxw==
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/cacert.pem b/pulsar-proxy/src/test/resources/authentication/tls/cacert.pem
index 55e9067458a..127f56dd777 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/cacert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/cacert.pem
@@ -2,61 +2,76 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:94
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            77:4f:f6:cf:99:ca:77:e8:a7:6e:1e:fd:e2:cf:ac:a9:da:68:d2:42
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 17 01:37:33 2018 GMT
-            Not After : Feb 16 01:37:33 2021 GMT
-        Subject: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: CN = CARoot
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (1024 bit)
-                Modulus (1024 bit):
-                    00:ea:16:8d:a5:b1:19:61:34:54:07:02:60:4e:6d:
-                    54:92:08:fd:fb:23:79:9c:05:bf:14:f7:bc:aa:db:
-                    2b:42:a4:35:74:86:e3:00:ad:8b:18:79:73:7d:f2:
-                    d1:74:dd:74:bc:b8:a2:4c:80:c9:f3:80:ce:bf:f8:
-                    6d:97:f5:05:4f:f4:b2:99:50:e8:d8:b0:c4:57:a0:
-                    e7:dc:82:57:75:2a:a2:02:21:76:f7:37:c2:dc:7c:
-                    4c:36:a6:73:6f:dc:75:48:72:ad:fa:98:02:70:b2:
-                    5e:a2:83:cc:c3:8d:20:a7:1e:bc:d7:1e:c1:d1:7e:
-                    39:35:4b:f5:be:6b:c1:0f:f9
+                RSA Public-Key: (2048 bit)
+                Modulus:
+                    00:b8:5e:c2:60:ed:c4:ee:3c:5b:ab:fc:64:52:f3:
+                    30:41:fc:10:5a:ac:a6:9b:0a:93:d0:d0:c9:bf:96:
+                    14:a7:cf:5c:3e:23:91:7e:54:ec:fe:2d:9f:c9:34:
+                    d1:4e:95:2f:85:9c:cc:be:90:a3:a4:cb:4d:a4:72:
+                    d2:84:e0:c7:42:c4:bf:70:b6:fa:d2:45:8b:83:66:
+                    1e:a4:e9:0e:06:a3:46:ea:a7:18:cd:33:b9:f1:ff:
+                    76:91:72:8f:cd:f9:93:43:c3:6e:17:1f:2d:86:df:
+                    b6:fb:2d:d6:be:2d:98:ad:de:00:c7:de:f9:68:b5:
+                    40:40:56:49:ae:23:e5:a1:3b:5f:15:5a:44:50:da:
+                    fb:02:d3:42:c6:87:0d:c0:8d:3a:e6:e2:aa:73:31:
+                    ab:79:58:51:cd:03:80:f3:12:ce:2f:35:04:8b:39:
+                    5f:b0:cc:b8:41:99:47:c1:17:96:8b:c2:44:84:b5:
+                    21:8a:15:52:fe:1a:5a:f9:88:cc:11:17:ee:48:dd:
+                    ba:bf:ed:67:6e:27:35:42:cf:07:5e:b1:8b:81:55:
+                    92:01:8e:61:fd:8e:82:74:b1:70:7a:3d:52:1f:16:
+                    78:12:bb:b5:09:62:ce:6d:18:4a:e9:f5:27:19:bc:
+                    93:4e:ed:dd:53:a8:c1:bb:48:b7:18:20:7b:79:48:
+                    48:9d
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
             X509v3 Subject Key Identifier: 
-                D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
+                0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
             X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-                DirName:/C=US/ST=CA/O=Apache/OU=Pulsar Incubator/CN=localhost
-                serial:88:08:98:B3:13:D8:00:94
+                keyid:0F:46:61:3E:6F:71:22:E6:1F:32:37:7C:B2:81:A6:CC:DB:9D:F5:7C
 
-            X509v3 Basic Constraints: 
+            X509v3 Basic Constraints: critical
                 CA:TRUE
-    Signature Algorithm: sha1WithRSAEncryption
-        5e:30:c5:7b:30:3e:1e:16:cd:ba:66:f1:2a:19:13:8a:1a:00:
-        08:f4:1e:8c:e4:3d:57:13:65:96:bf:07:58:55:52:37:3e:aa:
-        2c:19:de:ee:c3:92:6e:79:f3:06:0e:9a:7b:e0:02:50:c3:ef:
-        3b:84:ea:8f:e0:f0:16:a6:a6:67:8b:be:73:0e:5d:f7:88:39:
-        d3:d4:df:85:ad:7c:c1:4f:fa:55:55:6f:c2:48:4e:8e:82:fa:
-        72:3b:8e:9d:dc:f7:2e:9d:47:8e:e5:c9:a2:ee:b1:76:94:15:
-        7c:7a:62:bc:06:45:fa:61:2e:33:8c:18:3e:e9:d5:90:a5:a6:
-        80:5a
+    Signature Algorithm: sha256WithRSAEncryption
+         91:e8:d8:c4:32:2e:80:5c:d4:cb:24:7a:81:43:a9:c7:95:90:
+         1a:2e:7a:d3:0c:5d:b6:21:05:67:4d:98:5a:0d:71:ea:80:01:
+         95:42:fe:fa:f1:7c:dc:bd:76:ff:05:26:3b:f0:94:b3:09:2c:
+         34:dd:43:56:46:2b:15:35:99:d9:94:54:22:cf:a6:68:b0:d1:
+         79:e2:f0:9f:0b:02:7c:cf:1f:bd:d0:f6:49:c6:82:28:a5:c6:
+         ae:94:65:cf:fd:ad:a8:6c:c2:17:da:db:f3:be:30:1a:1b:b4:
+         2c:fa:08:71:9d:64:09:45:02:92:02:ad:eb:15:47:14:43:5b:
+         a8:2d:1a:ec:14:93:dc:ff:bb:51:33:a3:d5:4d:e2:77:ca:e1:
+         a5:98:5c:7a:b6:10:19:d3:d7:f5:14:a5:d5:08:f1:97:18:3d:
+         5f:a6:4e:a2:4a:0d:4b:d4:bb:56:6b:a8:44:35:62:c5:d8:c6:
+         67:11:93:1c:22:64:3e:aa:15:08:dc:87:39:dd:f6:e0:a0:d5:
+         00:db:27:79:3d:f4:35:7c:46:a9:fa:0c:fa:fc:74:f5:bf:f4:
+         fe:71:40:45:33:22:35:83:f7:1a:96:2a:fc:b2:33:e0:1a:e8:
+         24:48:91:5d:90:5c:4c:93:33:4c:40:de:26:bb:24:ac:48:9b:
+         ae:fe:19:34
 -----BEGIN CERTIFICATE-----
-MIIC8jCCAlugAwIBAgIJAIgImLMT2ACUMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDEz
-NzMzWhcNMjEwMjE2MDEzNzMzWjBaMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEZMBcGA1UECxMQUHVsc2FyIEluY3ViYXRvcjESMBAG
-A1UEAxMJbG9jYWxob3N0MIGfMA0GCSqGSIb3DQEBAQUAA4GNADCBiQKBgQDqFo2l
-sRlhNFQHAmBObVSSCP37I3mcBb8U97yq2ytCpDV0huMArYsYeXN98tF03XS8uKJM
-gMnzgM6/+G2X9QVP9LKZUOjYsMRXoOfcgld1KqICIXb3N8LcfEw2pnNv3HVIcq36
-mAJwsl6ig8zDjSCnHrzXHsHRfjk1S/W+a8EP+QIDAQABo4G/MIG8MB0GA1UdDgQW
-BBTUes0PRBsWKSUU7aLvEw+nRgl49jCBjAYDVR0jBIGEMIGBgBTUes0PRBsWKSUU
-7aLvEw+nRgl49qFepFwwWjELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYD
-VQQKEwZBcGFjaGUxGTAXBgNVBAsTEFB1bHNhciBJbmN1YmF0b3IxEjAQBgNVBAMT
-CWxvY2FsaG9zdIIJAIgImLMT2ACUMAwGA1UdEwQFMAMBAf8wDQYJKoZIhvcNAQEF
-BQADgYEAXjDFezA+HhbNumbxKhkTihoACPQejOQ9VxNllr8HWFVSNz6qLBne7sOS
-bnnzBg6ae+ACUMPvO4Tqj+DwFqamZ4u+cw5d94g509Tfha18wU/6VVVvwkhOjoL6
-cjuOndz3Lp1HjuXJou6xdpQVfHpivAZF+mEuM4wYPunVkKWmgFo=
+MIIDAzCCAeugAwIBAgIUd0/2z5nKd+inbh794s+sqdpo0kIwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowETEPMA0GA1UEAwwGQ0FSb290MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A
+MIIBCgKCAQEAuF7CYO3E7jxbq/xkUvMwQfwQWqymmwqT0NDJv5YUp89cPiORflTs
+/i2fyTTRTpUvhZzMvpCjpMtNpHLShODHQsS/cLb60kWLg2YepOkOBqNG6qcYzTO5
+8f92kXKPzfmTQ8NuFx8tht+2+y3Wvi2Yrd4Ax975aLVAQFZJriPloTtfFVpEUNr7
+AtNCxocNwI065uKqczGreVhRzQOA8xLOLzUEizlfsMy4QZlHwReWi8JEhLUhihVS
+/hpa+YjMERfuSN26v+1nbic1Qs8HXrGLgVWSAY5h/Y6CdLFwej1SHxZ4Eru1CWLO
+bRhK6fUnGbyTTu3dU6jBu0i3GCB7eUhInQIDAQABo1MwUTAdBgNVHQ4EFgQUD0Zh
+Pm9xIuYfMjd8soGmzNud9XwwHwYDVR0jBBgwFoAUD0ZhPm9xIuYfMjd8soGmzNud
+9XwwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAkejYxDIugFzU
+yyR6gUOpx5WQGi560wxdtiEFZ02YWg1x6oABlUL++vF83L12/wUmO/CUswksNN1D
+VkYrFTWZ2ZRUIs+maLDReeLwnwsCfM8fvdD2ScaCKKXGrpRlz/2tqGzCF9rb874w
+Ghu0LPoIcZ1kCUUCkgKt6xVHFENbqC0a7BST3P+7UTOj1U3id8rhpZhcerYQGdPX
+9RSl1Qjxlxg9X6ZOokoNS9S7VmuoRDVixdjGZxGTHCJkPqoVCNyHOd324KDVANsn
+eT30NXxGqfoM+vx09b/0/nFARTMiNYP3GpYq/LIz4BroJEiRXZBcTJMzTEDeJrsk
+rEibrv4ZNA==
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/client-cert.pem b/pulsar-proxy/src/test/resources/authentication/tls/client-cert.pem
index 61847f2e612..192d686246f 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/client-cert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/client-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:99
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:01
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 17 02:50:05 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=superUser
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache, OU = Apache Pulsar, CN = superUser
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:cd:43:7d:98:40:f9:b0:5b:bc:ae:db:c0:0b:ad:
                     26:90:96:e0:62:38:ed:68:b1:70:46:3b:de:44:f9:
                     14:51:86:10:eb:ca:90:e7:88:e8:f9:91:85:e0:dd:
@@ -33,41 +33,40 @@ Certificate:
                     e1:0b
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                53:7C:D5:D1:52:97:9A:D6:D5:EA:EC:B6:0C:9B:43:39:19:73:F6:2C
-            X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-
-    Signature Algorithm: sha1WithRSAEncryption
-        e4:03:82:ff:be:df:7c:73:2a:c5:8f:7d:87:ab:95:b1:2b:e5:
-        f7:41:22:4f:28:54:84:7a:cc:fe:70:89:0f:48:e5:8a:17:e1:
-        44:ad:12:e9:a1:3a:c7:84:55:f0:7c:29:52:0a:a1:ab:cc:5b:
-        31:e5:b2:37:73:3a:8d:f2:f1:fb:e8:f6:a2:b9:ef:11:10:f8:
-        31:43:8f:af:ce:09:f4:cb:96:0e:d4:58:42:6e:86:ab:b9:03:
-        19:8b:4a:6e:ef:50:c0:7e:c9:0b:1d:2b:42:bf:eb:d0:06:05:
-        84:ea:5a:8a:22:5c:56:fa:da:2a:9f:8a:b2:90:66:8c:5e:01:
-        87:45
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         96:c2:23:2d:46:d0:3d:23:0e:ab:3d:b6:1e:31:96:00:eb:ae:
+         17:ac:6e:c0:d4:1a:8d:0f:36:63:27:02:49:4e:24:cf:d3:80:
+         88:3a:4f:d0:f1:e5:1c:df:2d:8a:ab:ae:8d:48:77:a0:d0:dc:
+         d5:80:1c:a1:3d:0d:49:64:bf:cb:39:84:c9:f3:5d:e0:2d:ba:
+         a0:f2:ac:03:85:44:a1:97:6b:0b:de:ed:a7:49:19:46:b2:18:
+         49:21:62:43:52:36:6f:47:6c:21:6b:5e:41:85:28:71:6c:22:
+         27:35:76:82:ed:ac:ad:d7:fa:9d:4c:7d:6f:44:7e:06:dd:8a:
+         11:32:0c:d9:d0:f6:63:2a:40:ae:0d:5a:df:9e:d7:91:8a:db:
+         2d:95:f3:19:f0:8f:1e:34:e3:b2:31:67:38:74:fd:3f:e6:49:
+         5e:53:eb:88:ae:b1:45:71:0e:67:97:3c:99:4e:c7:ea:1e:02:
+         67:b4:54:ef:4f:10:55:4a:70:c0:eb:41:e4:50:d4:48:5e:70:
+         c5:0f:79:f2:06:3d:35:ea:ce:5d:13:8e:14:65:fc:98:21:16:
+         2d:5d:6d:f8:e0:6b:c7:c6:e4:8a:ca:c9:38:1f:93:27:86:28:
+         ef:96:e7:ad:6c:4a:9e:10:78:48:00:f4:4a:43:dc:87:1d:e3:
+         d3:39:53:68
 -----BEGIN CERTIFICATE-----
-MIIDLjCCApegAwIBAgIJAIgImLMT2ACZMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDI1
-MDA1WhcNMzAxMTE2MDAwMDAwWjBXMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEWMBQGA1UECxMNQXBhY2hlIFB1bHNhcjESMBAGA1UE
-AxMJc3VwZXJVc2VyMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAzUN9
-mED5sFu8rtvAC60mkJbgYjjtaLFwRjveRPkUUYYQ68qQ54jo+ZGF4N21tBS5eOOG
-1VRtaOwUkrT4IlsFPe0xJWUIBYTK5gwhElgyxxpgo0/SSp4oGXxFhACMidzeiuVP
-iJHMpPGBRUx9wv/iwYnGEnOV4ja9266LWmhqkFHeK4hfqmf0qONj3L4Zgsydf+aN
-+4K+IgE9VhM7WwS06MUY5i4N+rpKjejGWqFRmkpi16/dtPzi1c2umWxcYVYL1wwa
-d1z1OmpUtZ4zrKl1KJp2r9B6VwAbkRMx/UKIIUcFEAEvWbvHOtnhWEwbbHG2mO/d
-A4JYozLckKG2ph7hCwIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQf
-Fh1PcGVuU1NMIEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQUU3zV0VKX
-mtbV6uy2DJtDORlz9iwwHwYDVR0jBBgwFoAU1HrND0QbFiklFO2i7xMPp0YJePYw
-DQYJKoZIhvcNAQEFBQADgYEA5AOC/77ffHMqxY99h6uVsSvl90EiTyhUhHrM/nCJ
-D0jlihfhRK0S6aE6x4RV8HwpUgqhq8xbMeWyN3M6jfLx++j2ornvERD4MUOPr84J
-9MuWDtRYQm6Gq7kDGYtKbu9QwH7JCx0rQr/r0AYFhOpaiiJcVvraKp+KspBmjF4B
-h0U=
+MIIDFDCCAfygAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgEwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVzELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYDVQQKEwZBcGFj
+aGUxFjAUBgNVBAsTDUFwYWNoZSBQdWxzYXIxEjAQBgNVBAMTCXN1cGVyVXNlcjCC
+ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAM1DfZhA+bBbvK7bwAutJpCW
+4GI47WixcEY73kT5FFGGEOvKkOeI6PmRheDdtbQUuXjjhtVUbWjsFJK0+CJbBT3t
+MSVlCAWEyuYMIRJYMscaYKNP0kqeKBl8RYQAjInc3orlT4iRzKTxgUVMfcL/4sGJ
+xhJzleI2vduui1poapBR3iuIX6pn9KjjY9y+GYLMnX/mjfuCviIBPVYTO1sEtOjF
+GOYuDfq6So3oxlqhUZpKYtev3bT84tXNrplsXGFWC9cMGndc9TpqVLWeM6ypdSia
+dq/QelcAG5ETMf1CiCFHBRABL1m7xzrZ4VhMG2xxtpjv3QOCWKMy3JChtqYe4QsC
+AwEAAaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEB
+CwUAA4IBAQCWwiMtRtA9Iw6rPbYeMZYA664XrG7A1BqNDzZjJwJJTiTP04CIOk/Q
+8eUc3y2Kq66NSHeg0NzVgByhPQ1JZL/LOYTJ813gLbqg8qwDhUShl2sL3u2nSRlG
+shhJIWJDUjZvR2wha15BhShxbCInNXaC7ayt1/qdTH1vRH4G3YoRMgzZ0PZjKkCu
+DVrfnteRitstlfMZ8I8eNOOyMWc4dP0/5kleU+uIrrFFcQ5nlzyZTsfqHgJntFTv
+TxBVSnDA60HkUNRIXnDFD3nyBj016s5dE44UZfyYIRYtXW344GvHxuSKysk4H5Mn
+hijvluetbEqeEHhIAPRKQ9yHHePTOVNo
 -----END CERTIFICATE-----
diff --git a/pulsar-proxy/src/test/resources/authentication/tls/server-cert.pem b/pulsar-proxy/src/test/resources/authentication/tls/server-cert.pem
index 69ad71cd0cb..c09434c85d2 100644
--- a/pulsar-proxy/src/test/resources/authentication/tls/server-cert.pem
+++ b/pulsar-proxy/src/test/resources/authentication/tls/server-cert.pem
@@ -2,17 +2,17 @@ Certificate:
     Data:
         Version: 3 (0x2)
         Serial Number:
-            88:08:98:b3:13:d8:00:97
-        Signature Algorithm: sha1WithRSAEncryption
-        Issuer: C=US, ST=CA, O=Apache, OU=Pulsar Incubator, CN=localhost
+            61:e6:1b:07:90:6a:4f:f7:cd:46:b9:59:1d:3e:1c:39:0d:f2:5e:02
+        Signature Algorithm: sha256WithRSAEncryption
+        Issuer: CN = CARoot
         Validity
-            Not Before: Feb 17 02:06:21 2018 GMT
-            Not After : Nov 16 00:00:00 2030 GMT
-        Subject: C=US, ST=CA, O=Apache, OU=Apache Pulsar, CN=localhost
+            Not Before: May 30 13:38:24 2022 GMT
+            Not After : May 27 13:38:24 2032 GMT
+        Subject: C = US, ST = CA, O = Apache, OU = Apache Pulsar, CN = localhost
         Subject Public Key Info:
             Public Key Algorithm: rsaEncryption
-            RSA Public Key: (2048 bit)
-                Modulus (2048 bit):
+                RSA Public-Key: (2048 bit)
+                Modulus:
                     00:af:bf:b7:2d:98:ad:9d:f6:da:a3:13:d4:62:0f:
                     98:be:1c:a2:89:22:ba:6f:d5:fd:1f:67:e3:91:03:
                     98:80:81:0e:ed:d8:f6:70:7f:2c:36:68:3d:53:ea:
@@ -33,41 +33,40 @@ Certificate:
                     a7:35
                 Exponent: 65537 (0x10001)
         X509v3 extensions:
-            X509v3 Basic Constraints: 
-                CA:FALSE
-            Netscape Comment: 
-                OpenSSL Generated Certificate
-            X509v3 Subject Key Identifier: 
-                D3:F3:19:AE:74:B1:AF:E7:AF:08:7B:16:72:78:29:87:79:ED:30:8C
-            X509v3 Authority Key Identifier: 
-                keyid:D4:7A:CD:0F:44:1B:16:29:25:14:ED:A2:EF:13:0F:A7:46:09:78:F6
-
-    Signature Algorithm: sha1WithRSAEncryption
-        0f:04:f3:91:f2:87:19:fe:9d:f8:34:5a:24:4a:00:d1:58:bf:
-        1e:b2:77:67:07:bc:78:b5:4b:9a:4b:fd:a1:e5:dc:0e:09:84:
-        9e:59:c4:dd:cf:f7:2e:bf:da:f3:31:36:6b:81:6e:a2:88:76:
-        e4:2e:0b:36:44:82:36:8f:80:93:f4:9e:fc:ed:85:d0:97:da:
-        0f:fb:c9:b9:8b:da:ae:07:3d:4f:82:b7:0c:25:22:63:12:6b:
-        0a:e9:c4:12:a4:5c:ed:11:12:cc:fe:b0:2e:d4:c1:ec:79:01:
-        60:ea:cc:cc:e5:66:cc:57:f6:55:a9:09:4c:63:01:e9:b4:2e:
-        73:a5
+            X509v3 Subject Alternative Name: 
+                DNS:localhost, IP Address:127.0.0.1
+    Signature Algorithm: sha256WithRSAEncryption
+         88:89:d7:52:b3:61:49:73:7d:ee:aa:6f:47:11:cd:52:f1:ef:
+         9a:63:5f:43:a9:4f:66:c8:36:dd:44:24:ba:4f:c3:6c:94:90:
+         85:5e:29:fb:65:cf:03:3b:37:16:5e:88:07:70:97:54:93:f0:
+         f3:09:d7:65:60:09:00:fd:7f:dd:6a:ab:25:3a:30:c4:89:34:
+         43:82:f6:f5:f4:2d:39:3d:21:90:c4:00:27:c5:6a:23:41:20:
+         c6:42:35:56:91:17:fa:31:90:09:6a:4c:e4:a7:53:ae:61:b6:
+         d3:5b:82:71:08:d0:0b:af:34:0f:9b:bd:bc:8c:1c:31:43:43:
+         97:82:9a:ac:2a:53:ca:11:ce:6f:64:ac:86:c1:f0:62:14:aa:
+         c3:dd:15:5b:1c:02:6f:bb:40:87:17:b7:e5:9d:93:9a:51:c9:
+         1e:7a:8c:d1:22:75:44:f1:9d:90:4b:3e:1f:6c:ab:6f:e3:be:
+         cd:c7:15:9d:04:84:4a:1b:a7:ac:64:5d:d7:3e:23:98:b9:49:
+         dd:85:dd:80:4c:46:08:9b:f5:df:eb:19:c8:57:70:ac:43:f9:
+         d6:9c:1b:1b:2a:94:cf:c1:35:56:a2:f4:b1:00:5d:9e:1e:36:
+         54:72:ab:aa:ef:49:b2:f0:dc:cf:5b:22:51:bf:e4:c9:57:dc:
+         d0:48:0d:f2
 -----BEGIN CERTIFICATE-----
-MIIDLjCCApegAwIBAgIJAIgImLMT2ACXMA0GCSqGSIb3DQEBBQUAMFoxCzAJBgNV
-BAYTAlVTMQswCQYDVQQIEwJDQTEPMA0GA1UEChMGQXBhY2hlMRkwFwYDVQQLExBQ
-dWxzYXIgSW5jdWJhdG9yMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTgwMjE3MDIw
-NjIxWhcNMzAxMTE2MDAwMDAwWjBXMQswCQYDVQQGEwJVUzELMAkGA1UECBMCQ0Ex
-DzANBgNVBAoTBkFwYWNoZTEWMBQGA1UECxMNQXBhY2hlIFB1bHNhcjESMBAGA1UE
-AxMJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAr7+3
-LZitnfbaoxPUYg+YvhyiiSK6b9X9H2fjkQOYgIEO7dj2cH8sNmg9U+pYOqbViWZL
-vR5XcRNtSxHlQKV2hCSSQFiAlskfLMRV66N5c3BcN5qJ7S+6a+OCfGlKAlSLgV48
-v0yKy+osXoPntxAIX4JYo4nR2pK6KijuMCg/W64QcZbH4RLFsBqtRG9EOhFKmjwP
-jQaAezTvP2z0XsVEVB7I3ceAhYDZaObGUwN34f4YYQd3BUztWbxdQThq712hsmCY
-1EgolQKKDv3PexvSEcwQDFBz18w4bIPdeSaqkMibhIa8WeliafSYG8SAeH6gGoGd
-0uFm3cTM/GMErOynNQIDAQABo3sweTAJBgNVHRMEAjAAMCwGCWCGSAGG+EIBDQQf
-Fh1PcGVuU1NMIEdlbmVyYXRlZCBDZXJ0aWZpY2F0ZTAdBgNVHQ4EFgQU0/MZrnSx
-r+evCHsWcngph3ntMIwwHwYDVR0jBBgwFoAU1HrND0QbFiklFO2i7xMPp0YJePYw
-DQYJKoZIhvcNAQEFBQADgYEADwTzkfKHGf6d+DRaJEoA0Vi/HrJ3Zwe8eLVLmkv9
-oeXcDgmEnlnE3c/3Lr/a8zE2a4Fuooh25C4LNkSCNo+Ak/Se/O2F0JfaD/vJuYva
-rgc9T4K3DCUiYxJrCunEEqRc7RESzP6wLtTB7HkBYOrMzOVmzFf2VakJTGMB6bQu
-c6U=
+MIIDFDCCAfygAwIBAgIUYeYbB5BqT/fNRrlZHT4cOQ3yXgIwDQYJKoZIhvcNAQEL
+BQAwETEPMA0GA1UEAwwGQ0FSb290MB4XDTIyMDUzMDEzMzgyNFoXDTMyMDUyNzEz
+MzgyNFowVzELMAkGA1UEBhMCVVMxCzAJBgNVBAgTAkNBMQ8wDQYDVQQKEwZBcGFj
+aGUxFjAUBgNVBAsTDUFwYWNoZSBQdWxzYXIxEjAQBgNVBAMTCWxvY2FsaG9zdDCC
+ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAK+/ty2YrZ322qMT1GIPmL4c
+ookium/V/R9n45EDmICBDu3Y9nB/LDZoPVPqWDqm1YlmS70eV3ETbUsR5UCldoQk
+kkBYgJbJHyzEVeujeXNwXDeaie0vumvjgnxpSgJUi4FePL9MisvqLF6D57cQCF+C
+WKOJ0dqSuioo7jAoP1uuEHGWx+ESxbAarURvRDoRSpo8D40GgHs07z9s9F7FRFQe
+yN3HgIWA2WjmxlMDd+H+GGEHdwVM7Vm8XUE4au9dobJgmNRIKJUCig79z3sb0hHM
+EAxQc9fMOGyD3XkmqpDIm4SGvFnpYmn0mBvEgHh+oBqBndLhZt3EzPxjBKzspzUC
+AwEAAaMeMBwwGgYDVR0RBBMwEYIJbG9jYWxob3N0hwR/AAABMA0GCSqGSIb3DQEB
+CwUAA4IBAQCIiddSs2FJc33uqm9HEc1S8e+aY19DqU9myDbdRCS6T8NslJCFXin7
+Zc8DOzcWXogHcJdUk/DzCddlYAkA/X/daqslOjDEiTRDgvb19C05PSGQxAAnxWoj
+QSDGQjVWkRf6MZAJakzkp1OuYbbTW4JxCNALrzQPm728jBwxQ0OXgpqsKlPKEc5v
+ZKyGwfBiFKrD3RVbHAJvu0CHF7flnZOaUckeeozRInVE8Z2QSz4fbKtv477NxxWd
+BIRKG6esZF3XPiOYuUndhd2ATEYIm/Xf6xnIV3CsQ/nWnBsbKpTPwTVWovSxAF2e
+HjZUcquq70my8NzPWyJRv+TJV9zQSA3y
 -----END CERTIFICATE-----


[pulsar] 03/10: [refactor][proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode (#14713)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 58ad46722c25301a995f798eeec92366ed01ffc6
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Tue Mar 22 15:58:43 2022 +0200

    [refactor][proxy] Refactor Proxy code and fix connection stalling by switching to auto read mode (#14713)
    
    Refactor Proxy code to make it easier to understand and maintain. In addition, switch to use auto read mode since the proxies connections seem to stall in some cases since the proxied connection doesn't use Netty's auto read mode and the read handling doesn't seem complete.
    
    Currently, the proxy calls `.read()` when a message is written to the connection. There might be more messages flowing in the other direction and it could result in a blocked connection with the current solution that doesn't use Netty's auto read mode.
    
    Currently auto read is disabled in DirectProxyHandler for the connection between the proxy and the broker:
    https://github.com/apache/pulsar/blob/a26905371749798ec5288fb07a69978a36aacfaa/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java#L112
    
    - replace broker host parsing with a simple solution
    - pass remote host name to ProxyBackendHandler in the constructor
    - rename "targetBrokerUrl" to "brokerHostAndPort" since the "targetBrokerUrl" is really "hostname:port" string
    - move HA proxy message handling to ProxyBackendHandle and extract the logic to a method
    - remove the static "inboundOutboundChannelMap" which was used for log level 2
      - make it obsolete by passing the peer channel id to ParserProxyHandler
     - Enable auto read in proxy and remove `ctx.read()` / `channel.read()` calls
    - prepare for IPv6 support (reported as #14732) by improving the `host:port` parsing (pick last `:` since IPv6 address might contains multiple `:` characters)
    - Handle backpressure properly by switching auto read off when channel writability changes
      - change auto read of the proxy-broker connection based on the writability of the client-proxy connection
      - change auto read of the client-proxy connection based on the writability of the proxy-broker connection
    - Consistently handle write errors by delegating exception handling to exceptionCaught method by using `.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE)`
    
    (cherry picked from commit a1037c75c9c305369799b71d840fa73cb198b293)
    (cherry picked from commit 1e55685329e99f08fa50f4c99cf7f7bed9ffe2e0)
    (cherry picked from commit 9673d6cffa6e0ba92ab538cc19f1f91b6e4bf4ad)
---
 .../pulsar/proxy/server/BrokerProxyValidator.java  |   2 +-
 .../pulsar/proxy/server/DirectProxyHandler.java    | 180 ++++++++++-----------
 .../pulsar/proxy/server/ParserProxyHandler.java    | 114 +++++++------
 .../pulsar/proxy/server/ProxyConnection.java       |  70 ++++----
 .../proxy/server/BrokerProxyValidatorTest.java     |  20 +++
 5 files changed, 194 insertions(+), 192 deletions(-)

diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerProxyValidator.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerProxyValidator.java
index 3c49653cff9..b255b9c4ebd 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerProxyValidator.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerProxyValidator.java
@@ -113,7 +113,7 @@ public class BrokerProxyValidator {
     }
 
     public CompletableFuture<InetSocketAddress> resolveAndCheckTargetAddress(String hostAndPort) {
-        int pos = hostAndPort.indexOf(':');
+        int pos = hostAndPort.lastIndexOf(':');
         String host = hostAndPort.substring(0, pos);
         int port = Integer.parseInt(hostAndPort.substring(pos + 1));
         if (!isPortAllowed(port)) {
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 1587b72c831..8fa7787215d 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
@@ -21,15 +21,14 @@ package org.apache.pulsar.proxy.server;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
-
 import io.netty.bootstrap.Bootstrap;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelId;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.socket.SocketChannel;
@@ -41,22 +40,12 @@ import io.netty.handler.codec.haproxy.HAProxyProxiedProtocol;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.handler.timeout.ReadTimeoutHandler;
 import io.netty.util.CharsetUtil;
-import io.netty.util.concurrent.Future;
-import io.netty.util.concurrent.FutureListener;
-
 import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.Arrays;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
-
 import javax.net.ssl.SSLSession;
-
 import lombok.Getter;
-
 import org.apache.pulsar.PulsarVersion;
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
@@ -76,11 +65,11 @@ public class DirectProxyHandler {
 
     @Getter
     private final Channel inboundChannel;
+    private final ProxyConnection proxyConnection;
     @Getter
     Channel outboundChannel;
     @Getter
     private final Rate inboundChannelRequestsRate;
-    protected static Map<ChannelId, ChannelId> inboundOutboundChannelMap = new ConcurrentHashMap<>();
     private final String originalPrincipal;
     private final AuthData clientAuthData;
     private final String clientAuthMethod;
@@ -91,12 +80,13 @@ public class DirectProxyHandler {
     private final ProxyService service;
     private final Runnable onHandshakeCompleteAction;
 
-    public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection, String targetBrokerUrl,
+    public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection, String brokerHostAndPort,
                               InetSocketAddress targetBrokerAddress, int protocolVersion,
                               Supplier<SslHandler> sslHandlerSupplier) {
         this.service = service;
         this.authentication = proxyConnection.getClientAuthentication();
         this.inboundChannel = proxyConnection.ctx().channel();
+        this.proxyConnection = proxyConnection;
         this.inboundChannelRequestsRate = new Rate();
         this.originalPrincipal = proxyConnection.clientAuthRole;
         this.clientAuthData = proxyConnection.clientAuthData;
@@ -114,7 +104,18 @@ public class DirectProxyHandler {
         if (brokerProxyConnectTimeoutMs > 0) {
             b.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, brokerProxyConnectTimeoutMs);
         }
-        b.group(inboundChannel.eventLoop()).channel(inboundChannel.getClass()).option(ChannelOption.AUTO_READ, false);
+        b.group(inboundChannel.eventLoop())
+                .channel(inboundChannel.getClass());
+
+        String remoteHost;
+        try {
+            remoteHost = parseHost(brokerHostAndPort);
+        } catch (IllegalArgumentException e) {
+            log.warn("[{}] Failed to parse broker host '{}'", inboundChannel, brokerHostAndPort, e);
+            inboundChannel.close();
+            return;
+        }
+
         b.handler(new ChannelInitializer<SocketChannel>() {
             @Override
             protected void initChannel(SocketChannel ch) {
@@ -128,65 +129,58 @@ public class DirectProxyHandler {
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
-                ch.pipeline().addLast("proxyOutboundHandler", new ProxyBackendHandler(config, protocolVersion));
+                ch.pipeline().addLast("proxyOutboundHandler",
+                        new ProxyBackendHandler(config, protocolVersion, remoteHost));
             }
         });
 
-        URI targetBroker;
-        try {
-            // targetBrokerUrl is coming in the "hostname:6650" form, so we need
-            // to extract host and port
-            targetBroker = new URI("pulsar://" + targetBrokerUrl);
-        } catch (URISyntaxException e) {
-            log.warn("[{}] Failed to parse broker url '{}'", inboundChannel, targetBrokerUrl, e);
-            inboundChannel.close();
-            return;
-        }
-
         ChannelFuture f = b.connect(targetBrokerAddress);
         outboundChannel = f.channel();
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
                 log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
-                        targetBrokerAddress, targetBrokerUrl, future.cause());
+                        targetBrokerAddress, brokerHostAndPort, future.cause());
                 inboundChannel.close();
                 return;
             }
-            final ProxyBackendHandler cnx = (ProxyBackendHandler) outboundChannel.pipeline()
-                    .get("proxyOutboundHandler");
-            cnx.setRemoteHostName(targetBroker.getHost());
-
-            // if enable full parsing feature
-            if (service.getProxyLogLevel() == 2) {
-                //Set a map between inbound and outbound,
-                //so can find inbound by outbound or find outbound by inbound
-                inboundOutboundChannelMap.put(outboundChannel.id() , inboundChannel.id());
-            }
+        });
+    }
 
-            if (config.isHaProxyProtocolEnabled()) {
-                if (proxyConnection.hasHAProxyMessage()) {
-                    outboundChannel.writeAndFlush(encodeProxyProtocolMessage(proxyConnection.getHAProxyMessage()));
-                } else {
-                    if (inboundChannel.remoteAddress() instanceof InetSocketAddress) {
-                        InetSocketAddress clientAddress = (InetSocketAddress) inboundChannel.remoteAddress();
-                        String sourceAddress = clientAddress.getAddress().getHostAddress();
-                        int sourcePort = clientAddress.getPort();
-                        if (outboundChannel.localAddress() instanceof InetSocketAddress) {
-                            InetSocketAddress proxyAddress = (InetSocketAddress) inboundChannel.remoteAddress();
-                            String destinationAddress = proxyAddress.getAddress().getHostAddress();
-                            int destinationPort = proxyAddress.getPort();
-                            HAProxyMessage msg = new HAProxyMessage(HAProxyProtocolVersion.V1, HAProxyCommand.PROXY,
-                                    HAProxyProxiedProtocol.TCP4, sourceAddress, destinationAddress, sourcePort, destinationPort);
-                            outboundChannel.writeAndFlush(encodeProxyProtocolMessage(msg));
-                            msg.release();
-                        }
-                    }
-                }
+    private static String parseHost(String brokerPortAndHost) {
+        int pos = brokerPortAndHost.lastIndexOf(':');
+        if (pos > 0) {
+            return brokerPortAndHost.substring(0, pos);
+        } else {
+            throw new IllegalArgumentException("Illegal broker host:port '" + brokerPortAndHost + "'");
+        }
+    }
+
+    private void writeHAProxyMessage() {
+        if (proxyConnection.hasHAProxyMessage()) {
+            outboundChannel.writeAndFlush(encodeProxyProtocolMessage(proxyConnection.getHAProxyMessage()))
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+        } else {
+            if (inboundChannel.remoteAddress() instanceof InetSocketAddress
+                    && outboundChannel.localAddress() instanceof InetSocketAddress) {
+                InetSocketAddress clientAddress = (InetSocketAddress) inboundChannel.remoteAddress();
+                String sourceAddress = clientAddress.getAddress().getHostAddress();
+                int sourcePort = clientAddress.getPort();
+                InetSocketAddress proxyAddress = (InetSocketAddress) inboundChannel.remoteAddress();
+                String destinationAddress = proxyAddress.getAddress().getHostAddress();
+                int destinationPort = proxyAddress.getPort();
+                HAProxyMessage msg = new HAProxyMessage(HAProxyProtocolVersion.V1, HAProxyCommand.PROXY,
+                        HAProxyProxiedProtocol.TCP4, sourceAddress, destinationAddress, sourcePort,
+                        destinationPort);
+                outboundChannel.writeAndFlush(encodeProxyProtocolMessage(msg))
+                        .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+                msg.release();
             }
-        });
+        }
     }
 
+
+
     private ByteBuf encodeProxyProtocolMessage(HAProxyMessage msg) {
         // Max length of v1 version proxy protocol message is 108
         ByteBuf out = Unpooled.buffer(108);
@@ -218,30 +212,45 @@ public class DirectProxyHandler {
         Init, HandshakeCompleted
     }
 
-    public class ProxyBackendHandler extends PulsarDecoder implements FutureListener<Void> {
+    public class ProxyBackendHandler extends PulsarDecoder {
 
         private BackendState state = BackendState.Init;
-        private String remoteHostName;
+        private final String remoteHostName;
         protected ChannelHandlerContext ctx;
         private final ProxyConfiguration config;
         private final int protocolVersion;
 
-        public ProxyBackendHandler(ProxyConfiguration config, int protocolVersion) {
+        public ProxyBackendHandler(ProxyConfiguration config, int protocolVersion, String remoteHostName) {
             this.config = config;
             this.protocolVersion = protocolVersion;
+            this.remoteHostName = remoteHostName;
         }
 
         @Override
         public void channelActive(ChannelHandlerContext ctx) throws Exception {
             this.ctx = ctx;
+
+            if (config.isHaProxyProtocolEnabled()) {
+                writeHAProxyMessage();
+            }
+
             // Send the Connect command to broker
             authenticationDataProvider = authentication.getAuthData(remoteHostName);
             AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
             ByteBuf command;
             command = Commands.newConnect(authentication.getAuthMethodName(), authData, protocolVersion, "Pulsar proxy",
                     null /* target broker */, originalPrincipal, clientAuthData, clientAuthMethod);
-            outboundChannel.writeAndFlush(command);
-            outboundChannel.read();
+            outboundChannel.writeAndFlush(command)
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+        }
+
+        @Override
+        public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+            // handle backpressure
+            // stop/resume reading input from connection between the client and the proxy
+            // when the writability of the connection between the proxy and the broker changes
+            inboundChannel.config().setAutoRead(ctx.channel().isWritable());
+            super.channelWritabilityChanged(ctx);
         }
 
         @Override
@@ -262,7 +271,8 @@ public class DirectProxyHandler {
                 if (msg instanceof ByteBuf) {
                     ProxyService.bytesCounter.inc(((ByteBuf) msg).readableBytes());
                 }
-                inboundChannel.writeAndFlush(msg).addListener(this);
+                inboundChannel.writeAndFlush(msg)
+                        .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
                 break;
 
             default:
@@ -301,26 +311,13 @@ public class DirectProxyHandler {
                     log.debug("{} Mutual auth {}", ctx.channel(), authentication.getAuthMethodName());
                 }
 
-                outboundChannel.writeAndFlush(request);
-                outboundChannel.read();
+                outboundChannel.writeAndFlush(request)
+                        .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
             } catch (Exception e) {
                 log.error("Error mutual verify", e);
             }
         }
 
-        @Override
-        public void operationComplete(Future<Void> future) {
-            // This is invoked when the write operation on the paired connection
-            // is completed
-            if (future.isSuccess()) {
-                outboundChannel.read();
-            } else {
-                log.warn("[{}] [{}] Failed to write on proxy connection. Closing both connections.", inboundChannel,
-                        outboundChannel, future.cause());
-                inboundChannel.close();
-            }
-        }
-
         @Override
         protected void messageReceived() {
             // no-op
@@ -350,18 +347,7 @@ public class DirectProxyHandler {
             int maxMessageSize =
                     connected.hasMaxMessageSize() ? connected.getMaxMessageSize() : Commands.INVALID_MAX_MESSAGE_SIZE;
             inboundChannel.writeAndFlush(Commands.newConnected(connected.getProtocolVersion(), maxMessageSize))
-                    .addListener(future -> {
-                        if (future.isSuccess()) {
-                            // Start reading from both connections
-                            inboundChannel.read();
-                            outboundChannel.read();
-                        } else {
-                            log.warn("[{}] [{}] Failed to write to inbound connection. Closing both connections.",
-                                    inboundChannel,
-                                    outboundChannel, future.cause());
-                            inboundChannel.close();
-                        }
-                    });
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
         }
 
         private void startDirectProxying(CommandConnected connected) {
@@ -390,20 +376,20 @@ public class DirectProxyHandler {
                     inboundChannel.pipeline().addBefore("handler", "inboundParser",
                             new ParserProxyHandler(service, inboundChannel,
                                     ParserProxyHandler.FRONTEND_CONN,
-                                    connected.getMaxMessageSize()));
+                                    connected.getMaxMessageSize(), outboundChannel.id()));
                     outboundChannel.pipeline().addBefore("proxyOutboundHandler", "outboundParser",
                             new ParserProxyHandler(service, outboundChannel,
                                     ParserProxyHandler.BACKEND_CONN,
-                                    connected.getMaxMessageSize()));
+                                    connected.getMaxMessageSize(), inboundChannel.id()));
                 } else {
                     inboundChannel.pipeline().addBefore("handler", "inboundParser",
                             new ParserProxyHandler(service, inboundChannel,
                                     ParserProxyHandler.FRONTEND_CONN,
-                                    Commands.DEFAULT_MAX_MESSAGE_SIZE));
+                                    Commands.DEFAULT_MAX_MESSAGE_SIZE, outboundChannel.id()));
                     outboundChannel.pipeline().addBefore("proxyOutboundHandler", "outboundParser",
                             new ParserProxyHandler(service, outboundChannel,
                                     ParserProxyHandler.BACKEND_CONN,
-                                    Commands.DEFAULT_MAX_MESSAGE_SIZE));
+                                    Commands.DEFAULT_MAX_MESSAGE_SIZE, inboundChannel.id()));
                 }
             }
         }
@@ -419,10 +405,6 @@ public class DirectProxyHandler {
             ctx.close();
         }
 
-        public void setRemoteHostName(String remoteHostName) {
-            this.remoteHostName = remoteHostName;
-        }
-
         private boolean verifyTlsHostName(String hostname, ChannelHandlerContext ctx) {
             ChannelHandler sslHandler = ctx.channel().pipeline().get("tls");
 
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java
index 32f4ba73c43..77a6404c7d2 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java
@@ -19,10 +19,19 @@
 
 package org.apache.pulsar.proxy.server;
 
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufUtil;
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelId;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.pulsar.common.api.proto.PulsarApi;
 import org.apache.pulsar.common.api.raw.MessageParser;
@@ -33,59 +42,62 @@ import org.apache.pulsar.proxy.stats.TopicStats;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufUtil;
-import io.netty.buffer.CompositeByteBuf;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-
 
 public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
 
 
-    private Channel channel;
+    private final Channel channel;
     //inbound
     protected static final String FRONTEND_CONN = "frontendconn";
     //outbound
     protected static final String BACKEND_CONN = "backendconn";
 
-    private String connType;
+    private final String connType;
 
-    private int maxMessageSize;
+    private final int maxMessageSize;
+    private final ChannelId peerChannelId;
     private final ProxyService service;
 
 
-    //producerid+channelid as key
-    //or consumerid+channelid as key
-    private static Map<String, String> producerHashMap = new ConcurrentHashMap<>();
-    private static Map<String, String> consumerHashMap = new ConcurrentHashMap<>();
+    /**
+     * producerid + channelid as key.
+     */
+    private static final Map<String, String> producerHashMap = new ConcurrentHashMap<>();
+
+    /**
+     * consumerid + channelid as key.
+     */
+    private static final Map<String, String> consumerHashMap = new ConcurrentHashMap<>();
 
-    public ParserProxyHandler(ProxyService service, Channel channel, String type, int maxMessageSize) {
+    public ParserProxyHandler(ProxyService service, Channel channel, String type, int maxMessageSize,
+                              ChannelId peerChannelId) {
         this.service = service;
         this.channel = channel;
         this.connType = type;
         this.maxMessageSize = maxMessageSize;
+        this.peerChannelId = peerChannelId;
     }
 
     private void logging(Channel conn, PulsarApi.BaseCommand.Type cmdtype, String info, List<RawMessage> messages) throws Exception{
 
         if (messages != null) {
             // lag
-            for (int i=0; i<messages.size(); i++) {
-                info = info + "["+ (System.currentTimeMillis() - messages.get(i).getPublishTime()) + "] " + new String(ByteBufUtil.getBytes((messages.get(i)).getData()), "UTF8");
+            StringBuilder infoBuilder = new StringBuilder(info);
+            for (RawMessage message : messages) {
+                infoBuilder.append("[").append(System.currentTimeMillis() - message.getPublishTime()).append("] ")
+                        .append(new String(ByteBufUtil.getBytes(message.getData()), StandardCharsets.UTF_8));
             }
+            info = infoBuilder.toString();
         }
         // log conn format is like from source to target
         switch (this.connType) {
             case ParserProxyHandler.FRONTEND_CONN:
-                log.info(ParserProxyHandler.FRONTEND_CONN + ":{} cmd:{} msg:{}", "[" + conn.remoteAddress() + conn.localAddress() + "]", cmdtype, info);
+                log.info(ParserProxyHandler.FRONTEND_CONN + ":{} cmd:{} msg:{}", "[" + conn.remoteAddress()
+                        + conn.localAddress() + "]", cmdtype, info);
                 break;
             case ParserProxyHandler.BACKEND_CONN:
-                log.info(ParserProxyHandler.BACKEND_CONN + ":{} cmd:{} msg:{}", "[" + conn.localAddress() + conn.remoteAddress() + "]", cmdtype, info);
+                log.info(ParserProxyHandler.BACKEND_CONN + ":{} cmd:{} msg:{}", "[" + conn.localAddress()
+                        + conn.remoteAddress() + "]", cmdtype, info);
                 break;
         }
     }
@@ -93,9 +105,9 @@ public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
     public void channelRead(ChannelHandlerContext ctx, Object msg) {
         PulsarApi.BaseCommand cmd = null;
         PulsarApi.BaseCommand.Builder cmdBuilder = null;
-        TopicName topicName ;
-        List<RawMessage> messages = Lists.newArrayList();
-        ByteBuf buffer = (ByteBuf)(msg);
+        TopicName topicName;
+        List<RawMessage> messages = new ArrayList<>();
+        ByteBuf buffer = (ByteBuf) (msg);
 
         try {
             buffer.markReaderIndex();
@@ -113,20 +125,23 @@ public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
 
             switch (cmd.getType()) {
                 case PRODUCER:
-                    ParserProxyHandler.producerHashMap.put(String.valueOf(cmd.getProducer().getProducerId()) + "," + String.valueOf(ctx.channel().id()), cmd.getProducer().getTopic());
+                    ParserProxyHandler.producerHashMap.put(cmd.getProducer().getProducerId() + "," + ctx.channel().id(),
+                            cmd.getProducer().getTopic());
 
-                    logging(ctx.channel() , cmd.getType() , "{producer:" + cmd.getProducer().getProducerName() + ",topic:" + cmd.getProducer().getTopic() + "}", null);
+                    logging(ctx.channel(), cmd.getType(), "{producer:" + cmd.getProducer().getProducerName()
+                            + ",topic:" + cmd.getProducer().getTopic() + "}", null);
                     break;
 
                 case SEND:
                     if (service.getProxyLogLevel() != 2) {
-                        logging(ctx.channel() , cmd.getType() , "", null);
+                        logging(ctx.channel(), cmd.getType(), "", null);
                         break;
                     }
-                    topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(String.valueOf(cmd.getSend().getProducerId()) + "," + String.valueOf(ctx.channel().id())));
+                    topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(cmd.getSend().getProducerId() + ","
+                            + ctx.channel().id()));
                     MutableLong msgBytes = new MutableLong(0);
-                    MessageParser.parseMessage(topicName,  -1L,
-                            -1L,buffer,(message) -> {
+                    MessageParser.parseMessage(topicName, -1L,
+                            -1L, buffer, (message) -> {
                                 messages.add(message);
                                 msgBytes.add(message.getData().readableBytes());
                             }, maxMessageSize);
@@ -134,42 +149,43 @@ public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
                     TopicStats topicStats = this.service.getTopicStats().computeIfAbsent(topicName.toString(),
                         topic -> new TopicStats());
                     topicStats.getMsgInRate().recordMultipleEvents(messages.size(), msgBytes.longValue());
-                    logging(ctx.channel() , cmd.getType() , "" , messages);
+                    logging(ctx.channel(), cmd.getType(), "", messages);
                     break;
 
                 case SUBSCRIBE:
-                    ParserProxyHandler.consumerHashMap.put(String.valueOf(cmd.getSubscribe().getConsumerId()) + "," + String.valueOf(ctx.channel().id()) , cmd.getSubscribe().getTopic());
+                    ParserProxyHandler.consumerHashMap.put(cmd.getSubscribe().getConsumerId() + ","
+                                    + ctx.channel().id(), cmd.getSubscribe().getTopic());
 
-                    logging(ctx.channel() , cmd.getType() , "{consumer:" + cmd.getSubscribe().getConsumerName() + ",topic:" + cmd.getSubscribe().getTopic() + "}" , null);
+                    logging(ctx.channel(), cmd.getType(), "{consumer:" + cmd.getSubscribe().getConsumerName()
+                            + ",topic:" + cmd.getSubscribe().getTopic() + "}", null);
                     break;
 
                 case MESSAGE:
                     if (service.getProxyLogLevel() != 2) {
-                        logging(ctx.channel() , cmd.getType() , "" , null);
+                        logging(ctx.channel(), cmd.getType(), "", null);
                         break;
                     }
-                    topicName = TopicName.get(ParserProxyHandler.consumerHashMap.get(String.valueOf(cmd.getMessage().getConsumerId()) + "," + DirectProxyHandler.inboundOutboundChannelMap.get(ctx.channel().id())));
+                    topicName = TopicName.get(ParserProxyHandler.consumerHashMap.get(cmd.getMessage().getConsumerId()
+                            + "," + peerChannelId));
                     msgBytes = new MutableLong(0);
-                    MessageParser.parseMessage(topicName,  -1L,
-                                -1L,buffer,(message) -> {
-                                    messages.add(message);
-                                    msgBytes.add(message.getData().readableBytes());
-                                }, maxMessageSize);
+                    MessageParser.parseMessage(topicName, -1L,
+                            -1L, buffer, (message) -> {
+                                messages.add(message);
+                                msgBytes.add(message.getData().readableBytes());
+                            }, maxMessageSize);
                     // update topic stats
                     topicStats = this.service.getTopicStats().computeIfAbsent(topicName.toString(),
                             topic -> new TopicStats());
                     topicStats.getMsgOutRate().recordMultipleEvents(messages.size(), msgBytes.longValue());
-                    logging(ctx.channel() , cmd.getType() , "" , messages);
+                    logging(ctx.channel(), cmd.getType(), "", messages);
                     break;
 
                  default:
-                    logging(ctx.channel() , cmd.getType() , "" , null);
+                    logging(ctx.channel(), cmd.getType(), "", null);
                     break;
             }
         } catch (Exception e){
-
-            log.error("{},{},{}" , e.getMessage() , e.getStackTrace() ,  e.getCause());
-
+            log.error("channelRead error ", e);
         } finally {
 
             if (cmdBuilder != null) {
@@ -185,8 +201,8 @@ public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
             ByteBuf totalSizeBuf = Unpooled.buffer(4);
             totalSizeBuf.writeInt(buffer.readableBytes());
             CompositeByteBuf compBuf = Unpooled.compositeBuffer();
-            compBuf.addComponents(totalSizeBuf,buffer);
-            compBuf.writerIndex(totalSizeBuf.capacity()+buffer.capacity());
+            compBuf.addComponents(totalSizeBuf, buffer);
+            compBuf.writerIndex(totalSizeBuf.capacity() + buffer.capacity());
 
             // Release mssages
             messages.forEach(RawMessage::release);
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index f1e45f07e81..754e33f96a8 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -20,6 +20,7 @@ package org.apache.pulsar.proxy.server;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import io.netty.channel.ChannelFutureListener;
 import io.netty.handler.codec.haproxy.HAProxyMessage;
 import java.net.SocketAddress;
 import java.util.Collections;
@@ -69,7 +70,7 @@ import lombok.Getter;
  * Handles incoming discovery request from client and sends appropriate response back to client
  *
  */
-public class ProxyConnection extends PulsarHandler implements FutureListener<Void> {
+public class ProxyConnection extends PulsarHandler {
     private static final Logger LOG = LoggerFactory.getLogger(ProxyConnection.class);
     // ConnectionPool is used by the proxy to issue lookup requests
     private ConnectionPool connectionPool;
@@ -189,6 +190,17 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         ctx.close();
     }
 
+    @Override
+    public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
+        if (directProxyHandler != null && directProxyHandler.outboundChannel != null) {
+            // handle backpressure
+            // stop/resume reading input from connection between the proxy and the broker
+            // when the writability of the connection between the client and the proxy changes
+            directProxyHandler.outboundChannel.config().setAutoRead(ctx.channel().isWritable());
+        }
+        super.channelWritabilityChanged(ctx);
+    }
+
     @Override
     public void channelRead(final ChannelHandlerContext ctx, Object msg) throws Exception {
         if (msg instanceof HAProxyMessage) {
@@ -212,7 +224,8 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
                 directProxyHandler.getInboundChannelRequestsRate().recordEvent(bytes);
                 ProxyService.bytesCounter.inc(bytes);
             }
-            directProxyHandler.outboundChannel.writeAndFlush(msg).addListener(this);
+            directProxyHandler.outboundChannel.writeAndFlush(msg)
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
             break;
 
         default:
@@ -220,18 +233,6 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         }
     }
 
-    @Override
-    public void operationComplete(Future<Void> future) {
-        // This is invoked when the write operation on the paired connection is
-        // completed
-        if (future.isSuccess()) {
-            ctx.read();
-        } else {
-            LOG.warn("[{}] Error in writing to inbound channel. Closing", remoteAddress, future.cause());
-            directProxyHandler.outboundChannel.close();
-        }
-    }
-
     private synchronized void completeConnect(AuthData clientData) throws PulsarClientException {
         if (service.getConfiguration().isAuthenticationEnabled()) {
             if (service.getConfiguration().isForwardAuthorizationCredentials()) {
@@ -269,18 +270,18 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
                 ctx()
                         .writeAndFlush(
                                 Commands.newError(-1, ServerError.ServiceNotReady, "Target broker isn't available."))
-                        .addListener(future -> ctx().close());
+                        .addListener(ChannelFutureListener.CLOSE);
                 return;
             }
 
             brokerProxyValidator.resolveAndCheckTargetAddress(proxyToBrokerUrl)
-                    .thenAccept(address -> ctx().executor().submit(() -> {
+                    .thenAcceptAsync(address -> {
                         // Client already knows which broker to connect. Let's open a
                         // connection there and just pass bytes in both directions
                         state = State.ProxyConnectionToBroker;
                         directProxyHandler = new DirectProxyHandler(service, this, proxyToBrokerUrl, address,
                                 protocolVersionToAdvertise, sslHandlerSupplier);
-                    }))
+                    }, ctx.executor())
                     .exceptionally(throwable -> {
                         if (throwable instanceof TargetAddressDeniedException
                                 || throwable.getCause() instanceof TargetAddressDeniedException) {
@@ -299,7 +300,7 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
                                 .writeAndFlush(
                                         Commands.newError(-1, ServerError.ServiceNotReady,
                                                 "Target broker cannot be validated."))
-                                .addListener(future -> ctx().close());
+                                .addListener(ChannelFutureListener.CLOSE);
                         return null;
                     });
         } else {
@@ -308,7 +309,8 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
             // partitions metadata lookups
             state = State.ProxyLookupRequests;
             lookupProxyHandler = new LookupProxyHandler(service, this);
-            ctx.writeAndFlush(Commands.newConnected(protocolVersionToAdvertise));
+            ctx.writeAndFlush(Commands.newConnected(protocolVersionToAdvertise))
+                    .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
         }
     }
 
@@ -327,7 +329,8 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         }
 
         // auth not complete, continue auth with client side.
-        ctx.writeAndFlush(Commands.newAuthChallenge(authMethod, brokerData, protocolVersionToAdvertise));
+        ctx.writeAndFlush(Commands.newAuthChallenge(authMethod, brokerData, protocolVersionToAdvertise))
+                .addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
         if (LOG.isDebugEnabled()) {
             LOG.debug("[{}] Authentication in progress client by method {}.",
                 remoteAddress, authMethod);
@@ -406,8 +409,8 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
             doAuthentication(clientData);
         } catch (Exception e) {
             LOG.warn("[{}] Unable to authenticate: ", remoteAddress, e);
-            ctx.writeAndFlush(Commands.newError(-1, ServerError.AuthenticationError, "Failed to authenticate"));
-            close();
+            ctx.writeAndFlush(Commands.newError(-1, ServerError.AuthenticationError, "Failed to authenticate"))
+                    .addListener(ChannelFutureListener.CLOSE);
         }
     }
 
@@ -428,8 +431,8 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         } catch (Exception e) {
             String msg = "Unable to handleAuthResponse";
             LOG.warn("[{}] {} ", remoteAddress, msg, e);
-            ctx.writeAndFlush(Commands.newError(-1, ServerError.AuthenticationError, msg));
-            close();
+            ctx.writeAndFlush(Commands.newError(-1, ServerError.AuthenticationError, msg))
+                    .addListener(ChannelFutureListener.CLOSE);
         }
     }
 
@@ -462,25 +465,6 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         lookupProxyHandler.handleLookup(lookup);
     }
 
-    private synchronized void close() {
-        if (state != State.Closed) {
-            state = State.Closed;
-            if (directProxyHandler != null && directProxyHandler.outboundChannel != null) {
-                directProxyHandler.outboundChannel.close();
-                directProxyHandler = null;
-            }
-            if (connectionPool != null) {
-                try {
-                    connectionPool.close();
-                    connectionPool = null;
-                } catch (Exception e) {
-                    LOG.error("Error closing connection pool", e);
-                }
-            }
-            ctx.close();
-        }
-    }
-
     ClientConfigurationData createClientConfiguration()
             throws PulsarClientException.UnsupportedAuthenticationException {
         ClientConfigurationData clientConf = new ClientConfigurationData();
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/BrokerProxyValidatorTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/BrokerProxyValidatorTest.java
index 8e457554cf5..fba3c36e266 100644
--- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/BrokerProxyValidatorTest.java
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/BrokerProxyValidatorTest.java
@@ -90,6 +90,26 @@ public class BrokerProxyValidatorTest {
         brokerProxyValidator.resolveAndCheckTargetAddress("myhost.mydomain:6650").get();
     }
 
+    @Test
+    public void shouldAllowIPv6Address() throws Exception {
+        BrokerProxyValidator brokerProxyValidator = new BrokerProxyValidator(
+                createMockedAddressResolver("fd4d:801b:73fa:abcd:0000:0000:0000:0001"),
+                "*"
+                , "fd4d:801b:73fa:abcd::/64"
+                , "6650");
+        brokerProxyValidator.resolveAndCheckTargetAddress("myhost.mydomain:6650").get();
+    }
+
+    @Test
+    public void shouldAllowIPv6AddressNumeric() throws Exception {
+        BrokerProxyValidator brokerProxyValidator = new BrokerProxyValidator(
+                createMockedAddressResolver("fd4d:801b:73fa:abcd:0000:0000:0000:0001"),
+                "*"
+                , "fd4d:801b:73fa:abcd::/64"
+                , "6650");
+        brokerProxyValidator.resolveAndCheckTargetAddress("fd4d:801b:73fa:abcd:0000:0000:0000:0001:6650").get();
+    }
+
     private AddressResolver<InetSocketAddress> createMockedAddressResolver(String ipAddressResult) {
         AddressResolver<InetSocketAddress> inetSocketAddressResolver = mock(AddressResolver.class);
         when(inetSocketAddressResolver.resolve(any())).then(invocationOnMock -> {


[pulsar] 02/10: Fail proxy startup if brokerServiceURL is missing scheme (#14682)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6bafef1beb062414196dd877dadff2bc55485b03
Author: Michael Marshall <mm...@apache.org>
AuthorDate: Mon Mar 14 19:29:58 2022 -0500

    Fail proxy startup if brokerServiceURL is missing scheme (#14682)
    
    Make Pulsar Proxy fail on start up with a helpful error message if the `brokerServiceURL` or the `brokerServiceURLTLS` configurations are invalid.
    
    I recently ran into an issue where the configured URL did not have the `pulsar://` or the `pulsar+ssl://` prefix. It would have been very helpful if the proxy had just failed on startup when it had an invalid config.
    
    * Update documentation
    * Add validation checks to the `ProxyServiceStarter` class.
    
    (cherry picked from commit 342a5df24ca9dcf142e471ac0aafebc253042ceb)
    (cherry picked from commit 0a169d9f7c5c646dff1c24ba81957263579dcce8)
---
 conf/proxy.conf                                                |  3 ++-
 .../org/apache/pulsar/proxy/server/ProxyConfiguration.java     |  4 ++--
 .../org/apache/pulsar/proxy/server/ProxyServiceStarter.java    | 10 ++++++++++
 site2/docs/reference-configuration.md                          |  4 ++--
 4 files changed, 16 insertions(+), 5 deletions(-)

diff --git a/conf/proxy.conf b/conf/proxy.conf
index 91b01965310..5ac664ef48b 100644
--- a/conf/proxy.conf
+++ b/conf/proxy.conf
@@ -25,7 +25,8 @@ zookeeperServers=
 # Configuration store connection string (as a comma-separated list)
 configurationStoreServers=
 
-# if Service Discovery is Disabled this url should point to the discovery service provider.
+# If Service Discovery is Disabled this url should point to the discovery service provider.
+# The URL must begin with pulsar:// for plaintext or with pulsar+ssl:// for TLS.
 brokerServiceURL=
 brokerServiceURLTLS=
 
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 a5c44ee716d..a67099306a3 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
@@ -105,12 +105,12 @@ public class ProxyConfiguration implements PulsarConfiguration {
 
     @FieldContext(
         category = CATEGORY_BROKER_DISCOVERY,
-        doc = "The service url points to the broker cluster"
+        doc = "The service url points to the broker cluster. URL must have the pulsar:// prefix."
     )
     private String brokerServiceURL;
     @FieldContext(
         category = CATEGORY_BROKER_DISCOVERY,
-        doc = "The tls service url points to the broker cluster"
+        doc = "The tls service url points to the broker cluster. URL must have the pulsar+ssl:// prefix."
     )
     private String brokerServiceURLTLS;
 
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java
index 2a01d5b6228..cf11355daad 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java
@@ -136,6 +136,16 @@ public class ProxyServiceStarter {
                 config.setConfigurationStoreServers(configurationStoreServers);
             }
 
+            if (isNotBlank(config.getBrokerServiceURL())) {
+                checkArgument(config.getBrokerServiceURL().startsWith("pulsar://"),
+                        "brokerServiceURL must start with pulsar://");
+            }
+
+            if (isNotBlank(config.getBrokerServiceURLTLS())) {
+                checkArgument(config.getBrokerServiceURLTLS().startsWith("pulsar+ssl://"),
+                        "brokerServiceURLTLS must start with pulsar+ssl://");
+            }
+
             if ((isBlank(config.getBrokerServiceURL()) && isBlank(config.getBrokerServiceURLTLS()))
                     || config.isAuthorizationEnabled()) {
                 checkArgument(!isEmpty(config.getZookeeperServers()), "zookeeperServers must be provided");
diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md
index 67ed800984d..d03cb2eae9d 100644
--- a/site2/docs/reference-configuration.md
+++ b/site2/docs/reference-configuration.md
@@ -693,8 +693,8 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config
 |forwardAuthorizationCredentials| Forward client authorization credentials to Broker for re-authorization, and make sure authentication is enabled for this to take effect. |false|
 |zookeeperServers|  The ZooKeeper quorum connection string (as a comma-separated list)  ||
 |configurationStoreServers| Configuration store connection string (as a comma-separated list) ||
-| brokerServiceURL | The service URL pointing to the broker cluster. | |
-| brokerServiceURLTLS | The TLS service URL pointing to the broker cluster | |
+| brokerServiceURL | The service URL pointing to the broker cluster. Must begin with `pulsar://`. | |
+| brokerServiceURLTLS | The TLS service URL pointing to the broker cluster. Must begin with `pulsar+ssl://`. | |
 | brokerWebServiceURL | The Web service URL pointing to the broker cluster | |
 | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | |
 | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | |


[pulsar] 07/10: [Proxy] Remove unnecessary blocking DNS lookup in LookupProxyHandler (#15415)

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 462d44f906fa4dd873c3e4767fcdce84bea20f48
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Wed May 4 08:45:00 2022 +0300

    [Proxy] Remove unnecessary blocking DNS lookup in LookupProxyHandler (#15415)
    
    * [Proxy] Remove unnecessary blocking DNS lookup in LookupProxyHandler
    
    * Use existing code pattern for creating address
    
    (cherry picked from commit 7373a51690d728475d47846bfbcca4fa64f2e228)
    (cherry picked from commit 5980cdc109736ed7e0df7df0e031dcde73e99536)
---
 .../pulsar/proxy/server/LookupProxyHandler.java    | 45 ++++++++++------------
 1 file changed, 21 insertions(+), 24 deletions(-)

diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
index 82300a3ae7f..a8525e4da98 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java
@@ -37,6 +37,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetad
 import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
 import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion;
+import org.apache.pulsar.common.protocol.schema.SchemaVersion;
 import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -224,20 +225,16 @@ public class LookupProxyHandler {
     private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata,
             long clientRequestId) {
         TopicName topicName = TopicName.get(partitionMetadata.getTopic());
-        URI brokerURI;
-        try {
-            String availableBrokerServiceURL = getBrokerServiceUrl(clientRequestId);
-            if (availableBrokerServiceURL == null) {
-                log.warn("No available broker for {} to lookup partition metadata", topicName);
-                return;
-            }
-            brokerURI = new URI(availableBrokerServiceURL);
-        } catch (URISyntaxException e) {
-            proxyConnection.ctx().writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.MetadataError,
-                    e.getMessage(), clientRequestId));
+
+        String serviceUrl = getServiceUrl(clientRequestId);
+        if (serviceUrl == null) {
+            log.warn("No available broker for {} to lookup partition metadata", topicName);
+            return;
+        }
+        InetSocketAddress addr = getAddr(serviceUrl, clientRequestId);
+        if (addr == null) {
             return;
         }
-        InetSocketAddress addr = new InetSocketAddress(brokerURI.getHost(), brokerURI.getPort());
 
         if (log.isDebugEnabled()) {
             log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr,
@@ -293,7 +290,7 @@ public class LookupProxyHandler {
 
     private void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace,
                                             long clientRequestId) {
-        String serviceUrl = getBrokerServiceUrl(clientRequestId);
+        String serviceUrl = getServiceUrl(clientRequestId);
 
         if(!StringUtils.isNotBlank(serviceUrl)) {
             return;
@@ -355,8 +352,14 @@ public class LookupProxyHandler {
         }
 
         final long clientRequestId = commandGetSchema.getRequestId();
-        String serviceUrl = getBrokerServiceUrl(clientRequestId);
+        String serviceUrl = getServiceUrl(clientRequestId);
         String topic = commandGetSchema.getTopic();
+        Optional<SchemaVersion> schemaVersion;
+        if (commandGetSchema.hasSchemaVersion()) {
+            schemaVersion = Optional.of(commandGetSchema.getSchemaVersion().toByteArray()).map(BytesSchemaVersion::of);
+        } else {
+            schemaVersion = Optional.empty();
+        }
 
         if(!StringUtils.isNotBlank(serviceUrl)) {
             return;
@@ -375,12 +378,9 @@ public class LookupProxyHandler {
             // Connected to backend broker
             long requestId = proxyConnection.newRequestId();
             ByteBuf command;
-            byte[] schemaVersion = null;
-            if (commandGetSchema.hasSchemaVersion()) {
-                schemaVersion = commandGetSchema.getSchemaVersion().toByteArray();
-            }
-            command = Commands.newGetSchema(requestId, topic,
-                    Optional.ofNullable(schemaVersion).map(BytesSchemaVersion::of));
+
+            command = Commands.newGetSchema(requestId, topic, schemaVersion);
+
             clientCnx.sendGetRawSchema(command, requestId).whenComplete((r, t) -> {
                 if (t != null) {
                     log.warn("[{}] Failed to get schema {}: {}", clientAddress, topic, t);
@@ -402,10 +402,7 @@ public class LookupProxyHandler {
 
     }
 
-    /**
-     *  Get default broker service url or discovery an available broker
-     **/
-    private String getBrokerServiceUrl(long clientRequestId) {
+    private String getServiceUrl(long clientRequestId) {
         if (isBlank(brokerServiceURL)) {
             ServiceLookupData availableBroker;
             try {


[pulsar] 01/10: [Proxy] Log warning when opening connection to broker fails #14710

Posted by lh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 03fb6edde1661870c4c634260401a217286ba0d2
Author: Lari Hotari <lh...@apache.org>
AuthorDate: Wed Mar 16 09:29:47 2022 +0200

    [Proxy] Log warning when opening connection to broker fails #14710
    
    (cherry picked from commit dbc3fa353ade124b0795d2ba98f1246b8e06efcf)
---
 .../main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java    | 2 ++
 1 file changed, 2 insertions(+)

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 f7615172378..1587b72c831 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
@@ -148,6 +148,8 @@ public class DirectProxyHandler {
         f.addListener(future -> {
             if (!future.isSuccess()) {
                 // Close the connection if the connection attempt has failed.
+                log.warn("[{}] Establishing connection to {} ({}) failed. Closing inbound channel.", inboundChannel,
+                        targetBrokerAddress, targetBrokerUrl, future.cause());
                 inboundChannel.close();
                 return;
             }