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 2021/09/06 09:56:59 UTC

[GitHub] [pulsar] liangyepianzhou opened a new pull request #11933: Add transaction perf

liangyepianzhou opened a new pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933


   <!--
   ### Contribution Checklist
     
     - Name the pull request in the form "[Issue XYZ][component] Title of the pull request", where *XYZ* should be replaced by the actual issue number.
       Skip *Issue XYZ* if there is no associated github issue for this pull request.
       Skip *component* if you are unsure about which is the best component. E.g. `[docs] Fix typo in produce method`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   
   
   Master Issue: #https://github.com/apache/pulsar/issues/11765
   
   ### Motivation
   - When opening a transaction, what is the delay and rate of pulsar sending and consuming messages?
   - How does the granularity of the transaction affect the rate at which pulsar produces or consumes messages
   - Is the rate of abort and commit transactions different
   - Try to keep the functions of ordinary produce  perfand consume   perf as much as possible
   ### Modifications
   Add PerformanceTransaction.Class
   - According to the number of topics set by the user, each topic starts a thread to execute a number of transactions asynchronously. 
   - If there is a preset transaction value, the production or consumption of this topic is terminated after the corresponding number of transactions is executed, and the production and consumption of all topics are summed up. After the consumption ends, the main thread ends.
   - Each consumer can have multiple subscriptions, but each subscription has only one consumer. In order to better compare the rate of production and consumption.
   ### Verifying this change
   
   Not yet
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation
   
   Check the box below and label this PR (if you have committer privilege).
   
   Need to update docs? 
   
   - [ ] doc-required 
     
     (If you need help on updating docs, create a doc issue)
     
   - [ ] no-need-doc 
     
     (Please explain why)
     
   - [ ] doc 
     
     (If this PR contains doc changes)
   
   
   


-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711589330



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {

Review comment:
       It is indeed redundant. 




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713575554



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);

Review comment:
       When tasks were executed completely, `executorService.shutdownNow();` will be executed.  At this time, other threads may be executing, and an interrupt exception may occur




-- 
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] liangyepianzhou removed a comment on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927265990


   /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] eolivelli commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r726801376



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();

Review comment:
       do we want to abort the JVM ?
   

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "

Review comment:
       why some arguments have a double "-" and some other arguments have a single "-" ?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;

