You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/09/27 04:09:07 UTC

[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17831: [fix][proxy] Fix refresh client auth

codelipenghui commented on code in PR #17831:
URL: https://github.com/apache/pulsar/pull/17831#discussion_r980669864


##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);
+    }
+
+    @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.setAdvertisedAddress(null);
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setBrokerProxyAllowedTargetPorts("*");
+        proxyConfig.setWebServicePort(Optional.of(0));
+
+        proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
+        proxyConfig.setBrokerClientAuthenticationParameters(
+                AuthTokenUtils.createToken(SECRET_KEY, "Proxy", Optional.empty()));
+        proxyConfig.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        proxyConfig.setProperties(properties);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig,
+                new AuthenticationService(
+                        PulsarConfigurationLoader.convertFrom(proxyConfig))));
+    }
+
+    @AfterMethod(alwaysRun = true)

Review Comment:
   ```suggestion
       @AfterClass(alwaysRun = true)
   ```



##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);
+    }
+
+    @BeforeMethod

Review Comment:
   ```suggestion
       @BeforeClass
   ```



##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);
+    }
+
+    @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.setAdvertisedAddress(null);
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setBrokerProxyAllowedTargetPorts("*");
+        proxyConfig.setWebServicePort(Optional.of(0));
+
+        proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
+        proxyConfig.setBrokerClientAuthenticationParameters(
+                AuthTokenUtils.createToken(SECRET_KEY, "Proxy", Optional.empty()));
+        proxyConfig.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        proxyConfig.setProperties(properties);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig,
+                new AuthenticationService(
+                        PulsarConfigurationLoader.convertFrom(proxyConfig))));
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        proxyService.close();
+    }
+
+    private void startProxy(boolean forwardAuthData) throws Exception {
+        pulsar.getConfiguration().setAuthenticateOriginalAuthData(forwardAuthData);
+        proxyConfig.setForwardAuthorizationCredentials(forwardAuthData);
+        proxyService.start();
+    }
+
+    @DataProvider
+    Object[] forwardAuthDataProvider() {
+        return new Object[]{true,false};
+    }
+
+    @Test(dataProvider = "forwardAuthDataProvider")
+    public void testAuthDataRefresh(boolean forwardAuthData) throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        startProxy(forwardAuthData);
+
+        AuthenticationToken authenticationToken = new AuthenticationToken(() -> {
+            Calendar calendar = Calendar.getInstance();
+            calendar.add(Calendar.SECOND, 10);

Review Comment:
   Reducing the token timeout is better to make the test more efficient.



##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);

Review Comment:
   ```suggestion
           conf.setAuthenticationRefreshCheckSeconds(1);
   ```



##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);
+    }
+
+    @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.setAdvertisedAddress(null);
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setBrokerProxyAllowedTargetPorts("*");
+        proxyConfig.setWebServicePort(Optional.of(0));
+
+        proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
+        proxyConfig.setBrokerClientAuthenticationParameters(
+                AuthTokenUtils.createToken(SECRET_KEY, "Proxy", Optional.empty()));
+        proxyConfig.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        proxyConfig.setProperties(properties);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig,
+                new AuthenticationService(
+                        PulsarConfigurationLoader.convertFrom(proxyConfig))));
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        proxyService.close();
+    }
+
+    private void startProxy(boolean forwardAuthData) throws Exception {
+        pulsar.getConfiguration().setAuthenticateOriginalAuthData(forwardAuthData);
+        proxyConfig.setForwardAuthorizationCredentials(forwardAuthData);
+        proxyService.start();
+    }
+
+    @DataProvider
+    Object[] forwardAuthDataProvider() {
+        return new Object[]{true,false};
+    }
+
+    @Test(dataProvider = "forwardAuthDataProvider")
+    public void testAuthDataRefresh(boolean forwardAuthData) throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        startProxy(forwardAuthData);
+
+        AuthenticationToken authenticationToken = new AuthenticationToken(() -> {
+            Calendar calendar = Calendar.getInstance();
+            calendar.add(Calendar.SECOND, 10);
+            return AuthTokenUtils.createToken(SECRET_KEY, "client", Optional.of(calendar.getTime()));
+        });
+
+        pulsarClient = PulsarClient.builder().serviceUrl(proxyService.getServiceUrl())
+                .authentication(authenticationToken)
+                .operationTimeout(3, TimeUnit.SECONDS)
+                .build();
+
+        admin = PulsarAdmin.builder().serviceHttpUrl(pulsar.getWebServiceAddress())
+                .authentication(authenticationToken).build();
+
+        String namespaceName = "my-tenant/my-ns";
+        admin.clusters().createCluster("proxy-authorization",
+                ClusterData.builder().serviceUrlTls(brokerUrlTls.toString()).build());
+        admin.tenants().createTenant("my-tenant",
+                new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("proxy-authorization")));
+        admin.namespaces().createNamespace(namespaceName);
+
+        String topic = "persistent://my-tenant/my-ns/my-topic1";
+        Producer<byte[]> producer = spy(pulsarClient.newProducer()
+                .topic(topic).create());
+        int i = 0;
+        while (i <= 15) {
+            producer.send(String.valueOf(i).getBytes(StandardCharsets.UTF_8));
+            i++;
+            Thread.sleep(1000);
+        }
+
+        List<CompletableFuture<List<String>>> futures = new ArrayList<>(10);
+        for (i = 0; i < 10; i++) {
+            futures.add(pulsarClient.getPartitionsForTopic(topic));
+        }

