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/06/21 17:41:09 UTC

[GitHub] [pulsar] poorbarcode opened a new pull request, #16165: [improve][client] [PIP-165] Auto release client useless connections

poorbarcode opened a new pull request, #16165:
URL: https://github.com/apache/pulsar/pull/16165

   Fixes #15516
   
   see: [[PIP-165] Auto release client useless connections](https://github.com/apache/pulsar/issues/15516) 
   
   Master Issue: #15516
   
   
   ### Documentation
   - [ ] `doc-required` 
   - [ ] `doc-not-needed` 
   - [x] `doc`
   - [ ] `doc-complete`


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931124531


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){

Review Comment:
   Already fixed, I have extract `ClientCnxIdleStateManager` to make `ClientCnx` more simple, you can review this in `ClientCnxIdleStateManager`.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931122623


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){

Review Comment:
   Already fixed.



-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1195601929

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r930768851


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -104,6 +122,27 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou
 
         this.shouldCloseDnsResolver = !addressResolver.isPresent();
         this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
+        // Auto release useless connections. see: https://github.com/apache/pulsar/issues/15516.
+        this.connectionMaxIdleSeconds = conf.getConnectionMaxIdleSeconds();
+        this.autoReleaseIdleConnectionsEnabled = connectionMaxIdleSeconds > 0;
+        if (autoReleaseIdleConnectionsEnabled) {
+            // Start async task for release useless connections.
+            this.idleDetectionIntervalSeconds = connectionMaxIdleSeconds;
+            if (this.idleDetectionIntervalSeconds < 30){
+                log.warn("Connection idle detect interval seconds default same as max idle seconds, but max idle"
+                                + " seconds less than " + IDLE_DETECTION_INTERVAL_SECONDS_MIN + ", to avoid checking"
+                                + " connection status too much, use default value : "
+                                + IDLE_DETECTION_INTERVAL_SECONDS_MIN);
+                this.idleDetectionIntervalSeconds = IDLE_DETECTION_INTERVAL_SECONDS_MIN;
+            }
+            asyncReleaseUselessConnectionsTask = eventLoopGroup.scheduleAtFixedRate(() -> {

Review Comment:
   `scheduleAtFixedDelay` is better ?



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931118598


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.

Review Comment:
   > Shall we state more clear on whether his method WILL or WILL NOT change the state to idle?
   
   Already fixed, thanks.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931718685


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   > I'm not sure if we have to comment here to remind other contributors?
   
   Good idea. Already added a comment.



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1197649855

   @codelipenghui @Technoboy- @AlphaWang @congbobo184 @gaozhangmin I extract a new class `ClientCnxIdleState` to make `ClientCnx` more simple, could you review this PR again?


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931124287


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**

Review Comment:
   Already fixed, I have extract `ClientCnxIdleStateManager` to make `ClientCnx` more simple, you can review this in `ClientCnxIdleStateManager`.



-- 
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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931680732


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed
+     * to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the state is successfully transformed,
+     * "idleMarkTime" will be  assigned to current time.
+     */
+    public void tryMarkIdle(){
+        if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            this.idleMarkTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#USING} as much as possible, This method is used
+     * when connection borrow.
+     * @return Whether change idle-stat to #{@link IdleState#USING} success. False is returned only if the connection
+     * has already been released.
+     */
+    public boolean tryMarkReuse(){
+        while (true){

Review Comment:
   Is it possible that there is a long infinite loop here?



-- 
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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931679401


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed
+     * to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the state is successfully transformed,
+     * "idleMarkTime" will be  assigned to current time.
+     */
+    public void tryMarkIdle(){
+        if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            this.idleMarkTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#USING} as much as possible, This method is used
+     * when connection borrow.
+     * @return Whether change idle-stat to #{@link IdleState#USING} success. False is returned only if the connection
+     * has already been released.
+     */
+    public boolean tryMarkReuse(){
+        while (true){

Review Comment:
   got it, thanks



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931718764


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTXTest.java:
##########
@@ -0,0 +1,161 @@
+/**
+ * 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 java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.SystemTopicNames;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+public class AutoCloseUselessClientConTXTest extends AutoCloseUselessClientConSupports {
+
+    private static String topicName = UUID.randomUUID().toString().replaceAll("-","");
+    private static String topicFullName = "persistent://public/default/" + topicName;
+
+    @BeforeMethod
+    public void before() throws PulsarAdminException, MetadataStoreException {
+        // Create Topics
+        PulsarAdmin pulsarAdmin_0 = super.getAllAdmins().get(0);
+        List<String> topicList_defaultNamespace = pulsarAdmin_0.topics().getList("public/default");
+        if (!topicList_defaultNamespace.contains(topicName)
+                && !topicList_defaultNamespace.contains(topicFullName + "-partition-0")
+                && !topicList_defaultNamespace.contains(topicFullName)){
+            pulsarAdmin_0.topics().createNonPartitionedTopic(topicFullName);
+        }
+        List<String> topicList_systemNamespace = pulsarAdmin_0.topics().getList("pulsar/system");
+
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN,
+                            new PartitionedTopicMetadata(2));
+        }
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_LOG)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_LOG,
+                            new PartitionedTopicMetadata(2));
+        }
+    }
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+        updateConfig(conf, "BROKER-INIT");
+    }
+
+    @Override
+    protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) {
+        ServiceConfiguration conf = super.createConfForAdditionalBroker(additionalBrokerIndex);
+        updateConfig(conf, "BROKER-" + additionalBrokerIndex);
+        return conf;
+    }
+
+    /**
+     * Override for make client enable transaction.
+     */
+    @Override
+    protected PulsarClient createNewPulsarClient(ClientBuilder clientBuilder) throws PulsarClientException {
+        try {
+            if (!admin.clusters().getClusters().contains("test")){
+                admin.clusters().createCluster("test", ClusterData.builder().build());
+            }
+            if (!admin.tenants().getTenants().contains("pulsar")){
+                admin.tenants().createTenant("pulsar",
+                        TenantInfo.builder().allowedClusters(Collections.singleton("test")).build());
+            }
+            if (!admin.namespaces().getNamespaces("pulsar").contains("pulsar/system")) {
+                admin.namespaces().createNamespace("pulsar/system");
+            }
+        }catch (Exception e){
+            e.printStackTrace();

Review Comment:
   Already fixed.



-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r927388872


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java:
##########
@@ -124,6 +124,19 @@ public class ClientConfigurationData implements Serializable, Cloneable {
     )
     private int connectionsPerBroker = 1;
 
+    @ApiModelProperty(
+            name = "connectionMaxIdleSeconds",
+            value = "Release the connection if it is not used for more than [connectionMaxIdleSeconds] seconds. "
+                    + "If  [connectionMaxIdleSeconds] < 0, disabled the feature that auto release the idle connections"
+    )
+    private int connectionMaxIdleSeconds = 180;
+
+    @ApiModelProperty(
+            name = "connectionIdleDetectionIntervalSeconds",
+            value = "How often check idle connections"
+    )
+    private int connectionIdleDetectionIntervalSeconds = 60;

Review Comment:
   The `connectionMaxIdleSeconds` should always >= `connectionIdleDetectionIntervalSeconds`? and is it able to remove this configuration? Instead, we can have an interval second based on the  `connectionMaxIdleSeconds` to simplify the configuration.



##########
pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java:
##########
@@ -566,4 +572,9 @@ ClientBuilder authentication(String authPluginClassName, Map<String, String> aut
      * @return
      */
     ClientBuilder socks5ProxyPassword(String socks5ProxyPassword);
+
+    /**
+     * Disable The "auto release useless connections" feature.
+     */
+    ClientBuilder disabledAutoReleaseUselessConnections();

Review Comment:
   Can we only use `connectionMaxIdleSeconds`? -1 means the auto release is disabled.



-- 
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


[GitHub] [pulsar] AlphaWang commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
AlphaWang commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931019688


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.

Review Comment:
   Shall we state more clear on whether his method WILL or WILL NOT change the state to idle? 



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){

Review Comment:
   Suggest adding a whitespace before `{` to follow the code style guide.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){

Review Comment:
   Shall we keep the name consistent with other siblings `isXxx`?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**

Review Comment:
   May better to add an empty line before each method, to follow the code style standard.



-- 
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


[GitHub] [pulsar] mattisonchao commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196841995

   @AlphaWang  Please review again.


-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1164060760

   @Jason918 @codelipenghui @gaoran10 @Technoboy- @gaozhangmin  could you take a look


-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1165206342

   > Overall look good to me, Please correct the javadocs.
   
   Thanks for helping to correct it.


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196052948

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196194836

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931126219


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -357,5 +418,46 @@ int getPoolSize() {
 
     private static final Logger log = LoggerFactory.getLogger(ConnectionPool.class);
 
+    public void doMarkAndReleaseUselessConnections(){
+        if (!autoReleaseIdleConnectionsEnabled){
+            return;
+        }
+        List<Runnable> releaseIdleConnectionTaskList = new ArrayList<>();
+        for (Map.Entry<InetSocketAddress, ConcurrentMap<Integer, CompletableFuture<ClientCnx>>> entry :
+                pool.entrySet()){
+            ConcurrentMap<Integer, CompletableFuture<ClientCnx>> innerPool = entry.getValue();
+            for (Map.Entry<Integer, CompletableFuture<ClientCnx>> entry0 : innerPool.entrySet()){
+                CompletableFuture<ClientCnx> future = entry0.getValue();
+                // Ensure connection has been connected.
+                if (!future.isDone()){
+                    continue;
+                }
+                if (future.isCompletedExceptionally()){
+                    continue;
+                }
+                try {
+                    final ClientCnx clientCnx = future.get();

Review Comment:
   We can use `join` to avoid catching useless exceptions.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   Question:
   Why disable this feature at broker?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java:
##########
@@ -743,7 +743,7 @@ public void run(Timeout timeout) throws Exception {
         });
     }
 
-    private ClientCnx cnx() {
+    ClientCnx cnx() {

Review Comment:
   Why remove `private`?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed
+     * to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the state is successfully transformed,
+     * "idleMarkTime" will be  assigned to current time.
+     */
+    public void tryMarkIdle(){
+        if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            this.idleMarkTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#USING} as much as possible, This method is used
+     * when connection borrow.
+     * @return Whether change idle-stat to #{@link IdleState#USING} success. False is returned only if the connection
+     * has already been released.
+     */
+    public boolean tryMarkReuse(){
+        while (true){

Review Comment:
   Why use loop here?



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931326884


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   The automatic release connection feature is not yet perfect for transaction scenarios, so turn it off first. Here is an example of a TC using a client: 
   
   https://github.com/apache/pulsar/blob/5df15dd2edd7eeab309fea35828915c8698ea339/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java#L146



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   The automatic release connection feature is not yet perfect for transaction scenarios, so turn it off first. Here is an example of the TC using a client: 
   
   https://github.com/apache/pulsar/blob/5df15dd2edd7eeab309fea35828915c8698ea339/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java#L146



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r905846208


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1283,17 +1283,13 @@ public boolean alreadyRelease(){
     }
 
     /**
-     * Changes the idle-state of the connection to #{@link IdleState#IDLE_MARKED}, This method only changes this
-     * connection from the #{@link IdleState#USING} state to the #{@link IdleState#IDLE_MARKED} state. if the
-     * idle-status is successfully changed, "idleMarkTime" is changed to current time.
-     * @return Whether change idle-stat to #{@link IdleState#IDLE_MARKED} success.
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the
+     * state is successfully transformed, "idleMarkTime" will be  assigned to current time.

Review Comment:
   Hi @gaozhangmin 
   src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:[1286] (sizes) LineLength: Line is longer than 120 characters



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1162716014

   /pulsarbot run-failure-checks


-- 
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


[GitHub] [pulsar] codelipenghui commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1188671638

   @poorbarcode Please help resolve the conflicts.


-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1195431848

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r927701257


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java:
##########
@@ -124,6 +124,19 @@ public class ClientConfigurationData implements Serializable, Cloneable {
     )
     private int connectionsPerBroker = 1;
 
+    @ApiModelProperty(
+            name = "connectionMaxIdleSeconds",
+            value = "Release the connection if it is not used for more than [connectionMaxIdleSeconds] seconds. "
+                    + "If  [connectionMaxIdleSeconds] < 0, disabled the feature that auto release the idle connections"
+    )
+    private int connectionMaxIdleSeconds = 180;
+
+    @ApiModelProperty(
+            name = "connectionIdleDetectionIntervalSeconds",
+            value = "How often check idle connections"
+    )
+    private int connectionIdleDetectionIntervalSeconds = 60;

Review Comment:
   > The connectionMaxIdleSeconds should always >= connectionIdleDetectionIntervalSeconds?
   
   no such limit.
   
   >  is it able to remove this configuration? Instead, we can have an interval second based on the connectionMaxIdleSeconds to simplify the configuration.
   
   Good idea. Already fixed.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931326884


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   The automatic release connection feature is not yet perfect for transaction scenarios, so turn it off first.



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1189399008

   > @poorbarcode Please help resolve the conflicts.
   
   completed.


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931321967


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java:
##########
@@ -743,7 +743,7 @@ public void run(Timeout timeout) throws Exception {
         });
     }
 
-    private ClientCnx cnx() {
+    ClientCnx cnx() {

Review Comment:
   Already fixed.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931124287


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**

Review Comment:
   Already fixed, extract `ClientCnxIdleStateManager` to make `ClientCnx` more simple, you can review this in `ClientCnxIdleStateManager`.



-- 
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


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r930766713


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -104,6 +122,27 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou
 
         this.shouldCloseDnsResolver = !addressResolver.isPresent();
         this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
+        // Auto release useless connections. see: https://github.com/apache/pulsar/issues/15516.
+        this.connectionMaxIdleSeconds = conf.getConnectionMaxIdleSeconds();
+        this.autoReleaseIdleConnectionsEnabled = connectionMaxIdleSeconds > 0;
+        if (autoReleaseIdleConnectionsEnabled) {
+            // Start async task for release useless connections.
+            this.idleDetectionIntervalSeconds = connectionMaxIdleSeconds;
+            if (this.idleDetectionIntervalSeconds < 30){

Review Comment:
   Maybe `if (this.idleDetectionIntervalSeconds < IDLE_DETECTION_INTERVAL_SECONDS_MIN){` ?



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r930770704


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -104,6 +122,27 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou
 
         this.shouldCloseDnsResolver = !addressResolver.isPresent();
         this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
+        // Auto release useless connections. see: https://github.com/apache/pulsar/issues/15516.
+        this.connectionMaxIdleSeconds = conf.getConnectionMaxIdleSeconds();
+        this.autoReleaseIdleConnectionsEnabled = connectionMaxIdleSeconds > 0;
+        if (autoReleaseIdleConnectionsEnabled) {
+            // Start async task for release useless connections.
+            this.idleDetectionIntervalSeconds = connectionMaxIdleSeconds;
+            if (this.idleDetectionIntervalSeconds < 30){

Review Comment:
   > Maybe if (this.idleDetectionIntervalSeconds < IDLE_DETECTION_INTERVAL_SECONDS_MIN) ?
   
   Already fixed. Thanks



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1197485324

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] gaozhangmin commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
gaozhangmin commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r904978622


##########
pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java:
##########
@@ -566,4 +572,9 @@ ClientBuilder authentication(String authPluginClassName, Map<String, String> aut
      * @return
      */
     ClientBuilder socks5ProxyPassword(String socks5ProxyPassword);
+
+    /**
+     * Disable The "auto release useless connections" feature.

Review Comment:
   ```suggestion
        * Disable The "auto release idle connections" feature.
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -161,6 +162,16 @@ public class ClientCnx extends PulsarHandler {
     protected AuthenticationDataProvider authenticationDataProvider;
     private TransactionBufferHandler transactionBufferHandler;
 
+    /** Create time. **/
+    private final long createTime;
+    /** The time when marks the connection is idle. **/

Review Comment:
   ```suggestion
       /** The time when the connection is marked as idle. **/
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,168 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return this connection is idle now.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#IDLE_MARKED}, This method only changes this
+     * connection from the #{@link IdleState#USING} state to the #{@link IdleState#IDLE_MARKED} state. if the
+     * idle-status is successfully changed, "idleMarkTime" is changed to current time.
+     * @return Whether change idle-stat to #{@link IdleState#IDLE_MARKED} success.
+     */
+    public boolean tryMarkIdle(){
+        if (!compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            return isIdle();
+        }
+        this.idleMarkTime = System.currentTimeMillis();
+        return true;
+    }

Review Comment:
   ```suggestion
       /**
        * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the
        * state is successfully transformed, "idleMarkTime" will be  assigned to current time.
        */
       public void tryMarkIdle(){
           if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
               this.idleMarkTime = System.currentTimeMillis();
           }
       }
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,168 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return this connection is idle now.
+     */

Review Comment:
   ```suggestion
   
       /**
        * Check if the client connection is eligible for being marked as idle. 
        *  if so, {@link org.apache.pulsar.client.impl.ClientCnx#tryMarkIdle()} will be invoked next.
        * @return if the connection is idle.
        */
   ```



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1195792869

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] mattisonchao commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1194924548

   @codelipenghui  Please take a look again.


-- 
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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r927799320


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java:
##########
@@ -102,6 +102,13 @@ public ClientBuilder listenerName(String listenerName) {
         return this;
     }
 
+    @Override
+    public ClientBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds) {
+        checkArgument(connectionMaxIdleSeconds >= 0, "Param connectionMaxIdleSeconds at least 1.");

Review Comment:
   Do we need this check? The connectionMaxIdleSeconds can be -1



-- 
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


[GitHub] [pulsar] Technoboy- merged pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
Technoboy- merged PR #16165:
URL: https://github.com/apache/pulsar/pull/16165


-- 
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


[GitHub] [pulsar] gaozhangmin commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
gaozhangmin commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r907968490


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,164 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check if the client connection is eligible for being marked as idle. 
+     *  if so, {@link org.apache.pulsar.client.impl.ClientCnx#tryMarkIdle()} will be invoked next.
+     * @return if the connection is idle.
+     */

Review Comment:
   ```suggestion
       /**
        * Check if the client connection is eligible for being marked as idle.
        *  if so, {@link org.apache.pulsar.client.impl.ClientCnx#tryMarkIdle()} will be invoked next.
        *  @return true if the connection is eligible 
        */
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,164 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check if the client connection is eligible for being marked as idle. 
+     *  if so, {@link org.apache.pulsar.client.impl.ClientCnx#tryMarkIdle()} will be invoked next.
+     * @return if the connection is idle.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the
+     * state is successfully transformed, "idleMarkTime" will be  assigned to current time.
+     */

Review Comment:
   ```suggestion
       /**
        * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, 
        * state is only allowed  to  transform  from previous state  #{@link IdleState#USING}. 
        * If the state is successfully transformed, "idleMarkTime" will be  assigned to current time.
        */
   ```



-- 
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


[GitHub] [pulsar] poorbarcode closed pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode closed pull request #16165: [improve][client] [PIP-165] Auto release client useless connections
URL: https://github.com/apache/pulsar/pull/16165


-- 
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


[GitHub] [pulsar] gaozhangmin commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
gaozhangmin commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r910605589


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,168 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return this connection is idle now.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#IDLE_MARKED}, This method only changes this
+     * connection from the #{@link IdleState#USING} state to the #{@link IdleState#IDLE_MARKED} state. if the
+     * idle-status is successfully changed, "idleMarkTime" is changed to current time.
+     * @return Whether change idle-stat to #{@link IdleState#IDLE_MARKED} success.
+     */
+    public boolean tryMarkIdle(){
+        if (!compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            return isIdle();
+        }
+        this.idleMarkTime = System.currentTimeMillis();
+        return true;
+    }

Review Comment:
   @poorbarcode  this suggestion remained  unchanged.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r929510203


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java:
##########
@@ -102,6 +102,13 @@ public ClientBuilder listenerName(String listenerName) {
         return this;
     }
 
+    @Override
+    public ClientBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds) {
+        checkArgument(connectionMaxIdleSeconds >= 0, "Param connectionMaxIdleSeconds at least 1.");

Review Comment:
   Already remove this check. Thanks



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r927701562


##########
pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java:
##########
@@ -566,4 +572,9 @@ ClientBuilder authentication(String authPluginClassName, Map<String, String> aut
      * @return
      */
     ClientBuilder socks5ProxyPassword(String socks5ProxyPassword);
+
+    /**
+     * Disable The "auto release useless connections" feature.
+     */
+    ClientBuilder disabledAutoReleaseUselessConnections();

Review Comment:
   > Can we only use connectionMaxIdleSeconds? -1 means the auto release is disabled.
   
   Good idea. Already fixed.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r930773553


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -104,6 +122,27 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou
 
         this.shouldCloseDnsResolver = !addressResolver.isPresent();
         this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
+        // Auto release useless connections. see: https://github.com/apache/pulsar/issues/15516.
+        this.connectionMaxIdleSeconds = conf.getConnectionMaxIdleSeconds();
+        this.autoReleaseIdleConnectionsEnabled = connectionMaxIdleSeconds > 0;
+        if (autoReleaseIdleConnectionsEnabled) {
+            // Start async task for release useless connections.
+            this.idleDetectionIntervalSeconds = connectionMaxIdleSeconds;
+            if (this.idleDetectionIntervalSeconds < 30){
+                log.warn("Connection idle detect interval seconds default same as max idle seconds, but max idle"
+                                + " seconds less than " + IDLE_DETECTION_INTERVAL_SECONDS_MIN + ", to avoid checking"
+                                + " connection status too much, use default value : "
+                                + IDLE_DETECTION_INTERVAL_SECONDS_MIN);
+                this.idleDetectionIntervalSeconds = IDLE_DETECTION_INTERVAL_SECONDS_MIN;
+            }
+            asyncReleaseUselessConnectionsTask = eventLoopGroup.scheduleAtFixedRate(() -> {

Review Comment:
   Already fixed. Thanks



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931714217


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed
+     * to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the state is successfully transformed,
+     * "idleMarkTime" will be  assigned to current time.
+     */
+    public void tryMarkIdle(){
+        if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            this.idleMarkTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#USING} as much as possible, This method is used
+     * when connection borrow.
+     * @return Whether change idle-stat to #{@link IdleState#USING} success. False is returned only if the connection
+     * has already been released.
+     */
+    public boolean tryMarkReuse(){
+        while (true){

Review Comment:
   > Is it possible that there is a long infinite loop here?
   
   no, less than 3 times.



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196825763

   @codelipenghui @Technoboy- I extract a new class `ClientCnxIdleStateManager` to make `ClientCnx` more simple, could you review this PR again?


-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1162787292

   /pulsarbot run-failure-checks


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r905845580


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1203,9 +1203,9 @@ private void checkRequestTimeout() {
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
 
     /**
-     * Check client connection is now free. This method may change the state to idle.
-     * This method will not change the state to idle.
-     * @return this connection is idle now.
+     * Check if the client connection is eligible for being marked as idle. 

Review Comment:
   Hi @gaozhangmin 
   src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:[1206] (regexp) RegexpSingleline: Trailing whitespace



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r908034600


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1188,4 +1201,164 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check if the client connection is eligible for being marked as idle. 
+     *  if so, {@link org.apache.pulsar.client.impl.ClientCnx#tryMarkIdle()} will be invoked next.
+     * @return if the connection is idle.
+     */

Review Comment:
   > *  @return true if the connection is eligible 
   
   Hi @gaozhangmin 
   This line ends with `space`, style check will fail.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931323513


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1265,4 +1278,165 @@ private void checkRequestTimeout() {
     }
 
     private static final Logger log = LoggerFactory.getLogger(ClientCnx.class);
+
+    /**
+     * Check client connection is now free. This method may change the state to idle.
+     * This method will not change the state to idle.
+     * @return true if the connection is eligible.
+     */
+    public boolean idleCheck(){
+        if (pendingRequests != null && !pendingRequests.isEmpty()){
+            return false;
+        }
+        if (waitingLookupRequests != null  && !waitingLookupRequests.isEmpty()){
+            return false;
+        }
+        if (!consumers.isEmpty()){
+            return false;
+        }
+        if (!producers.isEmpty()){
+            return false;
+        }
+        if (!transactionMetaStoreHandlers.isEmpty()){
+            return false;
+        }
+        return true;
+    }
+    /**
+     * Get idle-stat.
+     * @return connection idle-stat
+     */
+    public IdleState getIdleStat(){
+        return STATE_UPDATER.get(this);
+    }
+    /**
+     * Compare and switch idle-stat.
+     * @return Whether the update is successful.Because there may be other threads competing, possible return false.
+     */
+    public boolean compareAndSetIdleStat(IdleState originalStat, IdleState newStat){
+        return STATE_UPDATER.compareAndSet(this, originalStat, newStat);
+    }
+
+    /**
+     * Indicates the usage status of the connection and whether it has been released.
+     */
+    public enum IdleState {
+        /** The connection is in use. **/
+        USING,
+        /** The connection is in idle. **/
+        IDLE_MARKED,
+        /** The connection is in idle and will be released soon. **/
+        BEFORE_RELEASE,
+        /** The connection has already been released. **/
+        RELEASED;
+    }
+
+    /**
+     * @return Whether this connection is in use.
+     */
+    public boolean isUsing(){
+        return getIdleStat() == IdleState.USING;
+    }
+
+    /**
+     * @return Whether this connection is in idle.
+     */
+    public boolean isIdle(){
+        return getIdleStat() == IdleState.IDLE_MARKED;
+    }
+
+    /**
+     * @return Whether this connection is in idle and will be released soon.
+     */
+    public boolean isWillBeRelease(){
+        return getIdleStat() == IdleState.BEFORE_RELEASE;
+    }
+
+    /**
+     * @return Whether this connection has already been released.
+     */
+    public boolean alreadyRelease(){
+        return getIdleStat() == IdleState.RELEASED;
+    }
+
+    /**
+     * Try to transform the state of the connection to #{@link IdleState#IDLE_MARKED}, state should only be transformed
+     * to #{@link IdleState#IDLE_MARKED} from state  #{@link IdleState#USING}. if the state is successfully transformed,
+     * "idleMarkTime" will be  assigned to current time.
+     */
+    public void tryMarkIdle(){
+        if (compareAndSetIdleStat(IdleState.USING, IdleState.IDLE_MARKED)){
+            this.idleMarkTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Changes the idle-state of the connection to #{@link IdleState#USING} as much as possible, This method is used
+     * when connection borrow.
+     * @return Whether change idle-stat to #{@link IdleState#USING} success. False is returned only if the connection
+     * has already been released.
+     */
+    public boolean tryMarkReuse(){
+        while (true){

Review Comment:
   When `state change` is in race condition, it may take more than one execution to succeed.



-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931326884


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   The automatic release connection feature is not yet perfect for transaction scenarios, so turn it off first. Here is an example of the TC using `ClientCnx`: 
   
   https://github.com/apache/pulsar/blob/5df15dd2edd7eeab309fea35828915c8698ea339/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java#L146



-- 
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


[GitHub] [pulsar] Technoboy- commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931704003


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTXTest.java:
##########
@@ -0,0 +1,161 @@
+/**
+ * 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 java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.SystemTopicNames;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+public class AutoCloseUselessClientConTXTest extends AutoCloseUselessClientConSupports {
+
+    private static String topicName = UUID.randomUUID().toString().replaceAll("-","");
+    private static String topicFullName = "persistent://public/default/" + topicName;
+
+    @BeforeMethod
+    public void before() throws PulsarAdminException, MetadataStoreException {
+        // Create Topics
+        PulsarAdmin pulsarAdmin_0 = super.getAllAdmins().get(0);
+        List<String> topicList_defaultNamespace = pulsarAdmin_0.topics().getList("public/default");
+        if (!topicList_defaultNamespace.contains(topicName)
+                && !topicList_defaultNamespace.contains(topicFullName + "-partition-0")
+                && !topicList_defaultNamespace.contains(topicFullName)){
+            pulsarAdmin_0.topics().createNonPartitionedTopic(topicFullName);
+        }
+        List<String> topicList_systemNamespace = pulsarAdmin_0.topics().getList("pulsar/system");
+
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN,
+                            new PartitionedTopicMetadata(2));
+        }
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_LOG)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_LOG,
+                            new PartitionedTopicMetadata(2));
+        }
+    }
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+        updateConfig(conf, "BROKER-INIT");
+    }
+
+    @Override
+    protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) {
+        ServiceConfiguration conf = super.createConfForAdditionalBroker(additionalBrokerIndex);
+        updateConfig(conf, "BROKER-" + additionalBrokerIndex);
+        return conf;
+    }
+
+    /**
+     * Override for make client enable transaction.
+     */
+    @Override
+    protected PulsarClient createNewPulsarClient(ClientBuilder clientBuilder) throws PulsarClientException {
+        try {
+            if (!admin.clusters().getClusters().contains("test")){
+                admin.clusters().createCluster("test", ClusterData.builder().build());
+            }
+            if (!admin.tenants().getTenants().contains("pulsar")){
+                admin.tenants().createTenant("pulsar",
+                        TenantInfo.builder().allowedClusters(Collections.singleton("test")).build());
+            }
+            if (!admin.namespaces().getNamespaces("pulsar").contains("pulsar/system")) {
+                admin.namespaces().createNamespace("pulsar/system");
+            }
+        }catch (Exception e){
+            e.printStackTrace();

Review Comment:
   Need to use log4j...



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTXTest.java:
##########
@@ -0,0 +1,161 @@
+/**
+ * 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 java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.SystemTopicNames;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+public class AutoCloseUselessClientConTXTest extends AutoCloseUselessClientConSupports {
+
+    private static String topicName = UUID.randomUUID().toString().replaceAll("-","");
+    private static String topicFullName = "persistent://public/default/" + topicName;
+
+    @BeforeMethod
+    public void before() throws PulsarAdminException, MetadataStoreException {
+        // Create Topics
+        PulsarAdmin pulsarAdmin_0 = super.getAllAdmins().get(0);
+        List<String> topicList_defaultNamespace = pulsarAdmin_0.topics().getList("public/default");
+        if (!topicList_defaultNamespace.contains(topicName)
+                && !topicList_defaultNamespace.contains(topicFullName + "-partition-0")
+                && !topicList_defaultNamespace.contains(topicFullName)){
+            pulsarAdmin_0.topics().createNonPartitionedTopic(topicFullName);
+        }
+        List<String> topicList_systemNamespace = pulsarAdmin_0.topics().getList("pulsar/system");
+
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN,
+                            new PartitionedTopicMetadata(2));
+        }
+        if (!pulsar.getPulsarResources()
+                .getNamespaceResources()
+                .getPartitionedTopicResources().partitionedTopicExists(SystemTopicNames.TRANSACTION_COORDINATOR_LOG)){
+            pulsar.getPulsarResources()
+                    .getNamespaceResources()
+                    .getPartitionedTopicResources()
+                    .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_LOG,
+                            new PartitionedTopicMetadata(2));
+        }
+    }
+
+    @Override
+    protected void doInitConf() throws Exception {
+        super.doInitConf();
+        updateConfig(conf, "BROKER-INIT");
+    }
+
+    @Override
+    protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) {
+        ServiceConfiguration conf = super.createConfForAdditionalBroker(additionalBrokerIndex);
+        updateConfig(conf, "BROKER-" + additionalBrokerIndex);
+        return conf;
+    }
+
+    /**
+     * Override for make client enable transaction.
+     */
+    @Override
+    protected PulsarClient createNewPulsarClient(ClientBuilder clientBuilder) throws PulsarClientException {
+        try {
+            if (!admin.clusters().getClusters().contains("test")){
+                admin.clusters().createCluster("test", ClusterData.builder().build());
+            }
+            if (!admin.tenants().getTenants().contains("pulsar")){
+                admin.tenants().createTenant("pulsar",
+                        TenantInfo.builder().allowedClusters(Collections.singleton("test")).build());
+            }
+            if (!admin.namespaces().getNamespaces("pulsar").contains("pulsar/system")) {
+                admin.namespaces().createNamespace("pulsar/system");
+            }
+        }catch (Exception e){
+            e.printStackTrace();

Review Comment:
   Need to use logger...



-- 
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


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931679770


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1440,6 +1440,9 @@ public synchronized PulsarClient getClient() throws PulsarServerException {
                 ClientConfigurationData conf =
                         ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
 
+                // Disabled auto release useless connections
+                conf.setConnectionMaxIdleSeconds(-1);

Review Comment:
   I'm not sure if we have to comment here to remind other contributors?



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196334446

   /pulsarbot rerun-failure-checks


-- 
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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#discussion_r931321784


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java:
##########
@@ -357,5 +418,46 @@ int getPoolSize() {
 
     private static final Logger log = LoggerFactory.getLogger(ConnectionPool.class);
 
+    public void doMarkAndReleaseUselessConnections(){
+        if (!autoReleaseIdleConnectionsEnabled){
+            return;
+        }
+        List<Runnable> releaseIdleConnectionTaskList = new ArrayList<>();
+        for (Map.Entry<InetSocketAddress, ConcurrentMap<Integer, CompletableFuture<ClientCnx>>> entry :
+                pool.entrySet()){
+            ConcurrentMap<Integer, CompletableFuture<ClientCnx>> innerPool = entry.getValue();
+            for (Map.Entry<Integer, CompletableFuture<ClientCnx>> entry0 : innerPool.entrySet()){
+                CompletableFuture<ClientCnx> future = entry0.getValue();
+                // Ensure connection has been connected.
+                if (!future.isDone()){
+                    continue;
+                }
+                if (future.isCompletedExceptionally()){
+                    continue;
+                }
+                try {
+                    final ClientCnx clientCnx = future.get();

Review Comment:
   Good idea. Already fixed.



-- 
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


[GitHub] [pulsar] poorbarcode commented on pull request #16165: [improve][client] [PIP-165] Auto release client useless connections

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on PR #16165:
URL: https://github.com/apache/pulsar/pull/16165#issuecomment-1196069423

   /pulsarbot rerun-failure-checks


-- 
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