Review comment:
       Please use:
   `--txn-disable`
   and
   `isDisableTransaction`

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/utils/PerformanceUtils.java
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.testclient.utils;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.testclient.PerformanceProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(PerformanceProducer.class);
+
+    public static AtomicReference<Transaction> buildTransaction(PulsarClient pulsarClient, boolean isEnableTransaction,
+                                                                long transactionTimeout) {
+        if (isEnableTransaction) {
+            while(true) {
+                AtomicReference atomicReference = null;
+                try {
+                    atomicReference = new AtomicReference(pulsarClient.newTransaction()
+                            .withTransactionTimeout(transactionTimeout, TimeUnit.SECONDS).build().get());
+                } catch (Exception e) {
+                    log.error("Failed to new transaction", e);

Review comment:
       I would suggest to fail the execution of the tool
   
   if you do not want to fail we should report the number of errors
   in theory this should never happen, as it is the sign that something is wrong.
   
   I have good experience of pulsar-perf in order to perform testing, and it is far better to see the command fail instead of having to look in the logs or to create GBs of spammy logs and then see that the workload broke Pulsar

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();
+                                        }
+                                        long receiveTime = System.nanoTime();
+                                        if (!arguments.isDisEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        numMessagesAckSuccess.increment();
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                numMessagesAckSuccess.increment();
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                }
+                            }
+                        }
+
+                        for(Producer<byte[]> producer : producers){
+                            for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) {
+                                long sendTime = System.nanoTime();
+                                if (!arguments.isDisEnableTransaction) {
+                                    producer.newMessage(transaction).value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send transaction message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                } else {
+                                    producer.newMessage().value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                }
+                            }
+                        }
+
+                        if(rateLimiter != null){

Review comment:
       please fix indent/parenthesis

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService

Review comment:
       remove "0"

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();
+                                        }
+                                        long receiveTime = System.nanoTime();
+                                        if (!arguments.isDisEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        numMessagesAckSuccess.increment();
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                numMessagesAckSuccess.increment();
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                }
+                            }
+                        }
+
+                        for(Producer<byte[]> producer : producers){
+                            for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) {
+                                long sendTime = System.nanoTime();
+                                if (!arguments.isDisEnableTransaction) {
+                                    producer.newMessage(transaction).value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send transaction message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                } else {
+                                    producer.newMessage().value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                }
+                            }
+                        }
+
+                        if(rateLimiter != null){
+                            rateLimiter.tryAcquire();
+                        }
+                        if (!arguments.isDisEnableTransaction) {
+                            if (!arguments.isAbortTransaction) {
+                                transaction.commit()
+                                        .thenRun(() -> {
+                                            numTxnOpSuccess.increment();
+                                            totalNumEndTxnOpSuccess.increment();
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Commit transaction {} failed with exception",
+                                                    transaction.getTxnID().toString(),
+                                                    exception);
+                                            totalNumEndTxnOpFailed.increment();
+                                            return null;
+                                        });
+                            } else {
+                                transaction.abort().thenRun(() -> {
+                                    numTxnOpSuccess.increment();
+                                    totalNumEndTxnOpSuccess.increment();
+                                }).exceptionally(exception -> {
+                                    if(exception instanceof InterruptedException && ! executing.get()){

Review comment:
       I am not sure that with CompletableFuture you can see an InterruptedException 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] congbobo184 commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r715273517



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -634,10 +685,15 @@ private static void runProducer(int producerId,
                     } else {
                         payloadData = payloadBytes;
                     }
-
-                    TypedMessageBuilder<byte[]> messageBuilder = producer.newMessage()
-                            .value(payloadData);
-                    if (arguments.delay >0) {
+                    TypedMessageBuilder<byte[]> messageBuilder;
+                    if (arguments.isEnableTransaction) {
+                        messageBuilder = producer.newMessage(transaction)

Review comment:
       this transaction should get under 
   
   ```
                       if(arguments.isEnableTransaction && arguments.numMessagesPerTransaction > 0){
                           numMsgPerTxnLimit.acquire();
                       }
   ```




-- 
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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r726976830



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();
+                                        }
+                                        long receiveTime = System.nanoTime();
+                                        if (!arguments.isDisEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        numMessagesAckSuccess.increment();
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                numMessagesAckSuccess.increment();
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                }
+                            }
+                        }
+
+                        for(Producer<byte[]> producer : producers){
+                            for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) {
+                                long sendTime = System.nanoTime();
+                                if (!arguments.isDisEnableTransaction) {
+                                    producer.newMessage(transaction).value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send transaction message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                } else {
+                                    producer.newMessage().value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                }
+                            }
+                        }
+
+                        if(rateLimiter != null){
+                            rateLimiter.tryAcquire();
+                        }
+                        if (!arguments.isDisEnableTransaction) {
+                            if (!arguments.isAbortTransaction) {
+                                transaction.commit()
+                                        .thenRun(() -> {
+                                            numTxnOpSuccess.increment();
+                                            totalNumEndTxnOpSuccess.increment();
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Commit transaction {} failed with exception",
+                                                    transaction.getTxnID().toString(),
+                                                    exception);
+                                            totalNumEndTxnOpFailed.increment();
+                                            return null;
+                                        });
+                            } else {
+                                transaction.abort().thenRun(() -> {
+                                    numTxnOpSuccess.increment();
+                                    totalNumEndTxnOpSuccess.increment();
+                                }).exceptionally(exception -> {
+                                    if(exception instanceof InterruptedException && ! executing.get()){

Review comment:
       We caught this exception here to prevent printing errors from affecting the reading of the log
   




-- 
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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r751210224



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -438,11 +550,27 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {

Review comment:
       The problem has been solved, please help me to 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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713567472



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);

Review comment:
       I think we should not use timeout because it has no effect

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);

Review comment:
       if build fail, we should shutdown the test process

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {

Review comment:
       This thread doesn't seem to make much sense

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,34 +497,90 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFailNum = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {
+                log.info(
+                        "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                                + "--- Transaction: {} transaction end successfully --- {} transaction end failed "
+                                + "--- {}  Txn/s --- AckRate: {} msg/s"
+                                + "--- Latency: mean: {} ms - med: {} "
+                                + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                        intFormat.format(total),
+                        dec.format(rate), dec.format(throughput),
+                        totalTxnOpSuccessNum,
+                        totalTxnOpFailNum,
+                        dec.format(rateOpenTxn),
+                        dec.format(rateAck),
+                        dec.format(reportHistogram.getMean()),
+                        reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
+                        reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
+                        reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
+            }else {
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                            + "--- Latency: mean: {} ms - med: {} "
+                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     dec.format(rate), dec.format(throughput), dec.format(reportHistogram.getMean()),
                     reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
                     reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
                     reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
-
             reportHistogram.reset();
             oldTime = now;
-        }
+        }}
 
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalEndTxnSuccess = 0;
+        long totalEndTxnFail = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double rateOpenTxn = 0;
+        if (arguments.isEnableTransaction) {
+            totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalEndTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed;
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+        if(arguments.isEnableTransaction){

Review comment:
       I still suggest that we should not modify the logic of the original log, and only print the relevant parameters separately when enabling transaction.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);
+                                                log.info("Receive message {} ", message);
+                                            } catch (Exception e) {
+                                                log.error("{} can`t receive message in 2 sec with exception {}",
+                                                        consumer, e);
+                                            }
+
+                                            messageReceived.increment();
+                                            long receiveTime = System.nanoTime();
+                                            if (arguments.isEnableTransaction) {
+                                                consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                        .thenRun(() -> {
+                                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                                    System.nanoTime() - receiveTime);
+                                                            messageAckRecorder.recordValue(latencyMicros);
+                                                            messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {
+                                                        return null;
+                                                    }
+                                                    log.error(
+                                                            "Ack message failed with transaction {} throw exception {}",
+                                                            transaction, exception);
+                                                    numMessagesAckFailed.increment();
+                                                    return null;
+                                                });
+                                            } else {
+                                                consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                    long latencyMicros = NANOSECONDS.toMicros(
+                                                            System.nanoTime() - receiveTime);
+                                                    messageAckRecorder.recordValue(latencyMicros);
+                                                    messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {
+                                                        return null;
+                                                    }
+                                                    log.error(
+                                                            "Ack message failed with transaction {} throw exception {}",
+                                                            transaction, exception);
+                                                    numMessagesAckFailed.increment();
+                                                    return null;
+                                                });
+                                            }
+                                        }
+                                    }
+                                    messageReceived.reset();
+                                }
+
+                                for(Producer<byte[]> producer : producers){
+                                    while (true){
+                                    if(messageSend.sum() >= arguments.numMessagesProducedPerTransaction){

Review comment:
       same as consumer

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);
+                                                log.info("Receive message {} ", message);
+                                            } catch (Exception e) {
+                                                log.error("{} can`t receive message in 2 sec with exception {}",
+                                                        consumer, e);
+                                            }
+
+                                            messageReceived.increment();
+                                            long receiveTime = System.nanoTime();
+                                            if (arguments.isEnableTransaction) {
+                                                consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                        .thenRun(() -> {
+                                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                                    System.nanoTime() - receiveTime);
+                                                            messageAckRecorder.recordValue(latencyMicros);
+                                                            messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {

Review comment:
       Why should we handle this exception separately?




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713576008



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);
+                                                log.info("Receive message {} ", message);
+                                            } catch (Exception e) {
+                                                log.error("{} can`t receive message in 2 sec with exception {}",
+                                                        consumer, e);
+                                            }
+
+                                            messageReceived.increment();
+                                            long receiveTime = System.nanoTime();
+                                            if (arguments.isEnableTransaction) {
+                                                consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                        .thenRun(() -> {
+                                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                                    System.nanoTime() - receiveTime);
+                                                            messageAckRecorder.recordValue(latencyMicros);
+                                                            messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {

Review comment:
       when tasks are executed completely, `executorService.shutdownNow();` will be executed.  At this time, other threads may be executing, and an interrupt exception may occur.




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711585202



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -619,7 +660,7 @@ private static void runProducer(int producerId,
                         }
                     }
                     rateLimiter.acquire();
-
+                    semaphore.acquire();

Review comment:
       You mean rateLimiter is the same as semaphore ?




-- 
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] eolivelli commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r751147196



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -438,11 +550,27 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {

Review comment:
       please fix spaces "if(" -> "if ("
   here and in the other places




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711583469



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,11 +487,23 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTransaction = 0;
+            double averageTimePerTransaction = 0;
+            if (arguments.isEnableTransaction) {
+                totalTransaction = totalNumTransaction.sum();
+                averageTimePerTransaction = elapsed / numTransaction.sumThenReset();
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            String transactionLog = arguments.isEnableTransaction ? "---transaction: " + totalTransaction +

Review comment:
       It is very enlightening. Printing the total number in each log is more helpful for users to grasp the overall situation, instead of waiting for the end of each time to know the overall situation of the performance test. In fact, as long as there is no conflict, we should put more information on 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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713582788



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {

Review comment:
       Ok i will improve the code 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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r744514948



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();

Review comment:
       Yes, I have done this




-- 
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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r726975293



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,701 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for a new transaction to ack messages from consumer topics and produce message to "
+                + "producer topics, and then commit or abort this transaction. "
+                + "Increasing the number of threads increases the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means"
+                + "not trying to create a topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers. The default value is 1.")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
+                + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
+                + "consumes a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disabled, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in a transaction."
+                        + "If transaction disabled, it means the number of messages consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"--txn-disEnable"}, description = "Disable transaction")
+        public boolean isDisEnableTransaction = false;
+
+        @Parameter(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable "
+                + "setting to false, -abort takes effect)")
+        public boolean isAbortTransaction = false;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(!arguments.isDisEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (!arguments.isDisEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client,
+                            !arguments.isDisEnableTransaction, arguments.transactionTimeout);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        if (arguments.numTransactions > 0) {
+                            if (totalNumEndTxnOpFailed.sum()
+                                    + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+                        if (arguments.testTime > 0) {
+                            if (System.nanoTime() > testEndTime) {
+                                log.info("------------------- DONE -----------------------");
+                                executing.compareAndSet(true, false);
+                                executorService.shutdownNow();
+                                break;
+                            }
+                        }
+
+                        Transaction transaction = atomicReference.get();
+                        for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                for (Consumer<byte[]> consumer : subscriptions) {
+                                    for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) {
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                        } catch (PulsarClientException e) {
+                                            log.error("Receive message failed", e);
+                                            executorService.shutdownNow();
+                                        }
+                                        long receiveTime = System.nanoTime();
+                                        if (!arguments.isDisEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        numMessagesAckSuccess.increment();
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                numMessagesAckSuccess.increment();
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                }
+                            }
+                        }
+
+                        for(Producer<byte[]> producer : producers){
+                            for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) {
+                                long sendTime = System.nanoTime();
+                                if (!arguments.isDisEnableTransaction) {
+                                    producer.newMessage(transaction).value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send transaction message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                } else {
+                                    producer.newMessage().value(payloadBytes)
+                                            .sendAsync().thenRun(() -> {
+                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                System.nanoTime() - sendTime);
+                                        messageSendRecorder.recordValue(latencyMicros);
+                                        messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        numMessagesSendSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Send message failed with exception : ", exception);
+                                        numMessagesSendFailed.increment();
+                                        return null;
+                                    });
+                                }
+                            }
+                        }
+
+                        if(rateLimiter != null){
+                            rateLimiter.tryAcquire();
+                        }
+                        if (!arguments.isDisEnableTransaction) {
+                            if (!arguments.isAbortTransaction) {
+                                transaction.commit()
+                                        .thenRun(() -> {
+                                            numTxnOpSuccess.increment();
+                                            totalNumEndTxnOpSuccess.increment();
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Commit transaction {} failed with exception",
+                                                    transaction.getTxnID().toString(),
+                                                    exception);
+                                            totalNumEndTxnOpFailed.increment();
+                                            return null;
+                                        });
+                            } else {
+                                transaction.abort().thenRun(() -> {
+                                    numTxnOpSuccess.increment();
+                                    totalNumEndTxnOpSuccess.increment();
+                                }).exceptionally(exception -> {
+                                    if(exception instanceof InterruptedException && ! executing.get()){

Review comment:
       When the task execution is completed, executorService.shutdownNow() will be called to end the performance test
   




-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713581526



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,34 +497,90 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFailNum = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {
+                log.info(
+                        "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                                + "--- Transaction: {} transaction end successfully --- {} transaction end failed "
+                                + "--- {}  Txn/s --- AckRate: {} msg/s"
+                                + "--- Latency: mean: {} ms - med: {} "
+                                + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                        intFormat.format(total),
+                        dec.format(rate), dec.format(throughput),
+                        totalTxnOpSuccessNum,
+                        totalTxnOpFailNum,
+                        dec.format(rateOpenTxn),
+                        dec.format(rateAck),
+                        dec.format(reportHistogram.getMean()),
+                        reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
+                        reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
+                        reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
+            }else {
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                            + "--- Latency: mean: {} ms - med: {} "
+                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     dec.format(rate), dec.format(throughput), dec.format(reportHistogram.getMean()),
                     reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
                     reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
                     reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
-
             reportHistogram.reset();
             oldTime = now;
-        }
+        }}
 
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalEndTxnSuccess = 0;
+        long totalEndTxnFail = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double rateOpenTxn = 0;
+        if (arguments.isEnableTransaction) {
+            totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalEndTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed;
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+        if(arguments.isEnableTransaction){

Review comment:
       I think we only need to print the log added by transaction when enable, don't need change the original log logic.




-- 
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] Anonymitaet commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
Anonymitaet commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r725755283



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -250,6 +263,22 @@
 
         @Parameter(names = {"-fc", "--format-class"}, description="Custom Formatter class name")
         public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter";
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 10;
+
+        @Parameter(names = {"-nmt", "--numMessage-perTransaction"},
+                description = "The number of messages per transaction send. "

Review comment:
       ```suggestion
                   description = "The number of messages sent by a transaction. "
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")

Review comment:
       ```suggestion
                           + "If transaction disabled, it means the number of messages consumed in a task.")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -182,6 +198,26 @@
 
         @Parameter(names = {"-bw", "--busy-wait"}, description = "Enable Busy-Wait on the Pulsar client")
         public boolean enableBusyWait = false;
+
+        @Parameter(names = {"-tto", "--txn-timeout"},  description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
+                description = "The number of messages per transaction acknowledgment. "
+                + "(Only --txn-enable true can it take effect")
+        public int numMessagesPerTransaction = 50;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-ntxn"}, description = "The number of transaction will be opened, if 0, it will keep open."

Review comment:
       ```suggestion
           @Parameter(names = {"-ntxn"}, description = "The number of opened transactions. 0 means keeping open."
   ```
   
   Do you mean this?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -182,6 +198,26 @@
 
         @Parameter(names = {"-bw", "--busy-wait"}, description = "Enable Busy-Wait on the Pulsar client")
         public boolean enableBusyWait = false;
+
+        @Parameter(names = {"-tto", "--txn-timeout"},  description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect)")

Review comment:
       ```suggestion
                   + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)")
   ```
   
   Please check all occurrences 

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")

Review comment:
       ```suggestion
                   "--partitions"}, description = "Create partitioned topics with a given number of partitions. 0 means"
                   + "not trying to create a topic")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")

Review comment:
       ```suggestion
                   "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")

Review comment:
       ```suggestion
                   "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")

Review comment:
       ```suggestion
                   "used for handling connections to brokers. The default value is 1.")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")

Review comment:
       ```suggestion
                           + "If transaction disabled, it means the number of messages produced in a task.")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")

Review comment:
       ```suggestion
           @Parameter(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")

Review comment:
       ```suggestion
                   + "This thread is for a new transaction to ack the messages from consumer topics and produce message to "
                   + "producer topics, and then commit or abort this transaction. "
                   + "Increasing the number of threads increases the parallelism of the performance test, "
                   + "thereby increasing the intensity of the stress test.")
   ```
   
   Do you mean this?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")

Review comment:
       ```suggestion
                   "--number-txn"}, description = "Set the number of transaction. 0 means keeping open."
                   + "If transaction disabled, it means the number of tasks. The task or transaction produces or "
                   + "consumes a specified number of messages.")
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,697 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.pulsar.testclient.utils.PerformanceUtils.buildTransaction;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOpSuccess = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesAckSuccess = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numMessagesSendSuccess = new LongAdder();
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."

Review comment:
       ```suggestion
                   description = "Set the number of messages consumed in a transaction."
   ```




-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713580868



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);
+                                                log.info("Receive message {} ", message);
+                                            } catch (Exception e) {
+                                                log.error("{} can`t receive message in 2 sec with exception {}",
+                                                        consumer, e);
+                                            }
+
+                                            messageReceived.increment();
+                                            long receiveTime = System.nanoTime();
+                                            if (arguments.isEnableTransaction) {
+                                                consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                        .thenRun(() -> {
+                                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                                    System.nanoTime() - receiveTime);
+                                                            messageAckRecorder.recordValue(latencyMicros);
+                                                            messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {

Review comment:
       ok




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711582941



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,11 +487,23 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTransaction = 0;
+            double averageTimePerTransaction = 0;
+            if (arguments.isEnableTransaction) {
+                totalTransaction = totalNumTransaction.sum();
+                averageTimePerTransaction = elapsed / numTransaction.sumThenReset();
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            String transactionLog = arguments.isEnableTransaction ? "---transaction: " + totalTransaction +

Review comment:
       emm~, If  totalEndTxnOpFailNum and totalEndTxnOpSuccessNum print in printAggregatedThroughput can be better?This is an instant data, not a conclusive data. The number of failed commit transactions should be relatively small, compared to printing the number of failures within this time at a fixed time, is it more valuable to print the last one?




-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r705493776



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -284,38 +316,9 @@ public static void main(String[] args) throws Exception {
         final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null;
         long startTime = System.nanoTime();
         long testEndTime = startTime + (long) (arguments.testTime * 1e9);
-        MessageListener<ByteBuffer> listener = (consumer, msg) -> {

Review comment:
       I think this code should not be moved, which makes review difficult




-- 
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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-951475812


   /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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r751201069



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -438,11 +550,27 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {

Review comment:
       Thank you for pointing it out. This is indeed my mistake. I will correct it later.




-- 
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] congbobo184 merged pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 merged pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933


   


-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r705497211



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -333,8 +336,65 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageTotal = new AtomicLong(0);
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg);
+                }
+                if (arguments.poolMessages) {
+                    msg.release();
+                }
+                if (arguments.isEnableTransaction

Review comment:
       It doesn't seem to prevent ack with old transaction here. It seems that there have race condition.




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711587721



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){

Review comment:
       Is the number of messages sent or received is not fixed in a transaction? The messageReceived here is  one per test thread, reset when the transaction is over, a test thread only executes one transaction at the same 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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711581923



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -424,15 +517,38 @@ public static void main(String[] args) throws Exception {
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalTransaction = 0;
+        long totalTransactionFailed = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double averageTimePerTransaction = 0;
+        if (arguments.isEnableTransaction) {
+            totalTransaction = totalNumTransaction.sum();
+            averageTimePerTransaction = elapsed / totalNumTransaction.sum();
+            totalTransactionFailed = totalNumTransactionFailed.sum();
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+
+            String commitOrAbortTransaction = arguments.isEnableTransaction
+                    ? " transaction commit --- " + totalTransactionFailed + "transaction commit Failed --- "

Review comment:
        Here should  be IsCommitTransaction.   Sorry, I didn't check it out. The log will be different when transaction commit or abort. 




-- 
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 #11933: [Transaction] Add transaction perf

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


   @congbobo184 @eolivelli Could you please help review the 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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711560028



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -181,6 +190,24 @@
 
         @Parameter(names = {"-bw", "--busy-wait"}, description = "Enable Busy-Wait on the Pulsar client")
         public boolean enableBusyWait = false;
+
+        @Parameter(names = {"-tto", "--txn-timeout"},  description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect)")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
+                description = "The number of messages per transaction acknowledgment. "
+                + "(Only --txn-enable true can it take effect")
+        public int numMessagesPerTransaction = 50;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "

Review comment:
       how about `-commit`

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,11 +487,23 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTransaction = 0;
+            double averageTimePerTransaction = 0;
+            if (arguments.isEnableTransaction) {
+                totalTransaction = totalNumTransaction.sum();
+                averageTimePerTransaction = elapsed / numTransaction.sumThenReset();
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            String transactionLog = arguments.isEnableTransaction ? "---transaction: " + totalTransaction +
+                    " transaction commit --- " + averageTimePerTransaction + " s/perTxn --- AckRate: " + rateAck
+                    + " msg/s" : "";
+
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",

Review comment:
       don't change this log, if enable transaction, we can create a new log.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {

Review comment:
       does this rate should use in openTxn, not here.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -284,7 +311,36 @@ public static void main(String[] args) throws Exception {
         final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null;
         long startTime = System.nanoTime();
         long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+
+        ClientBuilder clientBuilder = PulsarClient.builder() //
+                .enableTransaction(arguments.isEnableTransaction)
+                .serviceUrl(arguments.serviceURL) //
+                .connectionsPerBroker(arguments.maxConnections) //
+                .statsInterval(arguments.statsIntervalSeconds, TimeUnit.SECONDS) //
+                .ioThreads(arguments.ioThreads) //
+                .enableBusyWait(arguments.enableBusyWait)
+                .tlsTrustCertsFilePath(arguments.tlsTrustCertsFilePath);
+        if (isNotBlank(arguments.authPluginClassName)) {
+            clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams);
+        }
+
+        if (arguments.tlsAllowInsecureConnection != null) {
+            clientBuilder.allowTlsInsecureConnection(arguments.tlsAllowInsecureConnection);
+        }
+
+        if (isNotBlank(arguments.listenerName)) {
+            clientBuilder.listenerName(arguments.listenerName);
+        }
+        PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        LongAdder messageAckedCount = new LongAdder();

Review comment:
       messageAckedCount may should use `AtomicLong`, `LongAdder` not imprecise

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -307,34 +363,59 @@ public static void main(String[] args) throws Exception {
                 recorder.recordValue(latencyMillis);
                 cumulativeRecorder.recordValue(latencyMillis);
             }
+                if (arguments.isEnableTransaction) {
+                consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                        messageAckedCount.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                        messageAckedCount.increment();
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAckedCount.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                                messageAckedCount.increment();
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
 
-            consumer.acknowledgeAsync(msg);
-
-            if(arguments.poolMessages) {
-                msg.release();
+                if (arguments.isEnableTransaction
+                        && messageAckedCount.sum() == arguments.numMessagesPerTransaction) {

Review comment:
       Here you cannot guarantee that only one is updating the transaction. There should use `messageAckedCount.incrementAndGet()`. 

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -307,34 +363,59 @@ public static void main(String[] args) throws Exception {
                 recorder.recordValue(latencyMillis);
                 cumulativeRecorder.recordValue(latencyMillis);
             }
+                if (arguments.isEnableTransaction) {
+                consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                        messageAckedCount.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                        messageAckedCount.increment();
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAckedCount.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                                messageAckedCount.increment();
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
 
-            consumer.acknowledgeAsync(msg);
-
-            if(arguments.poolMessages) {
-                msg.release();
+                if (arguments.isEnableTransaction
+                        && messageAckedCount.sum() == arguments.numMessagesPerTransaction) {
+                    Transaction transaction = atomicReference.get();
+                    if (atomicReference.compareAndSet(transaction, pulsarClient.newTransaction().
+                            withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get())) {
+                        if (arguments.isCommitTransaction) {
+                            transaction.commit().exceptionally(exception -> {
+                                log.error("Commit transaction failed with exception : " + exception.getMessage());
+                                totalNumTransactionFailed.increment();
+                                return null;
+                            });
+                        } else {
+                            transaction.abort(); }
+                        messageAckedCount.reset();
+                        semaphore.release(arguments.numMessagesPerTransaction);
+                        totalNumTransaction.increment();
+                        numTransaction.increment();
+                    }
+                }
+            } catch (Exception e) {
+                log.error("Exception  got in listener : " + e.getMessage());

Review comment:
       ```log.error("Consume fail .", e);```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -424,15 +517,38 @@ public static void main(String[] args) throws Exception {
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalTransaction = 0;
+        long totalTransactionFailed = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double averageTimePerTransaction = 0;
+        if (arguments.isEnableTransaction) {
+            totalTransaction = totalNumTransaction.sum();
+            averageTimePerTransaction = elapsed / totalNumTransaction.sum();
+            totalTransactionFailed = totalNumTransactionFailed.sum();
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+
+            String commitOrAbortTransaction = arguments.isEnableTransaction
+                    ? " transaction commit --- " + totalTransactionFailed + "transaction commit Failed --- "

Review comment:
       what that mean `transaction commit --- " + totalTransactionFailed`?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -307,34 +363,59 @@ public static void main(String[] args) throws Exception {
                 recorder.recordValue(latencyMillis);
                 cumulativeRecorder.recordValue(latencyMillis);
             }
+                if (arguments.isEnableTransaction) {
+                consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                        messageAckedCount.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                        messageAckedCount.increment();
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAckedCount.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                                messageAckedCount.increment();
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
 
-            consumer.acknowledgeAsync(msg);
-
-            if(arguments.poolMessages) {
-                msg.release();
+                if (arguments.isEnableTransaction
+                        && messageAckedCount.sum() == arguments.numMessagesPerTransaction) {
+                    Transaction transaction = atomicReference.get();
+                    if (atomicReference.compareAndSet(transaction, pulsarClient.newTransaction().
+                            withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get())) {
+                        if (arguments.isCommitTransaction) {
+                            transaction.commit().exceptionally(exception -> {
+                                log.error("Commit transaction failed with exception : " + exception.getMessage());
+                                totalNumTransactionFailed.increment();

Review comment:
       totalEndTxnOpFailNum and totalEndTxnOpSuccessNum

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -307,34 +363,59 @@ public static void main(String[] args) throws Exception {
                 recorder.recordValue(latencyMillis);
                 cumulativeRecorder.recordValue(latencyMillis);
             }
+                if (arguments.isEnableTransaction) {
+                consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                        messageAckedCount.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                        messageAckedCount.increment();
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAckedCount.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                                messageAckedCount.increment();
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
 
-            consumer.acknowledgeAsync(msg);
-
-            if(arguments.poolMessages) {
-                msg.release();
+                if (arguments.isEnableTransaction
+                        && messageAckedCount.sum() == arguments.numMessagesPerTransaction) {
+                    Transaction transaction = atomicReference.get();
+                    if (atomicReference.compareAndSet(transaction, pulsarClient.newTransaction().
+                            withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get())) {
+                        if (arguments.isCommitTransaction) {
+                            transaction.commit().exceptionally(exception -> {
+                                log.error("Commit transaction failed with exception : " + exception.getMessage());
+                                totalNumTransactionFailed.increment();
+                                return null;
+                            });
+                        } else {
+                            transaction.abort(); }

Review comment:
       The format here is not right

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,11 +487,23 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTransaction = 0;
+            double averageTimePerTransaction = 0;
+            if (arguments.isEnableTransaction) {
+                totalTransaction = totalNumTransaction.sum();
+                averageTimePerTransaction = elapsed / numTransaction.sumThenReset();
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            String transactionLog = arguments.isEnableTransaction ? "---transaction: " + totalTransaction +

Review comment:
       we can print totalEndTxnOpFailNum and totalEndTxnOpSuccessNum.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -619,7 +660,7 @@ private static void runProducer(int producerId,
                         }
                     }
                     rateLimiter.acquire();
-
+                    semaphore.acquire();

Review comment:
       the same as above

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);

Review comment:
       add final

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -597,7 +634,11 @@ private static void runProducer(int producerId,
             }
             // Send messages on all topics/producers
             long totalSent = 0;
+            AtomicReference<Transaction> transactionAtomicReference = buildTransaction(client, arguments);
+            AtomicLong numMessageSend = new AtomicLong(0);
+            Semaphore semaphore = new Semaphore(arguments.numMessagesPerTransaction);
             while (true) {
+                Transaction transaction = transactionAtomicReference.get();

Review comment:
       only enable transaction need this right?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -659,14 +706,40 @@ private static void runProducer(int producerId,
                             recorder.recordValue(latencyMicros);
                             cumulativeRecorder.recordValue(latencyMicros);
                         }
+                        if (arguments.isEnableTransaction
+                                && numMessageSend.incrementAndGet() == arguments.numMessagesPerTransaction) {
+                            try {
+                                if (transactionAtomicReference.compareAndSet(transaction,
+                                        pulsarClient.newTransaction()
+                                                .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS)
+                                                .build().get())) {
+                                    if (arguments.isCommitTransaction) {
+                                        transaction.commit().exceptionally(exception -> {
+                                            log.error("Commit transaction failed with exception : "
+                                                    + exception.getMessage());
+                                            totalNumTransactionFailed.increment();

Review comment:
       the same of PerformanceConsumer.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){

Review comment:
       every consumer receive `numMessagesReceivedPerTransaction` number messages and ack, this logic is not correct.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive(2, TimeUnit.SECONDS);
+                                            log.info("Receive message {} ", message);
+                                        }catch (Exception e){
+                                            log.error("{} can`t receive message in 2 sec with exception {}", consumer, e);
+                                        }
+
+                                        messageReceived.increment();
+                                        long receiveTime = System.nanoTime();
+                                        if (arguments.isEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                    }).exceptionally(exception -> {
+                                                log.error("Ack message failed with transaction {} throw exception {}", transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                            }).exceptionally(exception -> {
+                                                log.error("Ack message failed with transaction {} throw exception {}", transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                    }
+                                }
+
+                                for(Producer<byte[]> producer : producers){
+                                    if(messageSend.sum() >= arguments.numMessagesProducedPerTransaction){
+                                        break;
+                                    }
+                                    long sendTime = System.nanoTime();
+                                    messageSend.increment();
+                                    if (arguments.isEnableTransaction) {
+                                        producer.newMessage(transaction).value(payloadBytes)

Review comment:
       same as abrove

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -307,34 +363,59 @@ public static void main(String[] args) throws Exception {
                 recorder.recordValue(latencyMillis);
                 cumulativeRecorder.recordValue(latencyMillis);
             }
+                if (arguments.isEnableTransaction) {
+                consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                        messageAckedCount.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                        messageAckedCount.increment();
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAckedCount.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception {}", msg, throwable.getMessage());
+                                messageAckedCount.increment();
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
 
-            consumer.acknowledgeAsync(msg);
-
-            if(arguments.poolMessages) {
-                msg.release();
+                if (arguments.isEnableTransaction
+                        && messageAckedCount.sum() == arguments.numMessagesPerTransaction) {
+                    Transaction transaction = atomicReference.get();
+                    if (atomicReference.compareAndSet(transaction, pulsarClient.newTransaction().
+                            withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get())) {
+                        if (arguments.isCommitTransaction) {
+                            transaction.commit().exceptionally(exception -> {
+                                log.error("Commit transaction failed with exception : " + exception.getMessage());
+                                totalNumTransactionFailed.increment();
+                                return null;
+                            });
+                        } else {
+                            transaction.abort(); }

Review comment:
       totalEndTxnOpFailNum and totalEndTxnOpSuccessNum

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {

Review comment:
       use numTestThreads is correct, why use `testThreadLimit.tryAcquire()` 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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713583510



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,34 +497,90 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFailNum = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {
+                log.info(
+                        "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                                + "--- Transaction: {} transaction end successfully --- {} transaction end failed "
+                                + "--- {}  Txn/s --- AckRate: {} msg/s"
+                                + "--- Latency: mean: {} ms - med: {} "
+                                + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                        intFormat.format(total),
+                        dec.format(rate), dec.format(throughput),
+                        totalTxnOpSuccessNum,
+                        totalTxnOpFailNum,
+                        dec.format(rateOpenTxn),
+                        dec.format(rateAck),
+                        dec.format(reportHistogram.getMean()),
+                        reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
+                        reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
+                        reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
+            }else {
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                            + "--- Latency: mean: {} ms - med: {} "
+                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     dec.format(rate), dec.format(throughput), dec.format(reportHistogram.getMean()),
                     reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
                     reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
                     reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
-
             reportHistogram.reset();
             oldTime = now;
-        }
+        }}
 
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalEndTxnSuccess = 0;
+        long totalEndTxnFail = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double rateOpenTxn = 0;
+        if (arguments.isEnableTransaction) {
+            totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalEndTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed;
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+        if(arguments.isEnableTransaction){

Review comment:
       Is there any logic to change the original log?




-- 
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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927285006


   /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] lhotari commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-1028996287


   this PR introduced a flaky test, reported as #14109 . Please fix


-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711585202



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -619,7 +660,7 @@ private static void runProducer(int producerId,
                         }
                     }
                     rateLimiter.acquire();
-
+                    semaphore.acquire();

Review comment:
       Um, yes it is missing 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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r705837002



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -659,6 +723,27 @@ private static void runProducer(int producerId,
                             recorder.recordValue(latencyMicros);
                             cumulativeRecorder.recordValue(latencyMicros);
                         }
+                        if (arguments.isEnableTransaction

Review comment:
       Same problem as PerformanceConsumer




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711584853



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -597,7 +634,11 @@ private static void runProducer(int producerId,
             }
             // Send messages on all topics/producers
             long totalSent = 0;
+            AtomicReference<Transaction> transactionAtomicReference = buildTransaction(client, arguments);
+            AtomicLong numMessageSend = new AtomicLong(0);
+            Semaphore semaphore = new Semaphore(arguments.numMessagesPerTransaction);
             while (true) {
+                Transaction transaction = transactionAtomicReference.get();

Review comment:
       But now it seems that it has lost its legibility and is very unfriendly to code readers




-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713579108



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {

Review comment:
       we don’t need to do this, executorService has done this.




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711587721



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){

Review comment:
       Is the number of messages sent or received is not fixed in a transaction? The messageReceived here is a test thread, reset when the transaction is over, a test thread only executes one transaction at the same 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] congbobo184 commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927459448


   /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] liangyepianzhou removed a comment on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-951430447






-- 
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] eolivelli commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-941066355


   I am moving this PR to 2.10.0
   we could possibly cherry pick to 2.9.1
   
   I believe that it is not useful to add this to 2.8 branch as transactions evolved 


-- 
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] hangc0276 commented on pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-913925595


   Can we divide produce and consume in separate file? Users can test produce with transaction and consume with transaction individually. Otherwise, we may add a script in bin/pulsar-perf to start the transaction perf


-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713577476



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,34 +497,90 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFailNum = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {
+                log.info(
+                        "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                                + "--- Transaction: {} transaction end successfully --- {} transaction end failed "
+                                + "--- {}  Txn/s --- AckRate: {} msg/s"
+                                + "--- Latency: mean: {} ms - med: {} "
+                                + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                        intFormat.format(total),
+                        dec.format(rate), dec.format(throughput),
+                        totalTxnOpSuccessNum,
+                        totalTxnOpFailNum,
+                        dec.format(rateOpenTxn),
+                        dec.format(rateAck),
+                        dec.format(reportHistogram.getMean()),
+                        reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
+                        reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
+                        reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
+            }else {
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                            + "--- Latency: mean: {} ms - med: {} "
+                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     dec.format(rate), dec.format(throughput), dec.format(reportHistogram.getMean()),
                     reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
                     reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
                     reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
-
             reportHistogram.reset();
             oldTime = now;
-        }
+        }}
 
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalEndTxnSuccess = 0;
+        long totalEndTxnFail = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double rateOpenTxn = 0;
+        if (arguments.isEnableTransaction) {
+            totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalEndTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed;
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+        }
+        if(arguments.isEnableTransaction){

Review comment:
       I kept the original log logic in else.Isn't it done this way?




-- 
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] congbobo184 commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-929815808


   @eolivelli please review again, 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] liangyepianzhou commented on pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-913594641


   > Could we reuse the existing perf components? Such as `PerformanceClient`, `PerformanceProducer` and `PerformanceConsumer`.
   
   Their logic is different, and they did not extract the generic component. I don't think it's suitable


-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711581327



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,11 +487,23 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTransaction = 0;
+            double averageTimePerTransaction = 0;
+            if (arguments.isEnableTransaction) {
+                totalTransaction = totalNumTransaction.sum();
+                averageTimePerTransaction = elapsed / numTransaction.sumThenReset();
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            String transactionLog = arguments.isEnableTransaction ? "---transaction: " + totalTransaction +
+                    " transaction commit --- " + averageTimePerTransaction + " s/perTxn --- AckRate: " + rateAck
+                    + " msg/s" : "";
+
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",

Review comment:
       If transaction disable, the log will not be changed. We really need to create a new log?




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711586835



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {

Review comment:
       Here has a conflict : a test thread will only open a transaction at one time. Maybe this should be delete?




-- 
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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r705497211



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -333,8 +336,65 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageTotal = new AtomicLong(0);
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg);
+                }
+                if (arguments.poolMessages) {
+                    msg.release();
+                }
+                if (arguments.isEnableTransaction

Review comment:
       It doesn't seem to prevent ack with old transaction here. It seems that there have race condition.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -189,12 +193,16 @@
         public long transactionTimeout = 5;
 
         @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
-                description = "the number of a transaction produced")
+                description = "the number of messages of consumed by a transaction")

Review comment:
       `The number of messages per transaction acknowledgment. (Only `--txn-enable true` can it take effect)`is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -333,8 +336,65 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageTotal = new AtomicLong(0);
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();

Review comment:
       I don't see where totalMessageAck is used.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");

Review comment:
       consume topic should be `test-consume`

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+

Review comment:
       Don't add extra blank lines

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",

Review comment:
       This should be --topic-p, `All topics that needproduce for a transaction`

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -241,15 +262,28 @@
         @Parameter(names = {"-bw", "--busy-wait"}, description = "Enable Busy-Wait on the Pulsar client")
         public boolean enableBusyWait = false;
 
-        @Parameter(names = { "-am", "--access-mode" }, description = "Producer access mode")
+        @Parameter(names = {"-am", "--access-mode"}, description = "Producer access mode")
         public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared;
 
-        @Parameter(names = { "-fp", "--format-payload" },
-                description = "Format %i as a message index in the stream from producer and/or %t as the timestamp nanoseconds.")
+        @Parameter(names = {"-fp", "--format-payload"},
+                description = "Format %i as a message index in the stream from producer and/or %t as the timestamp "
+                        + "nanoseconds.")
         public boolean formatPayload = false;
 
-        @Parameter(names = {"-fc", "--format-class"}, description="Custom Formatter class name")
+        @Parameter(names = {"-fc", "--format-class"}, description = "Custom Formatter class name")
         public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter";
+
+        @Parameter(names = {"-to", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesPerTransaction = 50;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")

Review comment:
       Enable or disable the transaction is better.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,612 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")
+        public boolean isCommitedTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "the rate of transaction  open, if 0 , don`t limit")
+        public int openTxnRate = 0;
+
+
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf produce");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+
+
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+                for (String topic : arguments.consumerTopic) {
+                    log.info("Creating  consume partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction).serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .ioThreads(arguments.ioThreads)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads).build();
+
+        ProducerBuilder<byte[]> producerBuilder = client.newProducer()//
+                .sendTimeout(0, TimeUnit.SECONDS);
+
+        final List<Future<Producer<byte[]>>> producerFutures = Lists.newArrayList();
+
+
+        ConsumerBuilder<byte[]> consumerBuilder = client.newConsumer(Schema.BYTES) //
+                .subscriptionType(arguments.subscriptionType)
+                .receiverQueueSize(arguments.receiverQueueSize)
+                .subscriptionInitialPosition(arguments.subscriptionInitialPosition)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest);
+
+
+        Iterator<String> consumerTopicIterator = arguments.consumerTopic.iterator();
+        Iterator<String> producerTopicIterator = arguments.producerTopic.iterator();
+
+        final List<List<Consumer<byte[]>>> consumers = Lists.newArrayList();
+        for (int i = 0; i < arguments.numTestThreads; i++) {
+            final List<Consumer<byte[]>> subscriptions = Lists.newArrayListWithCapacity(arguments.numSubscriptions);

Review comment:
       Maybe we shouldn't add two lists

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")

Review comment:
       Whether to commit or abort the transaction. (Only "--txn-enable true" can it take effect) is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -189,12 +193,16 @@
         public long transactionTimeout = 5;
 
         @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
-                description = "the number of a transaction produced")
+                description = "the number of messages of consumed by a transaction")
         public int numMessagesPerTransaction = 50;
 
-        @Parameter(names = {"-txn", "--txn-enable"}, description = "transaction enable")
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "whether transactions need to  be opened")
         public boolean isEnableTransaction = false;
 
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")

Review comment:
       `Whether to commit or abort the transaction. (Only "--txn-enable true" can it take effect)` is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -67,120 +68,149 @@
     private static final LongAdder totalMessagesReceived = new LongAdder();
     private static final LongAdder totalBytesReceived = new LongAdder();
 
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
     private static Recorder recorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5);
     private static Recorder cumulativeRecorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5);
 
     @Parameters(commandDescription = "Test pulsar consumer performance.")
     static class Arguments {
 
-        @Parameter(names = { "-h", "--help" }, description = "Help message", help = true)

Review comment:
       Do not change the format, the following are all. You should probably double check when submitting

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -189,12 +193,16 @@
         public long transactionTimeout = 5;
 
         @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
-                description = "the number of a transaction produced")
+                description = "the number of messages of consumed by a transaction")
         public int numMessagesPerTransaction = 50;
 
-        @Parameter(names = {"-txn", "--txn-enable"}, description = "transaction enable")
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "whether transactions need to  be opened")

Review comment:
       `Enable or disable the transaction` is better.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -284,38 +316,9 @@ public static void main(String[] args) throws Exception {
         final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null;
         long startTime = System.nanoTime();
         long testEndTime = startTime + (long) (arguments.testTime * 1e9);
-        MessageListener<ByteBuffer> listener = (consumer, msg) -> {

Review comment:
       I think this code should not be moved, which makes review difficult

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");

Review comment:
       produce topic should be test-produce

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -333,8 +336,65 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageTotal = new AtomicLong(0);
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg);
+                }
+                if (arguments.poolMessages) {
+                    msg.release();
+                }
+                if (arguments.isEnableTransaction
+                        && messageTotal.incrementAndGet() >= arguments.numMessagesPerTransaction) {
+                    Transaction transaction = atomicReference.get();
+                    if (atomicReference.compareAndSet(transaction, pulsarClient.newTransaction().
+                            withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get())) {
+                        messageTotal.set(0);
+                        if (arguments.isCommitedTransaction) {
+                            transaction.commit();

Review comment:
       Maybe we should add the statistics of the number of commits and aborts

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =

Review comment:
       `All topics that need ack for a transaction` is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -333,8 +336,65 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageTotal = new AtomicLong(0);
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {

Review comment:
       Maybe we should add ack failure statistics

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;

Review comment:
       numMessagesAckedPerTransaction

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},

Review comment:
       The number of messages per transaction acknowledgment. (Only --txn-enable true can it take effect)

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -241,15 +262,28 @@
         @Parameter(names = {"-bw", "--busy-wait"}, description = "Enable Busy-Wait on the Pulsar client")
         public boolean enableBusyWait = false;
 
-        @Parameter(names = { "-am", "--access-mode" }, description = "Producer access mode")
+        @Parameter(names = {"-am", "--access-mode"}, description = "Producer access mode")
         public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared;
 
-        @Parameter(names = { "-fp", "--format-payload" },
-                description = "Format %i as a message index in the stream from producer and/or %t as the timestamp nanoseconds.")
+        @Parameter(names = {"-fp", "--format-payload"},
+                description = "Format %i as a message index in the stream from producer and/or %t as the timestamp "
+                        + "nanoseconds.")
         public boolean formatPayload = false;
 
-        @Parameter(names = {"-fc", "--format-class"}, description="Custom Formatter class name")
+        @Parameter(names = {"-fc", "--format-class"}, description = "Custom Formatter class name")
         public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter";
+
+        @Parameter(names = {"-to", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesPerTransaction = 50;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")

Review comment:
       Whether to commit or abort the transaction. (Only "--txn-enable true" can it take effect) is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -659,6 +723,27 @@ private static void runProducer(int producerId,
                             recorder.recordValue(latencyMicros);
                             cumulativeRecorder.recordValue(latencyMicros);
                         }
+                        if (arguments.isEnableTransaction

Review comment:
       Same problem as PerformanceConsumer

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")

Review comment:
       The number of transactions, if 0 it will keep opening

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")

Review comment:
       This thread is for new transaction and ack the `consumerTopic` message and produce message to `producerTopic ` then commit or abort this transaction. Increasing the number of threads will increase the parallelism of the performance test, thereby increasing the intensity of the stress test.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",

Review comment:
       txn

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")
+        public boolean isCommitedTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "the rate of transaction  open, if 0 , don`t limit")
+        public int openTxnRate = 0;
+
+
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {

Review comment:
       throw `Exception` is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")

Review comment:
       "Number of subscriptions (per consumerTopic)"

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},

Review comment:
       same as follows

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")
+        public boolean isCommitedTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "the rate of transaction  open, if 0 , don`t limit")
+        public int openTxnRate = 0;
+
+
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf produce");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+
+
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+                for (String topic : arguments.consumerTopic) {
+                    log.info("Creating  consume partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction).serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .ioThreads(arguments.ioThreads)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads).build();
+
+        ProducerBuilder<byte[]> producerBuilder = client.newProducer()//
+                .sendTimeout(0, TimeUnit.SECONDS);

Review comment:
       enable transaction should .sendTimeout(0, TimeUnit.SECONDS);

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,613 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+        @Parameter(names = {"--separator"}, description = "Separator between the topic and topic number")
+        public String separator = "-";
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")

Review comment:
       Enable or disable the transaction is better.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,612 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+import java.io.FileInputStream;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder totalMessageAck = new LongAdder();
+    private static final LongAdder messageAck = new LongAdder();
+
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topic-c", description = "consumer will be created to consumer this topic", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("sub");
+
+        @Parameter(names = "--topic-c", description = "producer will be created to produce message to this topic",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads")
+        public int numTestThreads = 1;
+
+
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest;
+
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Exclusive;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+
+        @Parameter(names = {"-timeout", "--txn-timeout"}, description = "transaction timeout")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "the number of messages produced by  a transaction")
+        public int numMessagesProducedPerTransaction = 1;
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "the number of messages consumed by  a transaction")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-ntnx",
+                "--number-txn"}, description = "the number of transaction, if o, it will keep opening")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = " whether transactions need to  be opened ")
+        public boolean isEnableTransaction = false;
+
+        @Parameter(names = {"-end"}, description = " how to end a transaction, commit or abort")
+        public boolean isCommitedTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "the rate of transaction  open, if 0 , don`t limit")
+        public int openTxnRate = 0;
+
+
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf produce");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+
+
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+                for (String topic : arguments.consumerTopic) {
+                    log.info("Creating  consume partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction).serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .ioThreads(arguments.ioThreads)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads).build();
+
+        ProducerBuilder<byte[]> producerBuilder = client.newProducer()//
+                .sendTimeout(0, TimeUnit.SECONDS);
+
+        final List<Future<Producer<byte[]>>> producerFutures = Lists.newArrayList();
+
+
+        ConsumerBuilder<byte[]> consumerBuilder = client.newConsumer(Schema.BYTES) //
+                .subscriptionType(arguments.subscriptionType)
+                .receiverQueueSize(arguments.receiverQueueSize)
+                .subscriptionInitialPosition(arguments.subscriptionInitialPosition)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest);
+
+
+        Iterator<String> consumerTopicIterator = arguments.consumerTopic.iterator();
+        Iterator<String> producerTopicIterator = arguments.producerTopic.iterator();
+
+        final List<List<Consumer<byte[]>>> consumers = Lists.newArrayList();
+        for (int i = 0; i < arguments.numTestThreads; i++) {
+            final List<Consumer<byte[]>> subscriptions = Lists.newArrayListWithCapacity(arguments.numSubscriptions);
+            final List<Future<Consumer<byte[]>>> subscriptionFutures =
+                    Lists.newArrayListWithCapacity(arguments.numSubscriptions);
+            if (consumerTopicIterator.hasNext()) {

Review comment:
       In my understanding, each thread needs to create a corresponding number of consumers and producers. Maybe it should be a while loop.




-- 
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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r726864478



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/utils/PerformanceUtils.java
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.testclient.utils;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.testclient.PerformanceProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(PerformanceProducer.class);
+
+    public static AtomicReference<Transaction> buildTransaction(PulsarClient pulsarClient, boolean isEnableTransaction,
+                                                                long transactionTimeout) {
+        if (isEnableTransaction) {
+            while(true) {
+                AtomicReference atomicReference = null;
+                try {
+                    atomicReference = new AtomicReference(pulsarClient.newTransaction()
+                            .withTransactionTimeout(transactionTimeout, TimeUnit.SECONDS).build().get());
+                } catch (Exception e) {
+                    log.error("Failed to new transaction", e);

Review comment:
       Thank you very much for your suggestions, it is of great help to my work. 
   I will improve it later.




-- 
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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-939902233


   /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] congbobo184 commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r705472657



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -189,12 +193,16 @@
         public long transactionTimeout = 5;
 
         @Parameter(names = {"-nmt", "---numMessage-perTransaction"},
-                description = "the number of a transaction produced")
+                description = "the number of messages of consumed by a transaction")

Review comment:
       `The number of messages per transaction acknowledgment. (Only --txn-enable true can it take effect)`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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927321347


   /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] liangyepianzhou removed a comment on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou removed a comment on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927285006


   /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] congbobo184 commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r716126385



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -340,9 +344,123 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
 
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments.isEnableTransaction,
+                arguments.transactionTimeout);
+
+        AtomicLong messageAckedCount = new AtomicLong();
+        Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction);
+        Thread thread = Thread.currentThread();
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+                if(arguments.isEnableTransaction){
+                    try {
+                        messageReceiveLimiter.acquire();
+                    }catch (InterruptedException e){
+                        log.error("Got error: ", e);
+                    }
+                    }
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                if(arguments.totalNumTxn > 0) {
+                    if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= arguments.totalNumTxn) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception", msg, throwable);
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception", msg, throwable);
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
+                if (arguments.isEnableTransaction

Review comment:
       why use repeat arguments.isEnableTransaction

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -619,7 +669,15 @@ private static void runProducer(int producerId,
                         }
                     }
                     rateLimiter.acquire();
-
+                    if(arguments.isEnableTransaction && arguments.numMessagesPerTransaction > 0){

Review comment:
       the same as performanceConsumer

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message);
+        }
+        boolean noMessage = false;
+        for (int i = 0; i < 10; i++) {
+           Message message = consumeFromConsumeTopic.receive(2, TimeUnit.SECONDS);

Review comment:
       should receive 10 message and can't receive 11 messages

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {

Review comment:
       should add sub before send message, otherwise test will become unstable

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -406,33 +537,78 @@ public static void main(String[] args) throws Exception {
             long total = totalMessagesReceived.sum();
             double rate = messagesReceived.sumThenReset() / elapsed;
             double throughput = bytesReceived.sumThenReset() / elapsed * 8 / 1024 / 1024;
-
+            double rateAck = messageAck.sumThenReset() / elapsed;
+            long totalTxnOpSuccessNum = 0;
+            long totalTxnOpFailNum = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFailNum = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if(arguments.isEnableTransaction) {
+                log.info("--- Transaction: {} transaction end successfully --- {} transaction end failed "
+                                + "--- {}  Txn/s --- AckRate: {} msg/s",
+                        totalTxnOpSuccessNum,
+                        totalTxnOpFailNum,
+                        dec.format(rateOpenTxn),
+                        dec.format(rateAck));
+            }
             log.info(
-                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput received: {} msg --- {}  msg/s -- {} Mbit/s  "
+                            + "--- Latency: mean: {} ms - med: {} "
+                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     dec.format(rate), dec.format(throughput), dec.format(reportHistogram.getMean()),
                     reportHistogram.getValueAtPercentile(50), reportHistogram.getValueAtPercentile(95),
                     reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9),
                     reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue());
-
             reportHistogram.reset();
             oldTime = now;
         }
 
         pulsarClient.close();
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesReceived.sum() / elapsed;
         double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024;
+        long totalEndTxnSuccess = 0;
+        long totalEndTxnFail = 0;
+        long numTransactionOpenFailed = 0;
+        long numTransactionOpenSuccess = 0;
+        long totalnumMessageAckFailed = 0;
+        double rateAck = totalMessageAck.sum() / elapsed;
+        double rateOpenTxn = 0;
+        if (arguments.isEnableTransaction) {
+            totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalEndTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed;
+            totalnumMessageAckFailed = totalMessageAckFailed.sum();
+            numTransactionOpenFailed = totalNumTxnOpenTxnFail.sum();
+            numTransactionOpenSuccess = totalNumTxnOpenTxnSuccess.sum();
+        }
+        if(arguments.isEnableTransaction){

Review comment:
       two if block is repeat

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -680,20 +788,44 @@ private static void runProducer(int producerId,
             if (null != client) {
                 try {
                     client.close();
+                    PerfClientUtils.exit(-1);
                 } catch (PulsarClientException e) {
                     log.error("Failed to close test client", e);
                 }
             }
         }
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesSent.sum() / elapsed;
         double throughput = totalBytesSent.sum() / elapsed / 1024 / 1024 * 8;
+        long totalTxnSuccess = 0;
+        long totalTxnFail = 0;
+        double rateOpenTxn = 0;
+        long numTransactionOpenFailed = 0;
+        long numTransactionOpenSuccess = 0;
+        if (arguments.isEnableTransaction) {
+            totalTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = elapsed / (totalTxnFail + totalTxnSuccess);
+            numTransactionOpenFailed = totalNumTxnOpenTxnFail.sum();
+            numTransactionOpenSuccess = totalNumTxnOpenTxnSuccess.sum();
+        }
+
+        if(arguments.isEnableTransaction){

Review comment:
       arguments.isEnableTransaction

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);

Review comment:
       should receive 500 messages, I suggest you can send 50 messages to test.

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message);
+        }
+        boolean noMessage = false;
+        for (int i = 0; i < 10; i++) {
+           Message message = consumeFromConsumeTopic.receive(2, TimeUnit.SECONDS);
+           if(message == null){
+               noMessage = true;
+           }
+        }
+        Assert.assertTrue(noMessage);
+    }
+
+
+    @Test
+    public void testProduceTxnMessage() throws InterruptedException, PulsarClientException {
+        String argString = "%s -r 10 -u %s -m %d -txn";
+        String topic = testTopic + UUID.randomUUID();
+        int totalMessage = 500;
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), totalMessage);
+
+        Thread thread = new Thread(() -> {
+            try {
+                log.info("");
+                PerformanceProducer.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().subscriptionName("subName").topic(topic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .enableBatchIndexAcknowledgment(false)
+                .subscribe();
+        for (int i = 0; i < totalMessage; i++) {
+           Message message = consumer.receive();

Review comment:
       ack this message. If ack 500 messages, we can't receive message anymore. I suggest the test message change to 100.

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message);
+        }
+        boolean noMessage = false;
+        for (int i = 0; i < 10; i++) {
+           Message message = consumeFromConsumeTopic.receive(2, TimeUnit.SECONDS);
+           if(message == null){
+               noMessage = true;
+           }
+        }
+        Assert.assertTrue(noMessage);
+    }
+
+
+    @Test
+    public void testProduceTxnMessage() throws InterruptedException, PulsarClientException {
+        String argString = "%s -r 10 -u %s -m %d -txn";
+        String topic = testTopic + UUID.randomUUID();
+        int totalMessage = 500;
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), totalMessage);
+
+        Thread thread = new Thread(() -> {
+            try {
+                log.info("");
+                PerformanceProducer.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().subscriptionName("subName").topic(topic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .enableBatchIndexAcknowledgment(false)
+                .subscribe();
+        for (int i = 0; i < totalMessage; i++) {
+           Message message = consumer.receive();
+           Assert.assertNotNull(message);
+        }
+
+
+    }
+
+    @Test
+    public void testConsumeTxnMessage() throws InterruptedException, PulsarClientException, ExecutionException {
+        String argString = "%s -r 10 -u %s -txn -ss %s -st %s -sp %s -ntxn %d";
+        String subName = "sub";
+        String topic = testTopic + UUID.randomUUID();
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), subName,
+                SubscriptionType.Exclusive, SubscriptionInitialPosition.Earliest, 10);
+        Producer<byte[]> producer = pulsarClient.newProducer().topic(topic).sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        for (int i = 0; i < 505; i++) {

Review comment:
       should add sub before send message, otherwise test will become unstable

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)

Review comment:
       should add sub before send message, otherwise test will become unstable

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message);
+        }
+        boolean noMessage = false;
+        for (int i = 0; i < 10; i++) {
+           Message message = consumeFromConsumeTopic.receive(2, TimeUnit.SECONDS);
+           if(message == null){
+               noMessage = true;
+           }
+        }
+        Assert.assertTrue(noMessage);
+    }
+
+
+    @Test
+    public void testProduceTxnMessage() throws InterruptedException, PulsarClientException {
+        String argString = "%s -r 10 -u %s -m %d -txn";
+        String topic = testTopic + UUID.randomUUID();
+        int totalMessage = 500;
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), totalMessage);
+
+        Thread thread = new Thread(() -> {
+            try {
+                log.info("");
+                PerformanceProducer.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().subscriptionName("subName").topic(topic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .enableBatchIndexAcknowledgment(false)
+                .subscribe();
+        for (int i = 0; i < totalMessage; i++) {

Review comment:
       should add sub before send message, otherwise test will become unstable

##########
File path: pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java
##########
@@ -0,0 +1,225 @@
+/**
+ * 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.testclient;
+
+import com.google.common.collect.Sets;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+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.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfoImpl;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.testng.Assert.fail;
+
+@Slf4j
+public class PerformanceTransactionTest extends MockedPulsarServiceBaseTest {
+    private final String testTenant = "pulsar";
+    private final String testNamespace = "perf";
+    private final String myNamespace = testTenant + "/" + testNamespace;
+    private final String testTopic = "persistent://" + myNamespace + "/test-";
+    private final AtomicInteger lastExitCode = new AtomicInteger(0);
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        super.internalSetup();
+        PerfClientUtils.setExitProcedure(code -> {
+            log.error("JVM exit code is {}", code);
+            if (code != 0) {
+                throw new RuntimeException("JVM should exit with code " + code);
+            }
+        });
+        // Setup namespaces
+        admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
+        admin.tenants().createTenant(NamespaceName.SYSTEM_NAMESPACE.getTenant(),
+                new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test")));
+        admin.namespaces().createNamespace(myNamespace, Sets.newHashSet("test"));
+        admin.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString());
+        admin.topics().createPartitionedTopic(TopicName.TRANSACTION_COORDINATOR_ASSIGN.toString(), 1);
+    }
+
+    @AfterMethod(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+        int exitCode = lastExitCode.get();
+        if (exitCode != 0) {
+            fail("Unexpected JVM exit code "+exitCode);
+        }
+    }
+
+    @Test
+    public void testTxnPerf() throws Exception {
+        String argString = "--topics-c %s --topics-p %s -threads 5 -ntxn 500 -u %s -ss %s -np 1 -au %s";
+        String testConsumeTopic = testTopic + UUID.randomUUID().toString();
+        String testProduceTopic = testTopic + UUID.randomUUID().toString();
+        String testSub = "testSub";
+        admin.topics().createPartitionedTopic(testConsumeTopic, 1);
+        String args = String.format(argString, testConsumeTopic, testProduceTopic,
+                pulsar.getBrokerServiceUrl(), testSub, new URL(pulsar.getWebServiceAddress()));
+
+
+        PulsarClient pulsarClient = PulsarClient.builder()
+                .enableTransaction(true)
+                .serviceUrl(pulsar.getBrokerServiceUrl())
+                .connectionsPerBroker(100)
+                .statsInterval(0, TimeUnit.SECONDS)
+                .build();
+        Producer<byte[]> produceToConsumeTopic = pulsarClient.newProducer(Schema.BYTES)
+                .producerName("perf-transaction-producer")
+                .sendTimeout(0, TimeUnit.SECONDS)
+                .topic(testConsumeTopic)
+                .create();
+
+        CountDownLatch countDownLatch = new CountDownLatch(500);
+        for (int i = 0; i < 510
+                ; i++) {
+            produceToConsumeTopic.newMessage().value(("testConsume " + i).getBytes()).sendAsync().thenRun(() -> {
+                countDownLatch.countDown();
+            });
+        }
+
+        countDownLatch.await();
+
+        Thread thread = new Thread(() -> {
+            try {
+                PerformanceTransaction.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumeFromConsumeTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-consumeVerify")
+                .topic(testConsumeTopic)
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        Consumer<byte[]> consumeFromProduceTopic = pulsarClient.newConsumer(Schema.BYTES)
+                .consumerName("perf-transaction-produceVerify")
+                .topic(testProduceTopic)
+                .subscriptionName(testSub)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe();
+        for (int i = 0; i < 5; i++) {
+            Message message = consumeFromProduceTopic.receive(2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message);
+        }
+        boolean noMessage = false;
+        for (int i = 0; i < 10; i++) {
+           Message message = consumeFromConsumeTopic.receive(2, TimeUnit.SECONDS);
+           if(message == null){
+               noMessage = true;
+           }
+        }
+        Assert.assertTrue(noMessage);
+    }
+
+
+    @Test
+    public void testProduceTxnMessage() throws InterruptedException, PulsarClientException {
+        String argString = "%s -r 10 -u %s -m %d -txn";
+        String topic = testTopic + UUID.randomUUID();
+        int totalMessage = 500;
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), totalMessage);
+
+        Thread thread = new Thread(() -> {
+            try {
+                log.info("");
+                PerformanceProducer.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().subscriptionName("subName").topic(topic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .enableBatchIndexAcknowledgment(false)
+                .subscribe();
+        for (int i = 0; i < totalMessage; i++) {
+           Message message = consumer.receive();
+           Assert.assertNotNull(message);
+        }
+
+
+    }
+
+    @Test
+    public void testConsumeTxnMessage() throws InterruptedException, PulsarClientException, ExecutionException {
+        String argString = "%s -r 10 -u %s -txn -ss %s -st %s -sp %s -ntxn %d";
+        String subName = "sub";
+        String topic = testTopic + UUID.randomUUID();
+        String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), subName,
+                SubscriptionType.Exclusive, SubscriptionInitialPosition.Earliest, 10);
+        Producer<byte[]> producer = pulsarClient.newProducer().topic(topic).sendTimeout(0, TimeUnit.SECONDS)
+                .create();
+
+        for (int i = 0; i < 505; i++) {
+            producer.newMessage().send();
+        }
+        Thread thread = new Thread(() -> {
+            try {
+                log.info("");
+                PerformanceConsumer.main(args.split(" "));
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+        thread.start();
+        thread.join();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().subscriptionName(subName).topic(topic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionType(SubscriptionType.Exclusive)
+                .enableBatchIndexAcknowledgment(false)
+               .subscribe();
+        boolean noMessageAgain = false;
+        for (int i = 0; i < 6; i++) {
+            Message message = consumer.receive(2, TimeUnit.SECONDS);

Review comment:
       we can receive 5 message and can't receive more than 5 messages.

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -340,9 +344,123 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
 
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments.isEnableTransaction,
+                arguments.transactionTimeout);
+
+        AtomicLong messageAckedCount = new AtomicLong();
+        Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction);
+        Thread thread = Thread.currentThread();
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+                if(arguments.isEnableTransaction){
+                    try {
+                        messageReceiveLimiter.acquire();
+                    }catch (InterruptedException e){
+                        log.error("Got error: ", e);
+                    }
+                    }
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                if(arguments.totalNumTxn > 0) {
+                    if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= arguments.totalNumTxn) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {

Review comment:
       why use repeat arguments.isEnableTransaction




-- 
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] Anonymitaet commented on pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
Anonymitaet commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-913923207


   @liangyepianzhou Thanks for your contribution. Please do not forget to add docs accordingly to allow users to know your great code changes. And you can ping me to review the docs, 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] congbobo184 commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r715271777



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -304,38 +379,87 @@ public static void main(String[] args) throws Exception {
 
             long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
             if (latencyMillis >= 0) {
-                recorder.recordValue(latencyMillis);
-                cumulativeRecorder.recordValue(latencyMillis);
+                 recorder.recordValue(latencyMillis);
+                 cumulativeRecorder.recordValue(latencyMillis);
+            }
+            if (arguments.isEnableTransaction) {
+            consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                    totalMessageAck.increment();
+                    messageAck.increment();
+                }).exceptionally(throwable ->{
+                    log.error("Ack message {} failed with exception", msg, throwable);
+                    totalMessageAckFailed.increment();
+                    return null;
+                });
+            } else {
+                consumer.acknowledgeAsync(msg).thenRun(()->{
+                    totalMessageAck.increment();
+                    messageAck.increment();
+                }
+                ).exceptionally(throwable ->{
+                            log.error("Ack message {} failed with exception", msg, throwable);
+                            totalMessageAckFailed.increment();
+                            return null;
+                        }
+                );
             }
-
-            consumer.acknowledgeAsync(msg);
-
             if(arguments.poolMessages) {
                 msg.release();
             }
+            if (arguments.isEnableTransaction
+                    && messageAckedCount.incrementAndGet() == arguments.numMessagesPerTransaction) {
+                Transaction transaction = atomicReference.get();
+                AtomicBoolean updateTransaction = new AtomicBoolean(true);
+                while (true) {
+                    pulsarClient.newTransaction().withTransactionTimeout(arguments.transactionTimeout,

Review comment:
       why dont use `buildTransaction`

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -284,14 +320,53 @@ public static void main(String[] args) throws Exception {
         final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null;
         long startTime = System.nanoTime();
         long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+
+        ClientBuilder clientBuilder = PulsarClient.builder() //
+                .enableTransaction(arguments.isEnableTransaction)
+                .serviceUrl(arguments.serviceURL) //
+                .connectionsPerBroker(arguments.maxConnections) //
+                .statsInterval(arguments.statsIntervalSeconds, TimeUnit.SECONDS) //
+                .ioThreads(arguments.ioThreads) //
+                .enableBusyWait(arguments.enableBusyWait)
+                .tlsTrustCertsFilePath(arguments.tlsTrustCertsFilePath);
+        if (isNotBlank(arguments.authPluginClassName)) {
+            clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams);
+        }
+
+        if (arguments.tlsAllowInsecureConnection != null) {
+            clientBuilder.allowTlsInsecureConnection(arguments.tlsAllowInsecureConnection);
+        }
+
+        if (isNotBlank(arguments.listenerName)) {
+            clientBuilder.listenerName(arguments.listenerName);
+        }
+        PulsarClient pulsarClient = clientBuilder.build();
+
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        AtomicLong messageAckedCount = new AtomicLong();
+        Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction);
+        Thread thread = Thread.currentThread();
         MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+            try {
+            if(arguments.isEnableTransaction){
+                    messageReceiveLimiter.acquire();
+                }

Review comment:
       format error

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -680,20 +791,45 @@ private static void runProducer(int producerId,
             if (null != client) {
                 try {
                     client.close();
+                    PerfClientUtils.exit(-1);
                 } catch (PulsarClientException e) {
                     log.error("Failed to close test client", e);
                 }
             }
         }
     }
 
-    private static void printAggregatedThroughput(long start) {
+    private static void printAggregatedThroughput(long start, Arguments arguments) {
         double elapsed = (System.nanoTime() - start) / 1e9;
         double rate = totalMessagesSent.sum() / elapsed;
         double throughput = totalBytesSent.sum() / elapsed / 1024 / 1024 * 8;
+        long totalTxnSuccess = 0;
+        long totalTxnFail = 0;
+        double rateOpenTxn = 0;
+        long numTransactionOpenFailed = 0;
+        long numTransactionOpenSuccess = 0;
+        if (arguments.isEnableTransaction) {
+            totalTxnSuccess = totalEndTxnOpSuccessNum.sum();
+            totalTxnFail = totalEndTxnOpFailNum.sum();
+            rateOpenTxn = elapsed / (totalTxnFail + totalTxnSuccess);
+            numTransactionOpenFailed = totalNumTxnOpenTxnFail.sum();
+            numTransactionOpenSuccess = totalNumTxnOpenTxnSuccess.sum();
+        }
+
+        if(arguments.isEnableTransaction){
+            log.info("--- Transaction : {} transaction end successfully --- {} transaction end failed "
+                            + "--- {} transaction open successfully --- {} transaction open failed "
+                            + "--- {} Txn/s",
+                    totalTxnSuccess,
+                    totalTxnFail,
+                    numTransactionOpenSuccess,
+                    numTransactionOpenFailed,
+                    totalFormat.format(rateOpenTxn));
+        }
         log.info(
-            "Aggregated throughput stats --- {} records sent --- {} msg/s --- {} Mbit/s",
-            totalMessagesSent,
+            "Aggregated throughput stats --- {} records sent --- {} records send failed --- {} msg/s --- {} Mbit/s ",

Review comment:
       records sent --- {} msg/s

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,717 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                LongAdder messageSend = new LongAdder();
+                LongAdder messageReceived = new LongAdder();
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        try {
+                            Transaction transaction = atomicReference.get();
+                            for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                while(true) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {

Review comment:
       every sub need ack the number of `numMessagesReceivedPerTransaction ` messages

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -634,10 +685,15 @@ private static void runProducer(int producerId,
                     } else {
                         payloadData = payloadBytes;
                     }
-
-                    TypedMessageBuilder<byte[]> messageBuilder = producer.newMessage()
-                            .value(payloadData);
-                    if (arguments.delay >0) {
+                    TypedMessageBuilder<byte[]> messageBuilder;
+                    if (arguments.isEnableTransaction) {
+                        messageBuilder = producer.newMessage(transaction)

Review comment:
       this transaction should get under 
   
   ```
                       if(arguments.isEnableTransaction && arguments.numMessagesPerTransaction > 0){
                           numMsgPerTxnLimit.acquire();
                       }
   ```

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -454,25 +482,37 @@ public static void main(String[] args) throws Exception {
             long now = System.nanoTime();
             double elapsed = (now - oldTime) / 1e9;
             long total = totalMessagesSent.sum();
+            long totalTxnOpSuccess = 0;
+            long totalTxnOpFail = 0;
+            double rateOpenTxn = 0;
+            if (arguments.isEnableTransaction) {
+                totalTxnOpSuccess = totalEndTxnOpSuccessNum.sum();
+                totalTxnOpFail = totalEndTxnOpFailNum.sum();
+                rateOpenTxn = numTxnOp.sumThenReset() / elapsed;
+            }
             double rate = messagesSent.sumThenReset() / elapsed;
-            double failureRate = messagesFailed.sumThenReset() / elapsed;
             double throughput = bytesSent.sumThenReset() / elapsed / 1024 / 1024 * 8;
 
             reportHistogram = recorder.getIntervalHistogram(reportHistogram);
 
+            if (arguments.isEnableTransaction) {
+                log.info("--- Transaction : {} transaction end successfully ---{} transaction end failed "
+                                + "--- {} Txn/s",
+                        totalTxnOpSuccess, totalTxnOpFail, totalFormat.format(rateOpenTxn));
+            }
             log.info(
-                    "Throughput produced: {} msg --- {} msg/s --- {} Mbit/s --- failure {} msg/s --- Latency: mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
+                    "Throughput produced: {} msg --- {} msg/s --- {} Mbit/s  "
+                            + "--- Latency: mean: "
+                            + "{} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                     intFormat.format(total),
                     throughputFormat.format(rate), throughputFormat.format(throughput),
-                    throughputFormat.format(failureRate),

Review comment:
       why delete this?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -665,13 +721,68 @@ private static void runProducer(int producerId,
                         if (ex.getCause() instanceof ArrayIndexOutOfBoundsException) {
                             return null;
                         }
-                        log.warn("Write error on message", ex);
-                        messagesFailed.increment();
+                        log.warn("Write message error with exception", ex);
+                        totalMessagesSendFailed.increment();
                         if (arguments.exitOnFailure) {
                             PerfClientUtils.exit(-1);
                         }
                         return null;
                     });
+                    if (arguments.isEnableTransaction
+                            && numMessageSend.incrementAndGet() == arguments.numMessagesPerTransaction) {
+                        try {
+                            AtomicBoolean updateTransaction = new AtomicBoolean(true);
+
+                            while(true) {
+                                pulsarClient.newTransaction().withTransactionTimeout(arguments.transactionTimeout,

Review comment:
       use 'buildTransaction', same as above PerformanceConsumer

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -304,38 +379,87 @@ public static void main(String[] args) throws Exception {
 
             long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
             if (latencyMillis >= 0) {
-                recorder.recordValue(latencyMillis);
-                cumulativeRecorder.recordValue(latencyMillis);
+                 recorder.recordValue(latencyMillis);
+                 cumulativeRecorder.recordValue(latencyMillis);
+            }
+            if (arguments.isEnableTransaction) {
+            consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                    totalMessageAck.increment();
+                    messageAck.increment();
+                }).exceptionally(throwable ->{
+                    log.error("Ack message {} failed with exception", msg, throwable);
+                    totalMessageAckFailed.increment();
+                    return null;
+                });
+            } else {
+                consumer.acknowledgeAsync(msg).thenRun(()->{
+                    totalMessageAck.increment();
+                    messageAck.increment();
+                }
+                ).exceptionally(throwable ->{
+                            log.error("Ack message {} failed with exception", msg, throwable);
+                            totalMessageAckFailed.increment();
+                            return null;
+                        }
+                );
             }
-
-            consumer.acknowledgeAsync(msg);
-
             if(arguments.poolMessages) {
                 msg.release();
             }
+            if (arguments.isEnableTransaction
+                    && messageAckedCount.incrementAndGet() == arguments.numMessagesPerTransaction) {
+                Transaction transaction = atomicReference.get();
+                AtomicBoolean updateTransaction = new AtomicBoolean(true);
+                while (true) {
+                    pulsarClient.newTransaction().withTransactionTimeout(arguments.transactionTimeout,
+                            TimeUnit.SECONDS).build().thenAccept(newTransaction -> {
+                        atomicReference.compareAndSet(transaction, newTransaction);
+                        totalNumTxnOpenTxnSuccess.increment();
+                        if (arguments.isCommitTransaction) {
+                            transaction.commit()
+                                    .thenRun(() -> {
+                                        totalEndTxnOpSuccessNum.increment();
+                                        numTxnOp.increment();
+                                        totalNumTxnOp.increment();
+                                    })
+                                    .exceptionally(exception -> {
+                                        log.error("Commit transaction failed with exception : ", exception);
+                                        totalEndTxnOpFailNum.increment();
+                                        totalNumTxnOp.increment();
+                                        return null;
+                                    });
+                        } else {
+                            transaction.abort().thenRun(() -> {
+                                log.info("Abort transaction {}", transaction.getTxnID().toString());
+                                totalEndTxnOpSuccessNum.increment();
+                                numTxnOp.increment();
+                                totalNumTxnOp.increment();
+                            }).exceptionally(exception -> {
+                                log.error("Commit transaction {} failed with exception",
+                                        transaction.getTxnID().toString(),
+                                        exception);
+                                totalEndTxnOpFailNum.increment();
+                                return null;
+                            });
+                        }
+                        messageAckedCount.set(0);
+                        messageReceiveLimiter.release(arguments.numMessagesPerTransaction);
+                    }).exceptionally(exception -> {
+                        log.error("Failed to new transaction with exception:", exception);
+                        updateTransaction.set(false);
+                        totalEndTxnOpFailNum.increment();
+                        return null;
+                    });
+                    if(updateTransaction.get()){

Review comment:
       This code doesn't seem to work

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -304,38 +379,87 @@ public static void main(String[] args) throws Exception {
 
             long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
             if (latencyMillis >= 0) {
-                recorder.recordValue(latencyMillis);
-                cumulativeRecorder.recordValue(latencyMillis);
+                 recorder.recordValue(latencyMillis);
+                 cumulativeRecorder.recordValue(latencyMillis);
+            }
+            if (arguments.isEnableTransaction) {
+            consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {

Review comment:
       Need to have spaces in the if code block

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -304,38 +379,87 @@ public static void main(String[] args) throws Exception {
 
             long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
             if (latencyMillis >= 0) {
-                recorder.recordValue(latencyMillis);

Review comment:
       don't change this code

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,717 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                LongAdder messageSend = new LongAdder();
+                LongAdder messageReceived = new LongAdder();
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        try {
+                            Transaction transaction = atomicReference.get();
+                            for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                while(true) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                            break;
+                                        }
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();

Review comment:
       receive() don't throw exception

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,717 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                LongAdder messageSend = new LongAdder();
+                LongAdder messageReceived = new LongAdder();
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        try {
+                            Transaction transaction = atomicReference.get();
+                            for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                while(true) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                            break;
+                                        }
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                            log.info("Receive message {} ", message);
+                                        } catch (Exception e) {
+                                            log.error("{} can`t receive message in 2 sec with exception",
+                                                    consumer, e);
+                                        }
+
+                                        messageReceived.increment();
+                                        long receiveTime = System.nanoTime();
+                                        if (arguments.isEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                    }
+                                }
+                                messageReceived.reset();
+                            }
+
+                            for(Producer<byte[]> producer : producers){
+                                while (true){
+                                    if(messageSend.sum() >= arguments.numMessagesProducedPerTransaction){
+                                        break;
+                                    }
+                                    long sendTime = System.nanoTime();
+                                    messageSend.increment();
+                                    if (arguments.isEnableTransaction) {
+                                        producer.newMessage(transaction).value(payloadBytes)
+                                                .sendAsync().thenRun(() -> {
+                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                    System.nanoTime() - sendTime);
+                                            messageSendRecorder.recordValue(latencyMicros);
+                                            messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Send transaction message failed with exception : ", exception);
+                                            numMessagesSendFailed.increment();
+                                            return null;
+                                        });
+                                    } else {
+                                        producer.newMessage().value(payloadBytes)
+                                                .sendAsync().thenRun(() -> {
+                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                    System.nanoTime() - sendTime);
+                                            messageSendRecorder.recordValue(latencyMicros);
+                                            messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Send message failed with exception : ", exception);
+                                            numMessagesSendFailed.increment();
+                                            return null;
+                                        });
+                                    }
+                                }
+                                messageSend.reset();
+                            }
+
+                            if(rateLimiter != null){
+                                rateLimiter.tryAcquire();
+                            }
+                            if (arguments.isEnableTransaction) {
+                                if (arguments.isCommitTransaction) {
+                                    log.info("Committing transaction {}", transaction.getTxnID().toString());
+                                    transaction.commit()
+                                            .thenRun(()->{
+                                                totalNumEndTxnOpSuccess.increment();
+                                                log.info("Committed transaction {}", transaction.getTxnID().toString());
+                                            })
+                                            .exceptionally(exception -> {
+                                                if(exception instanceof InterruptedException && ! executing.get()){
+                                                    return null;
+                                                }
+                                                log.error("Commit transaction {} failed with exception",
+                                                        transaction.getTxnID().toString(),
+                                                        exception);
+                                                totalNumEndTxnOpFailed.increment();
+                                                return null;
+                                            });
+                                } else {
+                                    log.info("Aborting transaction {}", transaction.getTxnID().toString());
+                                    transaction.abort().thenRun(() -> {
+                                        log.info("Abort transaction {}", transaction.getTxnID().toString());
+                                        totalNumEndTxnOpSuccess.increment();
+                                    }).exceptionally(exception -> {
+                                        if(exception instanceof InterruptedException && ! executing.get()){
+                                            return null;
+                                        }
+                                        log.error("Commit transaction {} failed with exception",
+                                                transaction.getTxnID().toString(),
+                                                exception);
+                                        totalNumEndTxnOpFailed.increment();
+                                        return null;
+                                    });
+                                }
+                                AtomicBoolean updateTransaction = new AtomicBoolean(true);
+
+                                while(true) {
+                                    atomicReference.compareAndSet(transaction, client.newTransaction()

Review comment:
       same as PerformanceProducer and PerformanceConsumer

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,717 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                LongAdder messageSend = new LongAdder();
+                LongAdder messageReceived = new LongAdder();
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        try {
+                            Transaction transaction = atomicReference.get();
+                            for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                while(true) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){

Review comment:
       this code should delete?

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+                 RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    while(true) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                            break;
+                                        }
+                                        for (Consumer<byte[]> consumer : subscriptions) {
+                                            if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                                break;
+                                            }
+                                            Message message = null;
+                                            try {
+                                                message = consumer.receive(2, TimeUnit.SECONDS);
+                                                log.info("Receive message {} ", message);
+                                            } catch (Exception e) {
+                                                log.error("{} can`t receive message in 2 sec with exception {}",
+                                                        consumer, e);
+                                            }
+
+                                            messageReceived.increment();
+                                            long receiveTime = System.nanoTime();
+                                            if (arguments.isEnableTransaction) {
+                                                consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                        .thenRun(() -> {
+                                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                                    System.nanoTime() - receiveTime);
+                                                            messageAckRecorder.recordValue(latencyMicros);
+                                                            messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                        }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {
+                                                        return null;
+                                                    }
+                                                    log.error(
+                                                            "Ack message failed with transaction {} throw exception {}",
+                                                            transaction, exception);
+                                                    numMessagesAckFailed.increment();
+                                                    return null;
+                                                });
+                                            } else {
+                                                consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                    long latencyMicros = NANOSECONDS.toMicros(
+                                                            System.nanoTime() - receiveTime);
+                                                    messageAckRecorder.recordValue(latencyMicros);
+                                                    messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                }).exceptionally(exception -> {
+                                                    if (exception instanceof InterruptedException && !executing.get()) {
+                                                        return null;
+                                                    }
+                                                    log.error(
+                                                            "Ack message failed with transaction {} throw exception {}",
+                                                            transaction, exception);
+                                                    numMessagesAckFailed.increment();
+                                                    return null;
+                                                });
+                                            }
+                                        }
+                                    }
+                                    messageReceived.reset();
+                                }
+
+                                for(Producer<byte[]> producer : producers){
+                                    while (true){
+                                    if(messageSend.sum() >= arguments.numMessagesProducedPerTransaction){

Review comment:
       use for loop is better

##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,717 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnFail = new LongAdder();
+    private static final LongAdder totalNumTxnOpenTxnSuccess = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads,
+                arguments.numTestThreads,
+                0L, TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<Runnable>());
+
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+
+            RateLimiter rateLimiter = arguments.openTxnRate > 0
+                    ? RateLimiter.create(arguments.openTxnRate)
+                    : null;
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                LongAdder messageSend = new LongAdder();
+                LongAdder messageReceived = new LongAdder();
+                executorService.submit(() -> {
+                    //The producer and consumer clients are built in advance, and then this thread is
+                    //responsible for the production and consumption tasks of the transaction through the loop.
+                    //A thread may perform tasks of multiple transactions in a traversing manner.
+                    List<Producer<byte[]>> producers = null;
+                    List<List<Consumer<byte[]>>> consumers = null;
+                    try {
+                        producers = buildProducers(client, arguments);
+                        consumers = buildConsumer(client, arguments);
+                    } catch (Exception e) {
+                        log.error("Failed to build Producer/Consumer with exception : ", e);
+                        executorService.shutdownNow();
+                        PerfClientUtils.exit(-1);
+                    }
+                    AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                    //The while loop has no break, and finally ends the execution through the shutdownNow of
+                    //0the executorService
+                    while (true) {
+                        try {
+                            Transaction transaction = atomicReference.get();
+                            for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                while(true) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if (messageReceived.sum() == arguments.numMessagesReceivedPerTransaction) {
+                                            break;
+                                        }
+                                        Message message = null;
+                                        try {
+                                            message = consumer.receive();
+                                            log.info("Receive message {} ", message);
+                                        } catch (Exception e) {
+                                            log.error("{} can`t receive message in 2 sec with exception",
+                                                    consumer, e);
+                                        }
+
+                                        messageReceived.increment();
+                                        long receiveTime = System.nanoTime();
+                                        if (arguments.isEnableTransaction) {
+                                            consumer.acknowledgeAsync(message.getMessageId(), transaction)
+                                                    .thenRun(() -> {
+                                                        long latencyMicros = NANOSECONDS.toMicros(
+                                                                System.nanoTime() - receiveTime);
+                                                        messageAckRecorder.recordValue(latencyMicros);
+                                                        messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                                    }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        } else {
+                                            consumer.acknowledgeAsync(message).thenRun(() -> {
+                                                long latencyMicros = NANOSECONDS.toMicros(
+                                                        System.nanoTime() - receiveTime);
+                                                messageAckRecorder.recordValue(latencyMicros);
+                                                messageAckCumulativeRecorder.recordValue(latencyMicros);
+                                            }).exceptionally(exception -> {
+                                                if (exception instanceof InterruptedException && !executing.get()) {
+                                                    return null;
+                                                }
+                                                log.error(
+                                                        "Ack message failed with transaction {} throw exception",
+                                                        transaction, exception);
+                                                numMessagesAckFailed.increment();
+                                                return null;
+                                            });
+                                        }
+                                    }
+                                }
+                                messageReceived.reset();
+                            }
+
+                            for(Producer<byte[]> producer : producers){
+                                while (true){
+                                    if(messageSend.sum() >= arguments.numMessagesProducedPerTransaction){
+                                        break;
+                                    }
+                                    long sendTime = System.nanoTime();
+                                    messageSend.increment();
+                                    if (arguments.isEnableTransaction) {
+                                        producer.newMessage(transaction).value(payloadBytes)
+                                                .sendAsync().thenRun(() -> {
+                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                    System.nanoTime() - sendTime);
+                                            messageSendRecorder.recordValue(latencyMicros);
+                                            messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Send transaction message failed with exception : ", exception);
+                                            numMessagesSendFailed.increment();
+                                            return null;
+                                        });
+                                    } else {
+                                        producer.newMessage().value(payloadBytes)
+                                                .sendAsync().thenRun(() -> {
+                                            long latencyMicros = NANOSECONDS.toMicros(
+                                                    System.nanoTime() - sendTime);
+                                            messageSendRecorder.recordValue(latencyMicros);
+                                            messageSendRCumulativeRecorder.recordValue(latencyMicros);
+                                        }).exceptionally(exception -> {
+                                            if(exception instanceof InterruptedException && ! executing.get()){
+                                                return null;
+                                            }
+                                            log.error("Send message failed with exception : ", exception);
+                                            numMessagesSendFailed.increment();
+                                            return null;
+                                        });
+                                    }
+                                }
+                                messageSend.reset();
+                            }
+
+                            if(rateLimiter != null){

Review comment:
       this should process after current transaction commit or abort 




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r713574413



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,685 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumEndTxnOp = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpFailed = new LongAdder();
+    private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder();
+    private static final LongAdder numTxnOp = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+
+
+    private static final Recorder messageAckRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageAckCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static final Recorder messageSendRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static final Recorder messageSendRCumulativeRecorder =
+            new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-commit"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {

Review comment:
       This is for the while loop that prints the log below to be executed




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711584742



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java
##########
@@ -597,7 +634,11 @@ private static void runProducer(int producerId,
             }
             // Send messages on all topics/producers
             long totalSent = 0;
+            AtomicReference<Transaction> transactionAtomicReference = buildTransaction(client, arguments);
+            AtomicLong numMessageSend = new AtomicLong(0);
+            Semaphore semaphore = new Semaphore(arguments.numMessagesPerTransaction);
             while (true) {
+                Transaction transaction = transactionAtomicReference.get();

Review comment:
       If transaction is disable, there transaction will be null without exception.  In terms of results, it is no different from adding a judgment. I didn’t add that judgment in order to simplify




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711591130



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -284,7 +311,36 @@ public static void main(String[] args) throws Exception {
         final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null;
         long startTime = System.nanoTime();
         long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+
+        ClientBuilder clientBuilder = PulsarClient.builder() //
+                .enableTransaction(arguments.isEnableTransaction)
+                .serviceUrl(arguments.serviceURL) //
+                .connectionsPerBroker(arguments.maxConnections) //
+                .statsInterval(arguments.statsIntervalSeconds, TimeUnit.SECONDS) //
+                .ioThreads(arguments.ioThreads) //
+                .enableBusyWait(arguments.enableBusyWait)
+                .tlsTrustCertsFilePath(arguments.tlsTrustCertsFilePath);
+        if (isNotBlank(arguments.authPluginClassName)) {
+            clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams);
+        }
+
+        if (arguments.tlsAllowInsecureConnection != null) {
+            clientBuilder.allowTlsInsecureConnection(arguments.tlsAllowInsecureConnection);
+        }
+
+        if (isNotBlank(arguments.listenerName)) {
+            clientBuilder.listenerName(arguments.listenerName);
+        }
+        PulsarClient pulsarClient = clientBuilder.build();
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments);
+
+        LongAdder messageAckedCount = new LongAdder();

Review comment:
       Indeed, it’s great to use AtomicLong 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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-927265990


   /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] liangyepianzhou commented on a change in pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r716157771



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java
##########
@@ -340,9 +344,123 @@ public static void main(String[] args) throws Exception {
         if (isNotBlank(arguments.listenerName)) {
             clientBuilder.listenerName(arguments.listenerName);
         }
-
         PulsarClient pulsarClient = clientBuilder.build();
 
+        AtomicReference<Transaction> atomicReference = buildTransaction(pulsarClient, arguments.isEnableTransaction,
+                arguments.transactionTimeout);
+
+        AtomicLong messageAckedCount = new AtomicLong();
+        Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction);
+        Thread thread = Thread.currentThread();
+        MessageListener<ByteBuffer> listener = (consumer, msg) -> {
+                if(arguments.isEnableTransaction){
+                    try {
+                        messageReceiveLimiter.acquire();
+                    }catch (InterruptedException e){
+                        log.error("Got error: ", e);
+                    }
+                    }
+                if (arguments.testTime > 0) {
+                    if (System.nanoTime() > testEndTime) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                if(arguments.totalNumTxn > 0) {
+                    if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= arguments.totalNumTxn) {
+                        log.info("------------------- DONE -----------------------");
+                        printAggregatedStats();
+                        PerfClientUtils.exit(0);
+                        thread.interrupt();
+                    }
+                }
+                messagesReceived.increment();
+                bytesReceived.add(msg.size());
+
+                totalMessagesReceived.increment();
+                totalBytesReceived.add(msg.size());
+
+                if (limiter != null) {
+                    limiter.acquire();
+                }
+
+                long latencyMillis = System.currentTimeMillis() - msg.getPublishTime();
+                if (latencyMillis >= 0) {
+                    recorder.recordValue(latencyMillis);
+                    cumulativeRecorder.recordValue(latencyMillis);
+                }
+                if (arguments.isEnableTransaction) {
+                    consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> {
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    }).exceptionally(throwable ->{
+                        log.error("Ack message {} failed with exception", msg, throwable);
+                        totalMessageAckFailed.increment();
+                        return null;
+                    });
+                } else {
+                    consumer.acknowledgeAsync(msg).thenRun(()->{
+                        totalMessageAck.increment();
+                        messageAck.increment();
+                    }
+                    ).exceptionally(throwable ->{
+                                log.error("Ack message {} failed with exception", msg, throwable);
+                                totalMessageAckFailed.increment();
+                                return null;
+                            }
+                    );
+                }
+                if(arguments.poolMessages) {
+                    msg.release();
+                }
+                if (arguments.isEnableTransaction

Review comment:
       It is not good to write the Ack of the message and the commit of the transaction together. In this way if is too far away from else, it feels that the segmentation is not very good




-- 
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] liangyepianzhou commented on a change in pull request #11933: Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on a change in pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#discussion_r711587721



##########
File path: pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java
##########
@@ -0,0 +1,637 @@
+/**
+ * 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.testclient;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Histogram;
+import org.HdrHistogram.HistogramLogWriter;
+import org.HdrHistogram.Recorder;
+import org.apache.curator.shaded.com.google.common.util.concurrent.RateLimiter;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerBuilder;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.transaction.Transaction;
+import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.testclient.utils.PaddingDecimalFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerformanceTransaction {
+
+
+    private static final LongAdder totalNumTransaction = new LongAdder();
+    private static final LongAdder numTransaction = new LongAdder();
+
+    private static final LongAdder numMessagesAckFailed = new LongAdder();
+    private static final LongAdder numMessagesSendFailed = new LongAdder();
+    private static final LongAdder numTransactionCommitFailed = new LongAdder();
+
+    private static Recorder messageAckRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageAckCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+    private static Recorder messageSendRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+    private static Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5);
+
+
+    @Parameters(commandDescription = "Test pulsar transaction performance.")
+    static class Arguments {
+
+        @Parameter(names = {"-h", "--help"}, description = "Help message", help = true)
+        boolean help;
+
+        @Parameter(names = {"--conf-file"}, description = "Configuration file")
+        public String confFile;
+
+        @Parameter(names = "--topics-c", description = "All topics that need ack for a transaction", required =
+                true)
+        public List<String> consumerTopic = Collections.singletonList("test-consume");
+
+        @Parameter(names = "--topics-p", description = "All topics that need produce for a transaction",
+                required = true)
+        public List<String> producerTopic = Collections.singletonList("test-produce");
+
+        @Parameter(names = {"-threads", "--num-test-threads"}, description = "Number of test threads."
+                + "This thread is for new transaction and ack the consumerTopic message and produce message to "
+                + "producerTopic then commit or abort this transaction. "
+                + "Increasing the number of threads will increase the parallelism of the performance test, "
+                + "thereby increasing the intensity of the stress test.")
+        public int numTestThreads = 1;
+
+        @Parameter(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL")
+        public String adminURL;
+
+        @Parameter(names = {"-u", "--service-url"}, description = "Pulsar Service URL")
+        public String serviceURL;
+
+        @Parameter(names = {"-np",
+                "--partitions"}, description = "Create partitioned topics with the given number of partitions, set 0 "
+                + "to not try to create the topic")
+        public Integer partitions = null;
+
+        @Parameter(names = {"-c",
+                "--max-connections"}, description = "Max number of TCP connections to a single broker")
+        public int maxConnections = 100;
+
+        @Parameter(names = {"-time",
+                "--test-duration"}, description = "Test duration in secs. If 0, it will keep publishing")
+        public long testTime = 0;
+
+        @Parameter(names = {"-ioThreads", "--num-io-threads"}, description = "Set the number of threads to be " +
+                "used for handling connections to brokers, default is 1 thread")
+        public int ioThreads = 1;
+
+        @Parameter(names = {"-ss",
+                "--subscriptions"}, description = "A list of subscriptions to consume on (e.g. sub1,sub2)")
+        public List<String> subscriptions = Collections.singletonList("sub");
+
+        @Parameter(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)")
+        public int numSubscriptions = 1;
+
+        @Parameter(names = {"-sp", "--subscription-position"}, description = "Subscription position")
+        private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest;
+
+        @Parameter(names = {"-st", "--subscription-type"}, description = "Subscription type")
+        public SubscriptionType subscriptionType = SubscriptionType.Shared;
+
+        @Parameter(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue")
+        public int receiverQueueSize = 1000;
+
+        @Parameter(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout,"
+                + " and the TimeUnit is second. (Only --txn-enable true can it take effect) ")
+        public long transactionTimeout = 5;
+
+        @Parameter(names = {"-ntxn",
+                "--number-txn"}, description = "Set the number of transaction, if 0, it will keep opening."
+                + "If transaction disable, it means the number of task. The task or transaction will produce or "
+                + "and consume a specified number of messages.")
+        public long numTransactions = 0;
+
+        @Parameter(names = {"-nmp", "--numMessage-perTransaction-produce"},
+                description = "Set the number of messages produced in  a transaction."
+                        + "If transaction disable, it means the number of messages produced in a task.")
+        public int numMessagesProducedPerTransaction = 1;
+
+        @Parameter(names = {"-nmc", "--numMessage-perTransaction-consume"},
+                description = "Set the number of messages consumed in  a transaction."
+                        + "if transaction disable, it means the number of message consumed in a task.")
+        public int numMessagesReceivedPerTransaction = 1;
+
+        @Parameter(names = {"-txn", "--txn-enable"}, description = "Enable or disable transaction")
+        public boolean isEnableTransaction = true;
+
+        @Parameter(names = {"-end"}, description = "Whether to commit or abort the transaction. (Only --txn-enable "
+                + "true can it take effect)")
+        public boolean isCommitTransaction = true;
+
+        @Parameter(names = "-txnRate", description = "Set the rate of transaction/task open, if 0, it will don`t limit")
+        public int openTxnRate = 0;
+    }
+
+    public static void main(String[] args)
+            throws IOException, PulsarAdminException, ExecutionException, InterruptedException {
+        final Arguments arguments = new Arguments();
+        JCommander jc = new JCommander(arguments);
+        jc.setProgramName("pulsar-perf transaction");
+
+        try {
+            jc.parse(args);
+        } catch (ParameterException e) {
+            System.out.println(e.getMessage());
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+        if (arguments.help) {
+            jc.usage();
+            PerfClientUtils.exit(-1);
+        }
+
+
+        if (arguments.confFile != null) {
+            Properties prop = new Properties(System.getProperties());
+            prop.load(new FileInputStream(arguments.confFile));
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("brokerServiceUrl");
+            }
+
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("webServiceUrl");
+            }
+
+            // fallback to previous-version serviceUrl property to maintain backward-compatibility
+            if (arguments.serviceURL == null) {
+                arguments.serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/");
+            }
+
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("webServiceUrl");
+            }
+            if (arguments.adminURL == null) {
+                arguments.adminURL = prop.getProperty("adminURL", "http://localhost:8080/");
+            }
+        }
+
+
+        // Dump config variables
+        PerfClientUtils.printJVMInformation(log);
+
+        ObjectMapper m = new ObjectMapper();
+        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
+        log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments));
+
+        final byte[] payloadBytes = new byte[1024];
+        Random random = new Random(0);
+        for (int i = 0; i < payloadBytes.length; ++i) {
+            payloadBytes[i] = (byte) (random.nextInt(26) + 65);
+        }
+        if (arguments.partitions != null) {
+            PulsarAdminBuilder clientBuilder = PulsarAdmin.builder()
+                    .serviceHttpUrl(arguments.adminURL);
+            try (PulsarAdmin client = clientBuilder.build()) {
+                for (String topic : arguments.producerTopic) {
+                    log.info("Creating  produce partitioned topic {} with {} partitions", topic, arguments.partitions);
+                    try {
+                        client.topics().createPartitionedTopic(topic, arguments.partitions);
+                    } catch (PulsarAdminException.ConflictException alreadyExists) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Topic {} already exists: {}", topic, alreadyExists);
+                        }
+                        PartitionedTopicMetadata partitionedTopicMetadata =
+                                client.topics().getPartitionedTopicMetadata(topic);
+                        if (partitionedTopicMetadata.partitions != arguments.partitions) {
+                            log.error(
+                                    "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}",
+                                    topic, partitionedTopicMetadata.partitions, arguments.partitions);
+                            PerfClientUtils.exit(-1);
+                        }
+                    }
+                }
+            }
+        }
+
+        PulsarClient client =
+                PulsarClient.builder().enableTransaction(arguments.isEnableTransaction)
+                        .serviceUrl(arguments.serviceURL)
+                        .connectionsPerBroker(arguments.maxConnections)
+                        .statsInterval(0, TimeUnit.SECONDS)
+                        .ioThreads(arguments.ioThreads)
+                        .build();
+
+        ExecutorService executorService = Executors.newFixedThreadPool(arguments.numTestThreads);
+        Semaphore testThreadLimit = new Semaphore(arguments.numTestThreads);
+
+        long startTime = System.nanoTime();
+        long testEndTime = startTime + (long) (arguments.testTime * 1e9);
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (arguments.isEnableTransaction) {
+                printTxnAggregatedThroughput(startTime);
+            } else {
+                printAggregatedThroughput(startTime);
+            }
+            printAggregatedStats();
+        }));
+
+        // start perf test
+        AtomicBoolean executing = new AtomicBoolean(true);
+        new Thread(() -> {
+            RateLimiter rateLimiter = null;
+            if(arguments.openTxnRate != 0){
+                rateLimiter = RateLimiter.create( (double)arguments.openTxnRate / arguments.numTestThreads);
+            }
+            for(int i = 0; i < arguments.numTestThreads; i++) {
+                if (testThreadLimit.tryAcquire() && (rateLimiter == null || rateLimiter.tryAcquire())) {
+                    LongAdder messageSend = new LongAdder();
+                    LongAdder messageReceived = new LongAdder();
+                    executorService.submit(() -> {
+                        //The producer and consumer clients are built in advance, and then this thread is
+                        //responsible for the production and consumption tasks of the transaction through the loop.
+                        //A thread may perform tasks of multiple transactions in a traversing manner.
+                        List<Producer<byte[]>> producers = null;
+                        List<List<Consumer<byte[]>>> consumers = null;
+                        try {
+                            producers = buildProducers(client, arguments);
+                            consumers = buildConsumer(client, arguments);
+                        } catch (Exception e) {
+                            log.error("Failed to build Producer/Consumer with exception : " + e);
+                        }
+                        AtomicReference<Transaction> atomicReference = buildTransaction(client, arguments);
+                        //The while loop has no break, and finally ends the execution through the shutdownNow of
+                        //0the executorService
+                        while (true) {
+                            try {
+                                Transaction transaction = atomicReference.get();
+                                for (List<Consumer<byte[]>> subscriptions : consumers) {
+                                    if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){
+                                        break;
+                                    }
+                                    for (Consumer<byte[]> consumer : subscriptions) {
+                                        if(messageReceived.sum() == arguments.numMessagesReceivedPerTransaction){

Review comment:
       Is the number of messages sent or received is not fixed in a transaction? The ```messageReceived``` here is  one per test thread, reset when the transaction is over, a test thread only executes one transaction at the same 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] congbobo184 commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-929815808


   @eolivelli please review again, 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] congbobo184 commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-962788146


   @eolivelli please review again, 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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-963074707


   /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] liangyepianzhou commented on pull request #11933: [Transaction] Add transaction perf

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on pull request #11933:
URL: https://github.com/apache/pulsar/pull/11933#issuecomment-951430447


   /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