Review Comment:
   Any reason that we need to call 10 times?
   And if without this fix, we will get an exception here?
   
   From the PR description, it looks like even without this fix, the client will finally refresh the token, so does the test can't pass without this fix?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1243,6 +1254,13 @@ public void close() {
        }
     }
 
+    public void close(Throwable e) {

Review Comment:
   It should be `protected`, the `ProxyClientCnx` will also call this method.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -361,51 +369,54 @@ protected void handleConnected(CommandConnected connected) {
         state = State.Ready;
     }
 
-    @Override
-    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
-        checkArgument(authChallenge.hasChallenge());
-        checkArgument(authChallenge.getChallenge().hasAuthData());
+    protected final void sendMutualAuthCommand(String authMethod, AuthData authData) {
+        if (log.isDebugEnabled()) {
+            log.debug("{} Mutual auth {}", ctx.channel(), authMethod);
+        }
 
+        ByteBuf request = Commands.newAuthResponse(authMethod,
+                authData,
+                this.protocolVersion,
+                PulsarVersion.getVersion());
+
+        ctx.writeAndFlush(request).addListener(writeFuture -> {
+            if (!writeFuture.isSuccess()) {
+                log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
+                        writeFuture.cause().getMessage());
+                close(writeFuture.cause());
+            }
+        });
+    }
+
+    protected void prepareMutualAuth(CommandAuthChallenge authChallenge) throws AuthenticationException {
         if (Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData())) {
             try {
                 authenticationDataProvider = authentication.getAuthData(remoteHostName);
             } catch (PulsarClientException e) {
                 log.error("{} Error when refreshing authentication data provider: {}", ctx.channel(), e);
-                connectionFuture.completeExceptionally(e);
+                close(e);
                 return;
             }
         }
-
         // mutual authn. If auth not complete, continue auth; if auth complete, complete connectionFuture.
-        try {
-            AuthData authData = authenticationDataProvider
-                .authenticate(AuthData.of(authChallenge.getChallenge().getAuthData()));
-
-            checkState(!authData.isComplete());
-
-            ByteBuf request = Commands.newAuthResponse(authentication.getAuthMethodName(),
-                authData,
-                this.protocolVersion,
-                PulsarVersion.getVersion());
-
-            if (log.isDebugEnabled()) {
-                log.debug("{} Mutual auth {}", ctx.channel(), authentication.getAuthMethodName());
-            }
-
-            ctx.writeAndFlush(request).addListener(writeFuture -> {
-                if (!writeFuture.isSuccess()) {
-                    log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
-                        writeFuture.cause().getMessage());
-                    connectionFuture.completeExceptionally(writeFuture.cause());
-                }
-            });
+        AuthData authData =
+                authenticationDataProvider.authenticate(AuthData.of(authChallenge.getChallenge().getAuthData()));
+        checkState(!authData.isComplete());
+        sendMutualAuthCommand(authentication.getAuthMethodName(), authData);
+    }
 
+    @Override
+    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
+        checkArgument(authChallenge.hasChallenge());
+        checkArgument(authChallenge.getChallenge().hasAuthData());
+        try {
+            prepareMutualAuth(authChallenge);

Review Comment:
   I think we'd better mix the `sendMutualAuthCommand` method with the `prepareMutualAuth` method.
   My suggestion is
   
   ```java
   AuthData authData = prepareMutualAuthData(authChallenge);
   sendMutualAuthCommand(authentication.getAuthMethodName(), authData);
   ```



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -543,6 +557,48 @@ protected void handleAuthResponse(CommandAuthResponse authResponse) {
         }
     }
 
+    private CompletableFuture<AuthData> getOrRefreshOriginalClientAuthData(boolean isRefresh) {
+        if (!isRefresh) {
+            if (service.getConfiguration().isForwardAuthorizationCredentials()) {
+                return CompletableFuture.completedFuture(clientAuthData);
+            }
+            return CompletableFuture.completedFuture(null);
+        }
+
+        if (refreshAuthFuture != null && !refreshAuthFuture.isDone()) {
+            log.error("{} Mutual auth timeout", ctx.channel());
+            ctx.close();
+            return CompletableFuture.failedFuture(new AuthenticationException("Mutual auth timeout"));
+        }
+
+        refreshAuthFuture = new CompletableFuture<>();
+        try {
+            AuthData refreshAuthData = authState.refreshAuthentication();
+            ctx.writeAndFlush(Commands.newAuthChallenge(clientAuthMethod, refreshAuthData, protocolVersionToAdvertise))
+                    .addListener(writeFuture -> {
+                        if (writeFuture.isSuccess()) {
+                            if (LOG.isDebugEnabled()) {
+                                LOG.debug("{} Sent auth challenge to client to refresh credentials with method: {}",
+                                        ctx.channel(), clientAuthMethod);
+                            }
+                        } else {
+                            LOG.error("{} Failed to send request for mutual auth to client", ctx.channel(),
+                                    writeFuture.cause());
+                            refreshAuthFuture.completeExceptionally(writeFuture.cause());
+                            ctx.close();
+                        }
+                    });
+        } catch (AuthenticationException e) {
+            log.error("{} Failed to refresh authentication", ctx.channel(), e);
+            ctx.writeAndFlush(
+                            Commands.newError(-1, ServerError.AuthenticationError, "Failed to refresh authentication"))
+                    .addListener(ChannelFutureListener.CLOSE);
+            refreshAuthFuture.completeExceptionally(e);
+        }
+
+        return refreshAuthFuture;

Review Comment:
   Another point I'm curious about is why we need this method.
   
   If we need the client(user) to refresh the auth data, we can just let the proxy forward the auth challenge command to the client and forward the client auth response to the broker. Does this not work? or maybe I missed something.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -253,8 +254,25 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception {
         } else {
             log.info("{} Connected through proxy to target broker at {}", ctx.channel(), proxyToTargetBrokerAddress);
         }
-        // Send CONNECT command
-        ctx.writeAndFlush(newConnectCommand())
+        completeActive();
+    }
+
+    protected void completeActive() throws Exception {
+        sendConnectCommand(null, null, null);

Review Comment:
   It should be `sendConnectCommand(null, null, null);` right?
   Otherwise, how can we pass the client's auth data to the new connection command
   
   https://github.com/apache/pulsar/pull/17831/files#diff-454fee306809542a71a0f3ce2cd97d7325afcfd7027dc3d156d303aff4c29902L275-L278



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -543,6 +557,48 @@ protected void handleAuthResponse(CommandAuthResponse authResponse) {
         }
     }
 
+    private CompletableFuture<AuthData> getOrRefreshOriginalClientAuthData(boolean isRefresh) {
+        if (!isRefresh) {
+            if (service.getConfiguration().isForwardAuthorizationCredentials()) {
+                return CompletableFuture.completedFuture(clientAuthData);
+            }
+            return CompletableFuture.completedFuture(null);
+        }
+
+        if (refreshAuthFuture != null && !refreshAuthFuture.isDone()) {
+            log.error("{} Mutual auth timeout", ctx.channel());
+            ctx.close();
+            return CompletableFuture.failedFuture(new AuthenticationException("Mutual auth timeout"));
+        }
+
+        refreshAuthFuture = new CompletableFuture<>();
+        try {
+            AuthData refreshAuthData = authState.refreshAuthentication();
+            ctx.writeAndFlush(Commands.newAuthChallenge(clientAuthMethod, refreshAuthData, protocolVersionToAdvertise))
+                    .addListener(writeFuture -> {
+                        if (writeFuture.isSuccess()) {
+                            if (LOG.isDebugEnabled()) {
+                                LOG.debug("{} Sent auth challenge to client to refresh credentials with method: {}",
+                                        ctx.channel(), clientAuthMethod);
+                            }
+                        } else {
+                            LOG.error("{} Failed to send request for mutual auth to client", ctx.channel(),
+                                    writeFuture.cause());
+                            refreshAuthFuture.completeExceptionally(writeFuture.cause());
+                            ctx.close();
+                        }
+                    });
+        } catch (AuthenticationException e) {
+            log.error("{} Failed to refresh authentication", ctx.channel(), e);
+            ctx.writeAndFlush(
+                            Commands.newError(-1, ServerError.AuthenticationError, "Failed to refresh authentication"))
+                    .addListener(ChannelFutureListener.CLOSE);
+            refreshAuthFuture.completeExceptionally(e);
+        }
+
+        return refreshAuthFuture;

Review Comment:
   Looks like the `refreshAuthFuture` will only be completed with the exception, not the auth data. How does it work?



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java:
##########
@@ -18,46 +18,61 @@
  */
 package org.apache.pulsar.proxy.server;
 
-import io.netty.buffer.ByteBuf;
 import io.netty.channel.EventLoopGroup;
-import org.apache.pulsar.PulsarVersion;
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import javax.naming.AuthenticationException;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.client.impl.ClientCnx;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.api.AuthData;
-import org.apache.pulsar.common.protocol.Commands;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.pulsar.common.api.proto.CommandAuthChallenge;
 
+@Slf4j
 public class ProxyClientCnx extends ClientCnx {
-
-    String clientAuthRole;
-    AuthData clientAuthData;
-    String clientAuthMethod;
-    int protocolVersion;
+    private final boolean forwardClientAuthData;
+    private final String clientAuthMethod;
+    private final String clientAuthRole;
+    private final Function<Boolean, CompletableFuture<AuthData>> clientAuthDataSupplier;
 
     public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole,
-                          AuthData clientAuthData, String clientAuthMethod, int protocolVersion) {
-        super(conf, eventLoopGroup);
+                          Function<Boolean, CompletableFuture<AuthData>> clientAuthDataSupplier,
+                          String clientAuthMethod,
+                          int protocolVersion, boolean forwardClientAuthData) {
+        super(conf, eventLoopGroup, protocolVersion);
         this.clientAuthRole = clientAuthRole;
-        this.clientAuthData = clientAuthData;
+        this.clientAuthDataSupplier = clientAuthDataSupplier;
         this.clientAuthMethod = clientAuthMethod;
-        this.protocolVersion = protocolVersion;
+        this.forwardClientAuthData = forwardClientAuthData;
     }
 
     @Override
-    protected ByteBuf newConnectCommand() throws Exception {
-        if (log.isDebugEnabled()) {
-            log.debug("New Connection opened via ProxyClientCnx with params clientAuthRole = {},"
-                            + " clientAuthData = {}, clientAuthMethod = {}",
-                    clientAuthRole, clientAuthData, clientAuthMethod);
+    protected void completeActive() {
+        clientAuthDataSupplier.apply(false).thenAccept(clientAuthData -> {
+            try {
+                sendConnectCommand(clientAuthRole, clientAuthData, clientAuthMethod);
+            } catch (Exception e) {
+                log.error("{} Error during handshake", ctx.channel(), e);
+                close(e);
+            }
+        });
+    }
+
+    @Override
+    protected void prepareMutualAuth(CommandAuthChallenge authChallenge) throws AuthenticationException {
+        boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
+        if (!forwardClientAuthData || !isRefresh) {
+            super.prepareMutualAuth(authChallenge);
+            return;
         }
 
-        authenticationDataProvider = authentication.getAuthData(remoteHostName);
-        AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
-        return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
-            PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
-            clientAuthMethod);
+        clientAuthDataSupplier.apply(true).thenAccept(originalClientAuthData -> {
+            sendMutualAuthCommand(clientAuthMethod, originalClientAuthData);

Review Comment:
   Can the `clientAuthMethod` be different from the client and proxy client?



##########
pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.testng.Assert.assertEquals;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.mockito.Mockito;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyRefreshAuthTest extends ProducerConsumerBase {
+    private final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);
+
+    private ProxyService proxyService;
+    private final ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+
+        // enable tls and auth&auth at broker
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(false);
+        conf.setTopicLevelPoliciesEnabled(false);
+        conf.setProxyRoles(Collections.singleton("Proxy"));
+        conf.setAdvertisedAddress(null);
+        conf.setAuthenticateOriginalAuthData(true);
+        conf.setBrokerServicePort(Optional.of(0));
+        conf.setWebServicePort(Optional.of(0));
+
+        Set<String> superUserRoles = new HashSet<>();
+        superUserRoles.add("superUser");
+        conf.setSuperUserRoles(superUserRoles);
+
+        conf.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        conf.setProperties(properties);
+
+        conf.setClusterName("proxy-authorization");
+        conf.setNumExecutorThreadPoolSize(5);
+
+        conf.setAuthenticationRefreshCheckSeconds(3);
+    }
+
+    @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.setAdvertisedAddress(null);
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setBrokerProxyAllowedTargetPorts("*");
+        proxyConfig.setWebServicePort(Optional.of(0));
+
+        proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
+        proxyConfig.setBrokerClientAuthenticationParameters(
+                AuthTokenUtils.createToken(SECRET_KEY, "Proxy", Optional.empty()));
+        proxyConfig.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName()));
+        Properties properties = new Properties();
+        properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));
+        proxyConfig.setProperties(properties);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig,
+                new AuthenticationService(
+                        PulsarConfigurationLoader.convertFrom(proxyConfig))));
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        proxyService.close();
+    }
+
+    private void startProxy(boolean forwardAuthData) throws Exception {
+        pulsar.getConfiguration().setAuthenticateOriginalAuthData(forwardAuthData);
+        proxyConfig.setForwardAuthorizationCredentials(forwardAuthData);
+        proxyService.start();
+    }
+
+    @DataProvider
+    Object[] forwardAuthDataProvider() {
+        return new Object[]{true,false};
+    }
+
+    @Test(dataProvider = "forwardAuthDataProvider")
+    public void testAuthDataRefresh(boolean forwardAuthData) throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        startProxy(forwardAuthData);
+
+        AuthenticationToken authenticationToken = new AuthenticationToken(() -> {
+            Calendar calendar = Calendar.getInstance();
+            calendar.add(Calendar.SECOND, 10);
+            return AuthTokenUtils.createToken(SECRET_KEY, "client", Optional.of(calendar.getTime()));
+        });
+
+        pulsarClient = PulsarClient.builder().serviceUrl(proxyService.getServiceUrl())
+                .authentication(authenticationToken)
+                .operationTimeout(3, TimeUnit.SECONDS)

Review Comment:
   Is it related to this test?
   It might introduce a flaky test in the CI environment with a short operation timeout.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -253,8 +254,25 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception {
         } else {
             log.info("{} Connected through proxy to target broker at {}", ctx.channel(), proxyToTargetBrokerAddress);
         }
-        // Send CONNECT command
-        ctx.writeAndFlush(newConnectCommand())
+        completeActive();
+    }
+
+    protected void completeActive() throws Exception {

Review Comment:
   ```suggestion
       protected void handleChannelActive() throws Exception {
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -361,51 +369,54 @@ protected void handleConnected(CommandConnected connected) {
         state = State.Ready;
     }
 
-    @Override
-    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
-        checkArgument(authChallenge.hasChallenge());
-        checkArgument(authChallenge.getChallenge().hasAuthData());
+    protected final void sendMutualAuthCommand(String authMethod, AuthData authData) {
+        if (log.isDebugEnabled()) {
+            log.debug("{} Mutual auth {}", ctx.channel(), authMethod);
+        }
 
+        ByteBuf request = Commands.newAuthResponse(authMethod,
+                authData,
+                this.protocolVersion,
+                PulsarVersion.getVersion());
+
+        ctx.writeAndFlush(request).addListener(writeFuture -> {
+            if (!writeFuture.isSuccess()) {
+                log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
+                        writeFuture.cause().getMessage());
+                close(writeFuture.cause());
+            }
+        });
+    }
+
+    protected void prepareMutualAuth(CommandAuthChallenge authChallenge) throws AuthenticationException {
         if (Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData())) {
             try {
                 authenticationDataProvider = authentication.getAuthData(remoteHostName);
             } catch (PulsarClientException e) {
                 log.error("{} Error when refreshing authentication data provider: {}", ctx.channel(), e);
-                connectionFuture.completeExceptionally(e);
+                close(e);
                 return;
             }
         }
-
         // mutual authn. If auth not complete, continue auth; if auth complete, complete connectionFuture.
-        try {
-            AuthData authData = authenticationDataProvider
-                .authenticate(AuthData.of(authChallenge.getChallenge().getAuthData()));
-
-            checkState(!authData.isComplete());
-
-            ByteBuf request = Commands.newAuthResponse(authentication.getAuthMethodName(),
-                authData,
-                this.protocolVersion,
-                PulsarVersion.getVersion());
-
-            if (log.isDebugEnabled()) {
-                log.debug("{} Mutual auth {}", ctx.channel(), authentication.getAuthMethodName());
-            }
-
-            ctx.writeAndFlush(request).addListener(writeFuture -> {
-                if (!writeFuture.isSuccess()) {
-                    log.warn("{} Failed to send request for mutual auth to broker: {}", ctx.channel(),
-                        writeFuture.cause().getMessage());
-                    connectionFuture.completeExceptionally(writeFuture.cause());
-                }
-            });
+        AuthData authData =
+                authenticationDataProvider.authenticate(AuthData.of(authChallenge.getChallenge().getAuthData()));
+        checkState(!authData.isComplete());
+        sendMutualAuthCommand(authentication.getAuthMethodName(), authData);
+    }
 
+    @Override
+    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
+        checkArgument(authChallenge.hasChallenge());
+        checkArgument(authChallenge.getChallenge().hasAuthData());
+        try {
+            prepareMutualAuth(authChallenge);

Review Comment:
   And then the `sendMutualAuthCommand` can be private, the ProxyClientCnx.java only needs to re-implement the `prepareMutualAuthData(authChallenge)` method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org